You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by iv...@apache.org on 2022/05/31 08:58:21 UTC

[ignite] branch master updated: IGNITE-14636 SQL Calcite: implement STRING_AGG, GROUP_CONCAT and LISTAGG support - Fixes #10023.

This is an automated email from the ASF dual-hosted git repository.

ivandasch pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 63e96487e61 IGNITE-14636 SQL Calcite: implement STRING_AGG, GROUP_CONCAT and LISTAGG support - Fixes #10023.
63e96487e61 is described below

commit 63e96487e61f82ce920f6abd48392bc8c06fa6ab
Author: Ivan Daschinsky <iv...@apache.org>
AuthorDate: Tue May 31 11:14:49 2022 +0300

    IGNITE-14636 SQL Calcite: implement STRING_AGG, GROUP_CONCAT and LISTAGG support - Fixes #10023.
    
    Co-authored-by: Vladimir Ermakov <ve...@gridgain.com>
    Signed-off-by: Ivan Daschinsky <iv...@apache.org>
---
 docs/_docs/SQL/sql-calcite.adoc                    |    2 +-
 modules/calcite/src/main/codegen/config.fmpp       |    4 +-
 .../query/calcite/exec/exp/agg/Accumulator.java    |    6 +-
 .../calcite/exec/exp/agg/AccumulatorWrapper.java   |    4 +-
 .../query/calcite/exec/exp/agg/Accumulators.java   |  622 ++-
 .../calcite/exec/exp/agg/AccumulatorsFactory.java  |   89 +-
 .../query/calcite/externalize/RelJson.java         |    1 +
 .../query/calcite/externalize/RelJsonReader.java   |    6 +-
 .../query/calcite/prepare/IgniteSqlValidator.java  |    4 +-
 .../query/calcite/prepare/PlannerPhase.java        |   11 +-
 .../calcite/sql/fun/IgniteStdSqlOperatorTable.java |    3 +
 .../calcite/sql/generated/IgniteSqlParserImpl.java | 2126 ++++----
 .../generated/IgniteSqlParserImplConstants.java    |  138 +-
 .../generated/IgniteSqlParserImplTokenManager.java | 5694 ++++++++++----------
 .../calcite/integration/StdSqlOperatorsTest.java   |    3 +
 .../aggregate/aggregates/test_aggregate_types.test |   27 +-
 .../aggregates/test_aggregate_types.test_ignore    |  191 -
 .../aggregates/test_aggregate_types_scalar.test    |   16 +
 .../test_aggregate_types_scalar.test_ignored       |  111 -
 .../aggregates/test_distinct_string_agg.test       |   22 +
 .../test_distinct_string_agg.test_ignore           |   17 -
 .../sql/aggregate/aggregates/test_perfect_ht.test  |    9 +
 .../aggregates/test_perfect_ht.test_ignore         |    1 -
 .../sql/aggregate/aggregates/test_scalar_aggr.test |   48 +
 .../aggregates/test_scalar_aggr.test_ignore        |   58 -
 .../sql/aggregate/aggregates/test_string_agg.test  |  153 +
 .../aggregates/test_string_agg.test_ignore         |   91 -
 .../aggregate/aggregates/test_string_agg_big.test  |   16 +
 .../aggregates/test_string_agg_big.test_ignore     |   17 -
 .../aggregates/test_string_agg_many_groups.test    |   22 +
 .../test_string_agg_many_groups.test_slow_ignore   |   23 -
 31 files changed, 4776 insertions(+), 4759 deletions(-)

diff --git a/docs/_docs/SQL/sql-calcite.adoc b/docs/_docs/SQL/sql-calcite.adoc
index f138465c3a0..8cbb2af8db9 100644
--- a/docs/_docs/SQL/sql-calcite.adoc
+++ b/docs/_docs/SQL/sql-calcite.adoc
@@ -159,7 +159,7 @@ The Calcite-based SQL engine currently supports:
 |Group | Functions list
 
 |Aggregate functions
-|`COUNT`, `SUM`, `AVG`, `MIN`, `MAX`, `ANY_VALUE`
+|`COUNT`, `SUM`, `AVG`, `MIN`, `MAX`, `ANY_VALUE`, `LISTAGG`, `GROUP_CONCAT`, `STRING_AGG`
 
 |String functions
 |`UPPER`, `LOWER`, `INITCAP`, `TO_BASE64`, `FROM_BASE64`, `MD5`, `SHA1`, `SUBSTRING`, `LEFT`, `RIGHT`, `REPLACE`, `TRANSLATE`, `CHR`, `CHAR_LENGTH`, `CHARACTER_LENGTH`, `LENGTH`, `CONCAT`, `OVERLAY`, `POSITION`, `ASCII`, `REPEAT`, `SPACE`, `STRCMP`, `SOUNDEX`, `DIFFERENCE`, `REVERSE`, `TRIM`, `LTRIM`, `RTRIM`, `REGEXP_REPLACE`
diff --git a/modules/calcite/src/main/codegen/config.fmpp b/modules/calcite/src/main/codegen/config.fmpp
index f692ad75916..90a0b57d312 100644
--- a/modules/calcite/src/main/codegen/config.fmpp
+++ b/modules/calcite/src/main/codegen/config.fmpp
@@ -67,7 +67,6 @@ data: {
       "COMPUTE"
       "ASYNC"
       "QUERY"
-      "UUID"
     ]
 
     # List of non-reserved keywords to add;
@@ -97,7 +96,6 @@ data: {
       "COMPUTE"
       "ASYNC"
       "QUERY"
-      "UUID"
 
       # The following keywords are reserved in core Calcite,
       # are reserved in some version of SQL,
@@ -519,6 +517,8 @@ data: {
       "SYSTEM"
       "SYSTEM_TIME"
       "SYSTEM_USER"
+      "STRING_AGG"
+      "GROUP_CONCAT"
 #     "TABLE"
 #     "TABLESAMPLE"
       "TEMPORARY"
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulator.java
index a4fff5f131b..749db6a2226 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulator.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulator.java
@@ -26,12 +26,12 @@ import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactor
 /**
  *
  */
-public interface Accumulator extends Serializable {
+public interface Accumulator<Row> extends Serializable {
     /** */
-    void add(Object... args);
+    void add(Row row);
 
     /** */
-    void apply(Accumulator other);
+    void apply(Accumulator<Row> other);
 
     /** */
     Object end();
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorWrapper.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorWrapper.java
index 76e66c2e54e..044803e07c3 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorWrapper.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorWrapper.java
@@ -28,8 +28,8 @@ public interface AccumulatorWrapper<Row> {
     Object end();
 
     /** */
-    void apply(Accumulator accumulator);
+    void apply(Accumulator<Row> accumulator);
 
     /** */
-    Accumulator accumulator();
+    Accumulator<Row> accumulator();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
index b4ce6da261c..9a75e28a908 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
@@ -19,22 +19,27 @@ package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
 
 import java.math.BigDecimal;
 import java.math.MathContext;
+import java.util.ArrayList;
 import java.util.Comparator;
-import java.util.HashSet;
+import java.util.HashMap;
 import java.util.List;
-import java.util.Set;
+import java.util.Map;
+import java.util.Objects;
 import java.util.UUID;
 import java.util.function.Function;
 import java.util.function.Supplier;
 import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 import org.apache.ignite.internal.processors.query.calcite.type.UuidType;
 import org.apache.ignite.internal.util.typedef.F;
 
 import static org.apache.calcite.sql.type.SqlTypeName.ANY;
 import static org.apache.calcite.sql.type.SqlTypeName.BIGINT;
+import static org.apache.calcite.sql.type.SqlTypeName.CHAR;
 import static org.apache.calcite.sql.type.SqlTypeName.DECIMAL;
 import static org.apache.calcite.sql.type.SqlTypeName.DOUBLE;
 import static org.apache.calcite.sql.type.SqlTypeName.INTEGER;
@@ -46,177 +51,244 @@ import static org.apache.calcite.sql.type.SqlTypeName.VARCHAR;
  */
 public class Accumulators {
     /** */
-    public static Supplier<Accumulator> accumulatorFactory(AggregateCall call) {
-        if (!call.isDistinct())
-            return accumulatorFunctionFactory(call);
+    public static <Row> Supplier<Accumulator<Row>> accumulatorFactory(AggregateCall call, ExecutionContext<Row> ctx) {
+        Supplier<Accumulator<Row>> supplier = accumulatorFunctionFactory(call, ctx);
 
-        Supplier<Accumulator> fac = accumulatorFunctionFactory(call);
+        if (call.isDistinct())
+            return () -> new DistinctAccumulator<>(call, ctx.rowHandler(), supplier);
 
-        return () -> new DistinctAccumulator(fac);
+        return supplier;
     }
 
     /** */
-    public static Supplier<Accumulator> accumulatorFunctionFactory(AggregateCall call) {
+    private static <Row> Supplier<Accumulator<Row>> accumulatorFunctionFactory(
+        AggregateCall call,
+        ExecutionContext<Row> ctx
+    ) {
+        RowHandler<Row> hnd = ctx.rowHandler();
+
         switch (call.getAggregation().getName()) {
             case "COUNT":
-                return LongCount.FACTORY;
+                return () -> new LongCount<>(call, hnd);
             case "AVG":
-                return avgFactory(call);
+                return avgFactory(call, hnd);
             case "SUM":
-                return sumFactory(call);
+                return sumFactory(call, hnd);
             case "$SUM0":
-                return sumEmptyIsZeroFactory(call);
+                return sumEmptyIsZeroFactory(call, hnd);
             case "MIN":
-                return minFactory(call);
+                return minFactory(call, hnd);
             case "MAX":
-                return maxFactory(call);
+                return maxFactory(call, hnd);
             case "SINGLE_VALUE":
-                return SingleVal.FACTORY;
+                return () -> new SingleVal<>(call, hnd);
             case "ANY_VALUE":
-                return AnyVal.FACTORY;
+                return () -> new AnyVal<>(call, hnd);
+            case "LISTAGG":
+                return listAggregateSupplier(call, ctx);
             default:
                 throw new AssertionError(call.getAggregation().getName());
         }
     }
 
     /** */
-    private static Supplier<Accumulator> avgFactory(AggregateCall call) {
+    private static <Row> Supplier<Accumulator<Row>> listAggregateSupplier(
+        AggregateCall call,
+        ExecutionContext<Row> ctx
+    ) {
+        RowHandler<Row> hnd = ctx.rowHandler();
+
+        Supplier<Accumulator<Row>> accSup = () -> new ListAggAccumulator<>(call, hnd);
+
+        if (call.getCollation() != null && !call.getCollation().getFieldCollations().isEmpty()) {
+            Comparator<Row> cmp = ctx.expressionFactory().comparator(call.getCollation());
+
+            return () -> new SortingAccumulator<>(accSup, cmp);
+        }
+
+        return accSup;
+    }
+
+    /** */
+    private static <Row> Supplier<Accumulator<Row>> avgFactory(AggregateCall call, RowHandler<Row> hnd) {
         switch (call.type.getSqlTypeName()) {
             case ANY:
                 throw new UnsupportedOperationException("AVG() is not supported for type '" + call.type + "'.");
             case BIGINT:
             case DECIMAL:
-                return DecimalAvg.FACTORY;
+                return () -> new DecimalAvg<>(call, hnd);
             case DOUBLE:
             case REAL:
             case FLOAT:
             case INTEGER:
             default:
-                return DoubleAvg.FACTORY;
+                return () -> new DoubleAvg<>(call, hnd);
         }
     }
 
     /** */
-    private static Supplier<Accumulator> sumFactory(AggregateCall call) {
+    private static <Row> Supplier<Accumulator<Row>> sumFactory(AggregateCall call, RowHandler<Row> hnd) {
         switch (call.type.getSqlTypeName()) {
             case ANY:
                 throw new UnsupportedOperationException("SUM() is not supported for type '" + call.type + "'.");
 
             case BIGINT:
             case DECIMAL:
-                return () -> new Sum(new DecimalSumEmptyIsZero());
+                return () -> new Sum<>(call, new DecimalSumEmptyIsZero<>(call, hnd), hnd);
 
             case DOUBLE:
             case REAL:
             case FLOAT:
-                return () -> new Sum(new DoubleSumEmptyIsZero());
+                return () -> new Sum<>(call, new DoubleSumEmptyIsZero<>(call, hnd), hnd);
 
             case TINYINT:
             case SMALLINT:
             case INTEGER:
             default:
-                return () -> new Sum(new LongSumEmptyIsZero());
+                return () -> new Sum<>(call, new LongSumEmptyIsZero<>(call, hnd), hnd);
         }
     }
 
     /** */
-    private static Supplier<Accumulator> sumEmptyIsZeroFactory(AggregateCall call) {
+    private static <Row> Supplier<Accumulator<Row>> sumEmptyIsZeroFactory(AggregateCall call, RowHandler<Row> hnd) {
         switch (call.type.getSqlTypeName()) {
             case ANY:
                 throw new UnsupportedOperationException("SUM() is not supported for type '" + call.type + "'.");
 
             case BIGINT:
             case DECIMAL:
-                return DecimalSumEmptyIsZero.FACTORY;
+                return () -> new DecimalSumEmptyIsZero<>(call, hnd);
 
             case DOUBLE:
             case REAL:
             case FLOAT:
-                return DoubleSumEmptyIsZero.FACTORY;
+                return () -> new DoubleSumEmptyIsZero<>(call, hnd);
 
             case TINYINT:
             case SMALLINT:
             case INTEGER:
             default:
-                return LongSumEmptyIsZero.FACTORY;
+                return () -> new LongSumEmptyIsZero<>(call, hnd);
         }
     }
 
     /** */
-    private static Supplier<Accumulator> minFactory(AggregateCall call) {
+    private static <Row> Supplier<Accumulator<Row>> minFactory(AggregateCall call, RowHandler<Row> hnd) {
         switch (call.type.getSqlTypeName()) {
             case DOUBLE:
             case REAL:
             case FLOAT:
-                return DoubleMinMax.MIN_FACTORY;
+                return () -> new DoubleMinMax<>(call, hnd, true);
             case DECIMAL:
-                return DecimalMinMax.MIN_FACTORY;
+                return () -> new DecimalMinMax<>(call, hnd, true);
             case INTEGER:
-                return IntMinMax.MIN_FACTORY;
+                return () -> new IntMinMax<>(call, hnd, true);
             case CHAR:
             case VARCHAR:
-                return VarCharMinMax.MIN_FACTORY;
+                return () -> new VarCharMinMax<>(call, hnd, true);
             case BINARY:
             case VARBINARY:
-                return ComparableMinMax.VARBINARY_MIN_FACTORY;
+                return () -> new ComparableMinMax<Row, ByteString>(call, hnd, true,
+                    tf -> tf.createTypeWithNullability(tf.createSqlType(VARBINARY), true));
             case ANY:
-                if (call.type instanceof UuidType)
-                    return ComparableMinMax.UUID_MIN_FACTORY;
+                if (call.type instanceof UuidType) {
+                    return () -> new ComparableMinMax<Row, UUID>(call, hnd, true,
+                        tf -> tf.createTypeWithNullability(tf.createCustomType(UUID.class), true));
+                }
                 throw new UnsupportedOperationException("MIN() is not supported for type '" + call.type + "'.");
             case BIGINT:
             default:
-                return LongMinMax.MIN_FACTORY;
+                return () -> new LongMinMax<>(call, hnd, true);
         }
     }
 
     /** */
-    private static Supplier<Accumulator> maxFactory(AggregateCall call) {
+    private static <Row> Supplier<Accumulator<Row>> maxFactory(AggregateCall call, RowHandler<Row> hnd) {
         switch (call.type.getSqlTypeName()) {
             case DOUBLE:
             case REAL:
             case FLOAT:
-                return DoubleMinMax.MAX_FACTORY;
+                return () -> new DoubleMinMax<>(call, hnd, false);
             case DECIMAL:
-                return DecimalMinMax.MAX_FACTORY;
+                return () -> new DecimalMinMax<>(call, hnd, false);
             case INTEGER:
-                return IntMinMax.MAX_FACTORY;
+                return () -> new IntMinMax<>(call, hnd, false);
             case CHAR:
             case VARCHAR:
-                return VarCharMinMax.MAX_FACTORY;
+                return () -> new VarCharMinMax<>(call, hnd, false);
             case BINARY:
             case VARBINARY:
-                return ComparableMinMax.VARBINARY_MAX_FACTORY;
+                return () -> new ComparableMinMax<Row, ByteString>(call, hnd, false,
+                    tf -> tf.createTypeWithNullability(tf.createSqlType(VARBINARY), true));
             case ANY:
-                if (call.type instanceof UuidType)
-                    return ComparableMinMax.UUID_MAX_FACTORY;
+                if (call.type instanceof UuidType) {
+                    return () -> new ComparableMinMax<Row, UUID>(call, hnd, false,
+                        tf -> tf.createTypeWithNullability(tf.createCustomType(UUID.class), true));
+                }
                 throw new UnsupportedOperationException("MAX() is not supported for type '" + call.type + "'.");
             case BIGINT:
             default:
-                return LongMinMax.MAX_FACTORY;
+                return () -> new LongMinMax<>(call, hnd, false);
         }
     }
 
     /** */
-    private static class SingleVal extends AnyVal {
+    private abstract static class AbstractAccumulator<Row> implements Accumulator<Row> {
         /** */
-        private boolean touched;
+        private final RowHandler<Row> hnd;
 
         /** */
-        public static final Supplier<Accumulator> FACTORY = SingleVal::new;
+        private final transient AggregateCall aggCall;
+
+        /** */
+        AbstractAccumulator(AggregateCall aggCall, RowHandler<Row> hnd) {
+            this.aggCall = aggCall;
+            this.hnd = hnd;
+        }
+
+        /** */
+        <T> T get(int idx, Row row) {
+            List<Integer> argList = aggCall.getArgList();
+
+            assert idx < argList.size() : "idx=" + idx + "; arglist=" + argList;
+
+            return (T)hnd.get(argList.get(idx), row);
+        }
+
+        /** */
+        protected AggregateCall aggregateCall() {
+            return aggCall;
+        }
 
         /** */
-        @Override public void add(Object... args) {
+        int columnCount(Row row) {
+            return hnd.columnCount(row);
+        }
+    }
+
+    /** */
+    private static class SingleVal<Row> extends AnyVal<Row> {
+        /** */
+        private boolean touched;
+
+        /** */
+        SingleVal(AggregateCall aggCall, RowHandler<Row> hnd) {
+            super(aggCall, hnd);
+        }
+
+        /** {@inheritDoc}  */
+        @Override public void add(Row row) {
             if (touched)
                 throw new IllegalArgumentException("Subquery returned more than 1 value.");
 
             touched = true;
 
-            super.add(args);
+            super.add(row);
         }
 
-        /** */
-        @Override public void apply(Accumulator other) {
-            if (((SingleVal)other).touched) {
+        /** {@inheritDoc}  */
+        @Override public void apply(Accumulator<Row> other) {
+            if (((SingleVal<Row>)other).touched) {
                 if (touched)
                     throw new IllegalArgumentException("Subquery returned more than 1 value.");
                 else
@@ -228,57 +300,59 @@ public class Accumulators {
     }
 
     /** */
-    private static class AnyVal implements Accumulator {
+    private static class AnyVal<Row> extends AbstractAccumulator<Row> {
         /** */
         private Object holder;
 
         /** */
-        public static final Supplier<Accumulator> FACTORY = AnyVal::new;
-
-        /** */
-        @Override public void add(Object... args) {
-            assert args.length == 1 : args.length;
+        AnyVal(AggregateCall aggCall, RowHandler<Row> hnd) {
+            super(aggCall, hnd);
+        }
 
+        /** {@inheritDoc}  */
+        @Override public void add(Row row) {
             if (holder == null)
-                holder = args[0];
+                holder = get(0, row);
         }
 
-        /** */
-        @Override public void apply(Accumulator other) {
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator<Row> other) {
             if (holder == null)
-                holder = ((AnyVal)other).holder;
+                holder = ((AnyVal<Row>)other).holder;
         }
 
-        /** */
+        /** {@inheritDoc}  */
         @Override public Object end() {
             return holder;
         }
 
-        /** */
+        /** {@inheritDoc}  */
         @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
             return F.asList(typeFactory.createTypeWithNullability(typeFactory.createSqlType(ANY), true));
         }
 
-        /** */
+        /** {@inheritDoc}  */
         @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
             return typeFactory.createSqlType(ANY);
         }
     }
 
     /** */
-    public static class DecimalAvg implements Accumulator {
-        /** */
-        public static final Supplier<Accumulator> FACTORY = DecimalAvg::new;
-
+    private static class DecimalAvg<Row> extends AbstractAccumulator<Row> {
         /** */
         private BigDecimal sum = BigDecimal.ZERO;
 
         /** */
         private BigDecimal cnt = BigDecimal.ZERO;
 
+        /** */
+        DecimalAvg(AggregateCall aggCall, RowHandler<Row> hnd) {
+            super(aggCall, hnd);
+        }
+
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            BigDecimal in = (BigDecimal)args[0];
+        @Override public void add(Row row) {
+            BigDecimal in = get(0, row);
 
             if (in == null)
                 return;
@@ -288,8 +362,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            DecimalAvg other0 = (DecimalAvg)other;
+        @Override public void apply(Accumulator<Row> other) {
+            DecimalAvg<Row> other0 = (DecimalAvg<Row>)other;
 
             sum = sum.add(other0.sum);
             cnt = cnt.add(other0.cnt);
@@ -312,19 +386,21 @@ public class Accumulators {
     }
 
     /** */
-    public static class DoubleAvg implements Accumulator {
-        /** */
-        public static final Supplier<Accumulator> FACTORY = DoubleAvg::new;
-
+    private static class DoubleAvg<Row> extends AbstractAccumulator<Row> {
         /** */
         private double sum;
 
         /** */
         private long cnt;
 
+        /** */
+        DoubleAvg(AggregateCall aggCall, RowHandler<Row> hnd) {
+            super(aggCall, hnd);
+        }
+
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            Double in = (Double)args[0];
+        @Override public void add(Row row) {
+            Double in = get(0, row);
 
             if (in == null)
                 return;
@@ -334,8 +410,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            DoubleAvg other0 = (DoubleAvg)other;
+        @Override public void apply(Accumulator<Row> other) {
+            DoubleAvg<Row> other0 = (DoubleAvg<Row>)other;
 
             sum += other0.sum;
             cnt += other0.cnt;
@@ -358,24 +434,28 @@ public class Accumulators {
     }
 
     /** */
-    private static class LongCount implements Accumulator {
+    private static class LongCount<Row> extends AbstractAccumulator<Row> {
         /** */
-        public static final Supplier<Accumulator> FACTORY = LongCount::new;
+        private long cnt;
 
         /** */
-        private long cnt;
+        LongCount(AggregateCall aggCall, RowHandler<Row> hnd) {
+            super(aggCall, hnd);
+        }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            assert F.isEmpty(args) || args.length == 1;
+        @Override public void add(Row row) {
+            int argsCount = aggregateCall().getArgList().size();
+
+            assert argsCount == 0 || argsCount == 1;
 
-            if (F.isEmpty(args) || args[0] != null)
+            if (argsCount == 0 || get(0, row) != null)
                 cnt++;
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            LongCount other0 = (LongCount)other;
+        @Override public void apply(Accumulator<Row> other) {
+            LongCount<Row> other0 = (LongCount<Row>)other;
             cnt += other0.cnt;
         }
 
@@ -396,30 +476,34 @@ public class Accumulators {
     }
 
     /** */
-    private static class Sum implements Accumulator {
+    private static class Sum<Row> extends AbstractAccumulator<Row> {
         /** */
-        private Accumulator acc;
+        private final Accumulator<Row> acc;
 
         /** */
         private boolean empty = true;
 
         /** */
-        public Sum(Accumulator acc) {
+        public Sum(AggregateCall aggCall, Accumulator<Row> acc, RowHandler<Row> hnd) {
+            super(aggCall, hnd);
+
             this.acc = acc;
         }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            if (args[0] == null)
+        @Override public void add(Row row) {
+            Object val = get(0, row);
+
+            if (val == null)
                 return;
 
             empty = false;
-            acc.add(args[0]);
+            acc.add(row);
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            Sum other0 = (Sum)other;
+        @Override public void apply(Accumulator<Row> other) {
+            Sum<Row> other0 = (Sum<Row>)other;
 
             if (other0.empty)
                 return;
@@ -445,16 +529,18 @@ public class Accumulators {
     }
 
     /** */
-    private static class DoubleSumEmptyIsZero implements Accumulator {
+    private static class DoubleSumEmptyIsZero<Row> extends AbstractAccumulator<Row> {
         /** */
-        public static final Supplier<Accumulator> FACTORY = DoubleSumEmptyIsZero::new;
+        private double sum;
 
         /** */
-        private double sum;
+        DoubleSumEmptyIsZero(AggregateCall aggCall, RowHandler<Row> hnd) {
+            super(aggCall, hnd);
+        }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            Double in = (Double)args[0];
+        @Override public void add(Row row) {
+            Double in = get(0, row);
 
             if (in == null)
                 return;
@@ -463,8 +549,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            DoubleSumEmptyIsZero other0 = (DoubleSumEmptyIsZero)other;
+        @Override public void apply(Accumulator<Row> other) {
+            DoubleSumEmptyIsZero<Row> other0 = (DoubleSumEmptyIsZero<Row>)other;
 
             sum += other0.sum;
         }
@@ -486,17 +572,18 @@ public class Accumulators {
     }
 
     /** */
-    private static class LongSumEmptyIsZero implements Accumulator {
-        /** */
-        public static final Supplier<Accumulator> FACTORY = LongSumEmptyIsZero::new;
-
+    private static class LongSumEmptyIsZero<Row> extends AbstractAccumulator<Row> {
         /** */
         private long sum;
 
         /** */
+        LongSumEmptyIsZero(AggregateCall aggCall, RowHandler<Row> hnd) {
+            super(aggCall, hnd);
+        }
+
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            Long in = (Long)args[0];
+        @Override public void add(Row row) {
+            Long in = get(0, row);
 
             if (in == null)
                 return;
@@ -505,8 +592,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            LongSumEmptyIsZero other0 = (LongSumEmptyIsZero)other;
+        @Override public void apply(Accumulator<Row> other) {
+            LongSumEmptyIsZero<Row> other0 = (LongSumEmptyIsZero<Row>)other;
 
             sum += other0.sum;
         }
@@ -528,16 +615,18 @@ public class Accumulators {
     }
 
     /** */
-    private static class DecimalSumEmptyIsZero implements Accumulator {
+    private static class DecimalSumEmptyIsZero<Row> extends AbstractAccumulator<Row> {
         /** */
-        public static final Supplier<Accumulator> FACTORY = DecimalSumEmptyIsZero::new;
+        private BigDecimal sum;
 
         /** */
-        private BigDecimal sum;
+        DecimalSumEmptyIsZero(AggregateCall aggCall, RowHandler<Row> hnd) {
+            super(aggCall, hnd);
+        }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            BigDecimal in = (BigDecimal)args[0];
+        @Override public void add(Row row) {
+            BigDecimal in = get(0, row);
 
             if (in == null)
                 return;
@@ -546,8 +635,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            DecimalSumEmptyIsZero other0 = (DecimalSumEmptyIsZero)other;
+        @Override public void apply(Accumulator<Row> other) {
+            DecimalSumEmptyIsZero<Row> other0 = (DecimalSumEmptyIsZero<Row>)other;
 
             sum = sum == null ? other0.sum : sum.add(other0.sum);
         }
@@ -569,13 +658,7 @@ public class Accumulators {
     }
 
     /** */
-    private static class DoubleMinMax implements Accumulator {
-        /** */
-        public static final Supplier<Accumulator> MIN_FACTORY = () -> new DoubleMinMax(true);
-
-        /** */
-        public static final Supplier<Accumulator> MAX_FACTORY = () -> new DoubleMinMax(false);
-
+    private static class DoubleMinMax<Row> extends AbstractAccumulator<Row> {
         /** */
         private final boolean min;
 
@@ -586,13 +669,14 @@ public class Accumulators {
         private boolean empty = true;
 
         /** */
-        private DoubleMinMax(boolean min) {
+        private DoubleMinMax(AggregateCall aggCall, RowHandler<Row> hnd, boolean min) {
+            super(aggCall, hnd);
             this.min = min;
         }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            Double in = (Double)args[0];
+        @Override public void add(Row row) {
+            Double in = get(0, row);
 
             if (in == null)
                 return;
@@ -602,8 +686,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            DoubleMinMax other0 = (DoubleMinMax)other;
+        @Override public void apply(Accumulator<Row> other) {
+            DoubleMinMax<Row> other0 = (DoubleMinMax<Row>)other;
 
             if (other0.empty)
                 return;
@@ -629,13 +713,7 @@ public class Accumulators {
     }
 
     /** */
-    private static class VarCharMinMax implements Accumulator {
-        /** */
-        public static final Supplier<Accumulator> MIN_FACTORY = () -> new VarCharMinMax(true);
-
-        /** */
-        public static final Supplier<Accumulator> MAX_FACTORY = () -> new VarCharMinMax(false);
-
+    private static class VarCharMinMax<Row> extends AbstractAccumulator<Row> {
         /** */
         private final boolean min;
 
@@ -646,13 +724,14 @@ public class Accumulators {
         private boolean empty = true;
 
         /** */
-        private VarCharMinMax(boolean min) {
+        VarCharMinMax(AggregateCall aggCall, RowHandler<Row> hnd, boolean min) {
+            super(aggCall, hnd);
             this.min = min;
         }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            CharSequence in = (CharSequence)args[0];
+        @Override public void add(Row row) {
+            CharSequence in = get(0, row);
 
             if (in == null)
                 return;
@@ -665,8 +744,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            VarCharMinMax other0 = (VarCharMinMax)other;
+        @Override public void apply(Accumulator<Row> other) {
+            VarCharMinMax<Row> other0 = (VarCharMinMax<Row>)other;
 
             if (other0.empty)
                 return;
@@ -717,13 +796,7 @@ public class Accumulators {
     }
 
     /** */
-    private static class IntMinMax implements Accumulator {
-        /** */
-        public static final Supplier<Accumulator> MIN_FACTORY = () -> new IntMinMax(true);
-
-        /** */
-        public static final Supplier<Accumulator> MAX_FACTORY = () -> new IntMinMax(false);
-
+    private static class IntMinMax<Row> extends AbstractAccumulator<Row> {
         /** */
         private final boolean min;
 
@@ -734,13 +807,14 @@ public class Accumulators {
         private boolean empty = true;
 
         /** */
-        private IntMinMax(boolean min) {
+        private IntMinMax(AggregateCall aggCall, RowHandler<Row> hnd, boolean min) {
+            super(aggCall, hnd);
             this.min = min;
         }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            Integer in = (Integer)args[0];
+        @Override public void add(Row row) {
+            Integer in = get(0, row);
 
             if (in == null)
                 return;
@@ -750,8 +824,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            IntMinMax other0 = (IntMinMax)other;
+        @Override public void apply(Accumulator<Row> other) {
+            IntMinMax<Row> other0 = (IntMinMax<Row>)other;
 
             if (other0.empty)
                 return;
@@ -777,13 +851,7 @@ public class Accumulators {
     }
 
     /** */
-    private static class LongMinMax implements Accumulator {
-        /** */
-        public static final Supplier<Accumulator> MIN_FACTORY = () -> new LongMinMax(true);
-
-        /** */
-        public static final Supplier<Accumulator> MAX_FACTORY = () -> new LongMinMax(false);
-
+    private static class LongMinMax<Row> extends AbstractAccumulator<Row> {
         /** */
         private final boolean min;
 
@@ -794,13 +862,15 @@ public class Accumulators {
         private boolean empty = true;
 
         /** */
-        private LongMinMax(boolean min) {
+        private LongMinMax(AggregateCall aggCall, RowHandler<Row> hnd, boolean min) {
+            super(aggCall, hnd);
+
             this.min = min;
         }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            Long in = (Long)args[0];
+        @Override public void add(Row row) {
+            Long in = get(0, row);
 
             if (in == null)
                 return;
@@ -810,8 +880,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            LongMinMax other0 = (LongMinMax)other;
+        @Override public void apply(Accumulator<Row> other) {
+            LongMinMax<Row> other0 = (LongMinMax<Row>)other;
 
             if (other0.empty)
                 return;
@@ -837,13 +907,7 @@ public class Accumulators {
     }
 
     /** */
-    private static class DecimalMinMax implements Accumulator {
-        /** */
-        public static final Supplier<Accumulator> MIN_FACTORY = () -> new DecimalMinMax(true);
-
-        /** */
-        public static final Supplier<Accumulator> MAX_FACTORY = () -> new DecimalMinMax(false);
-
+    private static class DecimalMinMax<Row> extends AbstractAccumulator<Row> {
         /** */
         private final boolean min;
 
@@ -851,13 +915,14 @@ public class Accumulators {
         private BigDecimal val;
 
         /** */
-        private DecimalMinMax(boolean min) {
+        private DecimalMinMax(AggregateCall aggCall, RowHandler<Row> hnd, boolean min) {
+            super(aggCall, hnd);
             this.min = min;
         }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            BigDecimal in = (BigDecimal)args[0];
+        @Override public void add(Row row) {
+            BigDecimal in = get(0, row);
 
             if (in == null)
                 return;
@@ -866,8 +931,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            DecimalMinMax other0 = (DecimalMinMax)other;
+        @Override public void apply(Accumulator<Row> other) {
+            DecimalMinMax<Row> other0 = (DecimalMinMax<Row>)other;
 
             if (other0.val == null)
                 return;
@@ -892,23 +957,7 @@ public class Accumulators {
     }
 
     /** */
-    private static class ComparableMinMax<T extends Comparable<T>> implements Accumulator {
-        /** */
-        public static final Supplier<Accumulator> VARBINARY_MIN_FACTORY = () -> new ComparableMinMax<ByteString>(true,
-            tf -> tf.createTypeWithNullability(tf.createSqlType(VARBINARY), true));
-
-        /** */
-        public static final Supplier<Accumulator> VARBINARY_MAX_FACTORY = () -> new ComparableMinMax<ByteString>(false,
-            tf -> tf.createTypeWithNullability(tf.createSqlType(VARBINARY), true));
-
-        /** */
-        public static final Supplier<Accumulator> UUID_MIN_FACTORY = () -> new ComparableMinMax<UUID>(true,
-            tf -> tf.createTypeWithNullability(tf.createCustomType(UUID.class), true));
-
-        /** */
-        public static final Supplier<Accumulator> UUID_MAX_FACTORY = () -> new ComparableMinMax<UUID>(false,
-            tf -> tf.createTypeWithNullability(tf.createCustomType(UUID.class), true));
-
+    private static class ComparableMinMax<Row, T extends Comparable<T>> extends AbstractAccumulator<Row> {
         /** */
         private final boolean min;
 
@@ -922,14 +971,20 @@ public class Accumulators {
         private boolean empty = true;
 
         /** */
-        private ComparableMinMax(boolean min, Function<IgniteTypeFactory, RelDataType> typeSupplier) {
+        private ComparableMinMax(
+            AggregateCall aggCall,
+            RowHandler<Row> hnd,
+            boolean min,
+            Function<IgniteTypeFactory, RelDataType> typeSupplier
+        ) {
+            super(aggCall, hnd);
             this.min = min;
             this.typeSupplier = typeSupplier;
         }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            T in = (T)args[0];
+        @Override public void add(Row row) {
+            T in = get(0, row);
 
             if (in == null)
                 return;
@@ -942,8 +997,8 @@ public class Accumulators {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            ComparableMinMax<T> other0 = (ComparableMinMax<T>)other;
+        @Override public void apply(Accumulator<Row> other) {
+            ComparableMinMax<Row, T> other0 = (ComparableMinMax<Row, T>)other;
 
             if (other0.empty)
                 return;
@@ -972,39 +1027,174 @@ public class Accumulators {
     }
 
     /** */
-    private static class DistinctAccumulator implements Accumulator {
+    private static class SortingAccumulator<Row> implements Accumulator<Row> {
         /** */
-        private final Accumulator acc;
+        private final transient Comparator<Row> cmp;
 
         /** */
-        private final Set<Object> set = new HashSet<>();
+        private final List<Row> list;
 
         /** */
-        private DistinctAccumulator(Supplier<Accumulator> accSup) {
-            this.acc = accSup.get();
+        private final Accumulator<Row> acc;
+
+        /**
+         * @param accSup Accumulator supplier.
+         * @param cmp Comparator.
+         */
+        private SortingAccumulator(Supplier<Accumulator<Row>> accSup, Comparator<Row> cmp) {
+            this.cmp = cmp;
+
+            list = new ArrayList<>();
+            acc = accSup.get();
         }
 
         /** {@inheritDoc} */
-        @Override public void add(Object... args) {
-            Object in = args[0];
+        @Override public void add(Row row) {
+            list.add(row);
+        }
 
-            if (in == null)
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator<Row> other) {
+            SortingAccumulator<Row> other1 = (SortingAccumulator<Row>)other;
+
+            list.addAll(other1.list);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            list.sort(cmp);
+
+            for (Row row : list)
+                acc.add(row);
+
+            return acc.end();
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return acc.argumentTypes(typeFactory);
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return acc.returnType(typeFactory);
+        }
+    }
+
+    /** */
+    private static class ListAggAccumulator<Row> extends AbstractAccumulator<Row> {
+        /** Default separator. */
+        private static final String DEFAULT_SEPARATOR = ",";
+
+        /** */
+        private final List<Row> list;
+
+        /** */
+        private final boolean isDfltSep;
+
+        /** */
+        public ListAggAccumulator(AggregateCall aggCall, RowHandler<Row> hnd) {
+            super(aggCall, hnd);
+
+            isDfltSep = aggCall.getArgList().size() <= 1;
+
+            list = new ArrayList<>();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void add(Row row) {
+            if (row == null || get(0, row) == null)
+                return;
+
+            list.add(row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator<Row> other) {
+            ListAggAccumulator<Row> other0 = (ListAggAccumulator<Row>)other;
+
+            list.addAll(other0.list);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            if (list.isEmpty())
+                return null;
+
+            StringBuilder builder = new StringBuilder();
+
+            for (Row row: list) {
+                if (builder.length() != 0)
+                    builder.append(extractSeparator(row));
+
+                builder.append(Objects.toString(get(0, row)));
+            }
+
+            return builder.toString();
+        }
+
+        /** */
+        private String extractSeparator(Row row) {
+            if (isDfltSep || columnCount(row) <= 1)
+                return DEFAULT_SEPARATOR;
+
+            Object rawSep = get(1, row);
+
+            if (rawSep == null)
+                return DEFAULT_SEPARATOR;
+
+            return rawSep.toString();
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return F.asList(typeFactory.createTypeWithNullability(typeFactory.createSqlType(VARCHAR), true),
+                typeFactory.createTypeWithNullability(typeFactory.createSqlType(CHAR), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(VARCHAR), true);
+        }
+    }
+
+    /** */
+    private static class DistinctAccumulator<Row> extends AbstractAccumulator<Row> {
+        /** */
+        private final Accumulator<Row> acc;
+
+        /** */
+        private final Map<Object, Row> rows = new HashMap<>();
+
+        /** */
+        private DistinctAccumulator(AggregateCall aggCall, RowHandler<Row> hnd, Supplier<Accumulator<Row>> accSup) {
+            super(aggCall, hnd);
+            acc = accSup.get();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void add(Row row) {
+            if (row == null || columnCount(row) == 0 || get(0, row) == null)
+                return;
+
+            Object key = get(0, row);
+            if (key == null)
                 return;
 
-            set.add(in);
+            rows.put(key, row);
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator other) {
-            DistinctAccumulator other0 = (DistinctAccumulator)other;
+        @Override public void apply(Accumulator<Row> other) {
+            DistinctAccumulator<Row> other0 = (DistinctAccumulator<Row>)other;
 
-            set.addAll(other0.set);
+            rows.putAll(other0.rows);
         }
 
         /** {@inheritDoc} */
         @Override public Object end() {
-            for (Object o : set)
-                acc.add(o);
+            for (Row row: rows.values())
+                acc.add(row);
 
             return acc.end();
         }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java
index 6be23a8047c..23137186182 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
 
 import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.function.Function;
@@ -70,7 +72,7 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
     }
 
     /** */
-    private static Function<Object, Object> cast(RelDataType from, RelDataType to) {
+    static Function<Object, Object> cast(RelDataType from, RelDataType to) {
         assert !from.isStruct();
         assert !to.isStruct();
 
@@ -78,7 +80,7 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
     }
 
     /** */
-    private static Function<Object, Object> cast(Pair<RelDataType, RelDataType> types) {
+    static Function<Object, Object> cast(Pair<RelDataType, RelDataType> types) {
         try {
             return CACHE.get(types);
         }
@@ -169,13 +171,13 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
     /** */
     private final class WrapperPrototype implements Supplier<AccumulatorWrapper<Row>> {
         /** */
-        private Supplier<Accumulator> accFactory;
+        private Supplier<Accumulator<Row>> accFactory;
 
         /** */
         private final AggregateCall call;
 
         /** */
-        private Function<Object[], Object[]> inAdapter;
+        private Function<Row, Row> inAdapter;
 
         /** */
         private Function<Object, Object> outAdapter;
@@ -187,19 +189,19 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
 
         /** {@inheritDoc} */
         @Override public AccumulatorWrapper<Row> get() {
-            Accumulator accumulator = accumulator();
+            Accumulator<Row> accumulator = accumulator();
 
             return new AccumulatorWrapperImpl(accumulator, call, inAdapter, outAdapter);
         }
 
         /** */
-        @NotNull private Accumulator accumulator() {
+        @NotNull private Accumulator<Row> accumulator() {
             if (accFactory != null)
                 return accFactory.get();
 
             // init factory and adapters
-            accFactory = Accumulators.accumulatorFactory(call);
-            Accumulator accumulator = accFactory.get();
+            accFactory = Accumulators.accumulatorFactory(call, ctx);
+            Accumulator<Row> accumulator = accFactory.get();
 
             inAdapter = createInAdapter(accumulator);
             outAdapter = createOutAdapter(accumulator);
@@ -208,7 +210,7 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
         }
 
         /** */
-        @NotNull private Function<Object[], Object[]> createInAdapter(Accumulator accumulator) {
+        @NotNull private Function<Row, Row> createInAdapter(Accumulator<Row> accumulator) {
             if (type == AggregateType.REDUCE || F.isEmpty(call.getArgList()))
                 return Function.identity();
 
@@ -226,17 +228,42 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
             List<Function<Object, Object>> casts =
                 Commons.transform(Pair.zip(inTypes, outTypes), AccumulatorsFactory::cast);
 
-            return new Function<Object[], Object[]>() {
-                @Override public Object[] apply(Object[] args) {
-                    for (int i = 0; i < args.length; i++)
-                        args[i] = casts.get(i).apply(args[i]);
-                    return args;
+            final boolean ignoreNulls = call.ignoreNulls();
+
+            final int[] argMapping = new int[Collections.max(call.getArgList()) + 1];
+            Arrays.fill(argMapping, -1);
+
+            for (int i = 0; i < call.getArgList().size(); ++i)
+                argMapping[call.getArgList().get(i)] = i;
+
+            return new Function<Row, Row>() {
+                final RowHandler<Row> hnd = ctx.rowHandler();
+
+                final RowHandler.RowFactory<Row> rowFac = hnd.factory(ctx.getTypeFactory(), inputRowType);
+
+                @Override public Row apply(Row in) {
+                    Row out = rowFac.create();
+
+                    for (int i = 0; i < hnd.columnCount(in); ++i) {
+                        Object val = hnd.get(i, in);
+
+                        if (ignoreNulls && val == null)
+                            return null;
+
+                        int idx = i < argMapping.length ? argMapping[i] : -1;
+                        if (idx != -1)
+                            val = casts.get(idx).apply(val);
+
+                        hnd.set(i, out, val);
+                    }
+
+                    return out;
                 }
             };
         }
 
         /** */
-        @NotNull private Function<Object, Object> createOutAdapter(Accumulator accumulator) {
+        @NotNull private Function<Object, Object> createOutAdapter(Accumulator<Row> accumulator) {
             if (type == AggregateType.MAP)
                 return Function.identity();
 
@@ -255,39 +282,31 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
     /** */
     private final class AccumulatorWrapperImpl implements AccumulatorWrapper<Row> {
         /** */
-        private final Accumulator accumulator;
+        private final Accumulator<Row> accumulator;
 
         /** */
-        private final Function<Object[], Object[]> inAdapter;
+        private final Function<Row, Row> inAdapter;
 
         /** */
         private final Function<Object, Object> outAdapter;
 
-        /** */
-        private final List<Integer> argList;
-
         /** */
         private final int filterArg;
 
-        /** */
-        private final boolean ignoreNulls;
-
         /** */
         private final RowHandler<Row> handler;
 
         /** */
         AccumulatorWrapperImpl(
-            Accumulator accumulator,
+            Accumulator<Row> accumulator,
             AggregateCall call,
-            Function<Object[], Object[]> inAdapter,
+            Function<Row, Row> inAdapter,
             Function<Object, Object> outAdapter
         ) {
             this.accumulator = accumulator;
             this.inAdapter = inAdapter;
             this.outAdapter = outAdapter;
 
-            argList = call.getArgList();
-            ignoreNulls = call.ignoreNulls();
             filterArg = call.hasFilter() ? call.filterArg : -1;
 
             handler = ctx.rowHandler();
@@ -300,15 +319,11 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
             if (filterArg >= 0 && Boolean.TRUE != handler.get(filterArg, row))
                 return;
 
-            Object[] args = new Object[argList.size()];
-            for (int i = 0; i < argList.size(); i++) {
-                args[i] = handler.get(argList.get(i), row);
-
-                if (ignoreNulls && args[i] == null)
-                    return;
-            }
+            Row newRow = inAdapter.apply(row);
+            if (newRow == null)
+                return;
 
-            accumulator.add(inAdapter.apply(args));
+            accumulator.add(newRow);
         }
 
         /** {@inheritDoc} */
@@ -319,14 +334,14 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(Accumulator accumulator) {
+        @Override public void apply(Accumulator<Row> accumulator) {
             assert type == AggregateType.REDUCE;
 
             this.accumulator.apply(accumulator);
         }
 
         /** {@inheritDoc} */
-        @Override public Accumulator accumulator() {
+        @Override public Accumulator<Row> accumulator() {
             assert type == AggregateType.MAP;
 
             return accumulator;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJson.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJson.java
index c938a15fc00..04ddded380d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJson.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJson.java
@@ -699,6 +699,7 @@ class RelJson {
         map.put("operands", node.getArgList());
         map.put("filter", node.filterArg);
         map.put("name", node.getName());
+        map.put("coll", toJson(node.getCollation()));
         return map;
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJsonReader.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJsonReader.java
index a3a5000250d..7847ef20fbc 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJsonReader.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJsonReader.java
@@ -33,7 +33,6 @@ import org.apache.calcite.plan.RelOptSchema;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
@@ -325,10 +324,9 @@ public class RelJsonReader {
             Integer filterOperand = (Integer)jsonAggCall.get("filter");
             RelDataType type = relJson.toType(Commons.typeFactory(Commons.emptyCluster()), jsonAggCall.get("type"));
             String name = (String)jsonAggCall.get("name");
+            RelCollation collation = relJson.toCollation((List<Map<String, Object>>)jsonAggCall.get("coll"));
             return AggregateCall.create(aggregation, distinct, false, false, operands,
-                filterOperand == null ? -1 : filterOperand,
-                RelCollations.EMPTY,
-                type, name);
+                filterOperand == null ? -1 : filterOperand, null, collation, type, name);
         }
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
index 5c3ccb294c1..71c822289e2 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
@@ -389,7 +389,9 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
             case MIN:
             case MAX:
             case ANY_VALUE:
-
+            case GROUP_CONCAT:
+            case LISTAGG:
+            case STRING_AGG:
                 return;
             default:
                 throw newValidationError(call,
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java
index e1aab075b75..85ae8bc4cc9 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java
@@ -24,6 +24,7 @@ import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
 import org.apache.calcite.rel.rules.AggregateMergeRule;
 import org.apache.calcite.rel.rules.CoreRules;
 import org.apache.calcite.rel.rules.FilterJoinRule.FilterIntoJoinRule;
@@ -40,6 +41,7 @@ import org.apache.calcite.rel.rules.SortRemoveRule;
 import org.apache.calcite.tools.Program;
 import org.apache.calcite.tools.RuleSet;
 import org.apache.calcite.tools.RuleSets;
+import org.apache.calcite.util.Optionality;
 import org.apache.ignite.internal.processors.query.calcite.rule.CorrelateToNestedLoopRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.CorrelatedNestedLoopJoinRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.FilterConverterRule;
@@ -184,7 +186,14 @@ public enum PlannerPhase {
                                         .predicate(Aggregate::isSimple)
                                         .anyInputs())).toRule(),
 
-                    CoreRules.AGGREGATE_EXPAND_DISTINCT_AGGREGATES_TO_JOIN,
+                    // Rule is applicable to aggregates without ordering, otherwise application of this rule
+                    // leads to invalid projections (i.e. LISTAGG).
+                    AggregateExpandDistinctAggregatesRule.Config.JOIN
+                        .withOperandSupplier(op -> op.operand(LogicalAggregate.class)
+                            .predicate(agg -> agg.getAggCallList().stream().noneMatch(call ->
+                                    call.getAggregation().requiresGroupOrder() != Optionality.FORBIDDEN))
+                            .anyInputs())
+                        .toRule(),
 
                     SortRemoveRule.Config.DEFAULT
                         .withOperandSupplier(b ->
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/fun/IgniteStdSqlOperatorTable.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/fun/IgniteStdSqlOperatorTable.java
index d0e5391b0f7..d11b19cdad8 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/fun/IgniteStdSqlOperatorTable.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/fun/IgniteStdSqlOperatorTable.java
@@ -83,6 +83,9 @@ public class IgniteStdSqlOperatorTable extends ReflectiveSqlOperatorTable {
         register(SqlStdOperatorTable.ANY_VALUE);
         register(SqlStdOperatorTable.SINGLE_VALUE);
         register(SqlStdOperatorTable.FILTER);
+        register(SqlLibraryOperators.GROUP_CONCAT);
+        register(SqlLibraryOperators.STRING_AGG);
+        register(SqlStdOperatorTable.LISTAGG);
 
         // IS ... operator.
         register(SqlStdOperatorTable.IS_NULL);
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImpl.java
index a8aabb6be78..692acecd567 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImpl.java
@@ -8918,251 +8918,253 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     } else if (jj_2_1018(2)) {
       jj_consume_token(ASYNC);
     } else if (jj_2_1019(2)) {
-      jj_consume_token(A);
+      jj_consume_token(ABS);
     } else if (jj_2_1020(2)) {
-      jj_consume_token(ACTION);
+      jj_consume_token(ADD);
     } else if (jj_2_1021(2)) {
-      jj_consume_token(ALLOCATE);
+      jj_consume_token(ALLOW);
     } else if (jj_2_1022(2)) {
-      jj_consume_token(AND);
+      jj_consume_token(ARE);
     } else if (jj_2_1023(2)) {
-      jj_consume_token(ARRAY_MAX_CARDINALITY);
+      jj_consume_token(AS);
     } else if (jj_2_1024(2)) {
-      jj_consume_token(ASENSITIVE);
+      jj_consume_token(ASSERTION);
     } else if (jj_2_1025(2)) {
-      jj_consume_token(AT);
+      jj_consume_token(ATOMIC);
     } else if (jj_2_1026(2)) {
-      jj_consume_token(AVG);
+      jj_consume_token(BEFORE);
     } else if (jj_2_1027(2)) {
-      jj_consume_token(BEGIN_FRAME);
+      jj_consume_token(BEGIN_PARTITION);
     } else if (jj_2_1028(2)) {
-      jj_consume_token(BIGINT);
+      jj_consume_token(BINARY);
     } else if (jj_2_1029(2)) {
-      jj_consume_token(BLOB);
+      jj_consume_token(BOOLEAN);
     } else if (jj_2_1030(2)) {
-      jj_consume_token(BREADTH);
+      jj_consume_token(BY);
     } else if (jj_2_1031(2)) {
-      jj_consume_token(CALLED);
+      jj_consume_token(CARDINALITY);
     } else if (jj_2_1032(2)) {
-      jj_consume_token(CASCADED);
+      jj_consume_token(CAST);
     } else if (jj_2_1033(2)) {
-      jj_consume_token(CEIL);
+      jj_consume_token(CEILING);
     } else if (jj_2_1034(2)) {
-      jj_consume_token(CHARACTER);
+      jj_consume_token(CHARACTER_LENGTH);
     } else if (jj_2_1035(2)) {
-      jj_consume_token(CHECK);
+      jj_consume_token(CLASSIFIER);
     } else if (jj_2_1036(2)) {
-      jj_consume_token(CLOSE);
+      jj_consume_token(COALESCE);
     } else if (jj_2_1037(2)) {
-      jj_consume_token(COLLATION);
+      jj_consume_token(COLLECT);
     } else if (jj_2_1038(2)) {
-      jj_consume_token(COMMIT);
+      jj_consume_token(CONDITION);
     } else if (jj_2_1039(2)) {
-      jj_consume_token(CONNECTION);
+      jj_consume_token(CONSTRAINTS);
     } else if (jj_2_1040(2)) {
-      jj_consume_token(CONTAINS);
+      jj_consume_token(CONTINUE);
     } else if (jj_2_1041(2)) {
-      jj_consume_token(CORR);
+      jj_consume_token(CORRESPONDING);
     } else if (jj_2_1042(2)) {
-      jj_consume_token(COVAR_POP);
+      jj_consume_token(COVAR_SAMP);
     } else if (jj_2_1043(2)) {
-      jj_consume_token(CUME_DIST);
+      jj_consume_token(CURRENT_CATALOG);
     } else if (jj_2_1044(2)) {
-      jj_consume_token(CURRENT_DEFAULT_TRANSFORM_GROUP);
+      jj_consume_token(CURRENT_PATH);
     } else if (jj_2_1045(2)) {
-      jj_consume_token(CURRENT_ROW);
+      jj_consume_token(CURRENT_SCHEMA);
     } else if (jj_2_1046(2)) {
-      jj_consume_token(CURRENT_TIMESTAMP);
+      jj_consume_token(CURRENT_TRANSFORM_GROUP_FOR_TYPE);
     } else if (jj_2_1047(2)) {
-      jj_consume_token(CYCLE);
+      jj_consume_token(DATA);
     } else if (jj_2_1048(2)) {
-      jj_consume_token(DAY);
+      jj_consume_token(DEALLOCATE);
     } else if (jj_2_1049(2)) {
-      jj_consume_token(DECIMAL);
+      jj_consume_token(DECLARE);
     } else if (jj_2_1050(2)) {
-      jj_consume_token(DEFERRED);
+      jj_consume_token(DENSE_RANK);
     } else if (jj_2_1051(2)) {
-      jj_consume_token(DEREF);
+      jj_consume_token(DESC);
     } else if (jj_2_1052(2)) {
-      jj_consume_token(DETERMINISTIC);
+      jj_consume_token(DIAGNOSTICS);
     } else if (jj_2_1053(2)) {
-      jj_consume_token(DISCONNECT);
+      jj_consume_token(DOMAIN);
     } else if (jj_2_1054(2)) {
-      jj_consume_token(DYNAMIC);
+      jj_consume_token(EACH);
     } else if (jj_2_1055(2)) {
-      jj_consume_token(ELSE);
+      jj_consume_token(EMPTY);
     } else if (jj_2_1056(2)) {
-      jj_consume_token(END_FRAME);
+      jj_consume_token(END_PARTITION);
     } else if (jj_2_1057(2)) {
-      jj_consume_token(ESCAPE);
+      jj_consume_token(EVERY);
     } else if (jj_2_1058(2)) {
-      jj_consume_token(EXEC);
+      jj_consume_token(EXECUTE);
     } else if (jj_2_1059(2)) {
-      jj_consume_token(EXP);
+      jj_consume_token(EXTEND);
     } else if (jj_2_1060(2)) {
-      jj_consume_token(EXTRACT);
+      jj_consume_token(FALSE);
     } else if (jj_2_1061(2)) {
-      jj_consume_token(FIRST);
+      jj_consume_token(FIRST_VALUE);
     } else if (jj_2_1062(2)) {
-      jj_consume_token(FLOOR);
+      jj_consume_token(FOR);
     } else if (jj_2_1063(2)) {
-      jj_consume_token(FOUND);
+      jj_consume_token(FRAME_ROW);
     } else if (jj_2_1064(2)) {
-      jj_consume_token(FUNCTION);
+      jj_consume_token(FUSION);
     } else if (jj_2_1065(2)) {
-      jj_consume_token(GENERAL);
+      jj_consume_token(GET);
     } else if (jj_2_1066(2)) {
-      jj_consume_token(GO);
+      jj_consume_token(GOTO);
     } else if (jj_2_1067(2)) {
-      jj_consume_token(HOLD);
+      jj_consume_token(HOUR);
     } else if (jj_2_1068(2)) {
-      jj_consume_token(IMMEDIATE);
+      jj_consume_token(IMMEDIATELY);
     } else if (jj_2_1069(2)) {
-      jj_consume_token(INDICATOR);
+      jj_consume_token(INITIAL);
     } else if (jj_2_1070(2)) {
-      jj_consume_token(INOUT);
+      jj_consume_token(INPUT);
     } else if (jj_2_1071(2)) {
-      jj_consume_token(INT);
+      jj_consume_token(INTEGER);
     } else if (jj_2_1072(2)) {
-      jj_consume_token(IS);
+      jj_consume_token(ISOLATION);
     } else if (jj_2_1073(2)) {
-      jj_consume_token(JSON_ARRAYAGG);
+      jj_consume_token(JSON_EXISTS);
     } else if (jj_2_1074(2)) {
-      jj_consume_token(JSON_OBJECTAGG);
+      jj_consume_token(JSON_QUERY);
     } else if (jj_2_1075(2)) {
-      jj_consume_token(K);
+      jj_consume_token(KEY);
     } else if (jj_2_1076(2)) {
-      jj_consume_token(LANGUAGE);
+      jj_consume_token(LARGE);
     } else if (jj_2_1077(2)) {
-      jj_consume_token(LAST_VALUE);
+      jj_consume_token(LEAD);
     } else if (jj_2_1078(2)) {
-      jj_consume_token(LENGTH);
+      jj_consume_token(LEVEL);
     } else if (jj_2_1079(2)) {
-      jj_consume_token(LIKE_REGEX);
+      jj_consume_token(LN);
     } else if (jj_2_1080(2)) {
-      jj_consume_token(LOCALTIME);
+      jj_consume_token(LOCALTIMESTAMP);
     } else if (jj_2_1081(2)) {
-      jj_consume_token(LOWER);
+      jj_consume_token(M);
     } else if (jj_2_1082(2)) {
-      jj_consume_token(MATCH);
+      jj_consume_token(MATCHES);
     } else if (jj_2_1083(2)) {
-      jj_consume_token(MAX);
+      jj_consume_token(MEASURES);
     } else if (jj_2_1084(2)) {
-      jj_consume_token(METHOD);
+      jj_consume_token(MIN);
     } else if (jj_2_1085(2)) {
-      jj_consume_token(MOD);
+      jj_consume_token(MODIFIES);
     } else if (jj_2_1086(2)) {
-      jj_consume_token(MONTH);
+      jj_consume_token(MULTISET);
     } else if (jj_2_1087(2)) {
-      jj_consume_token(NAMES);
+      jj_consume_token(NATIONAL);
     } else if (jj_2_1088(2)) {
-      jj_consume_token(NCLOB);
+      jj_consume_token(NO);
     } else if (jj_2_1089(2)) {
-      jj_consume_token(NORMALIZE);
+      jj_consume_token(NOT);
     } else if (jj_2_1090(2)) {
-      jj_consume_token(NTILE);
+      jj_consume_token(NULLIF);
     } else if (jj_2_1091(2)) {
-      jj_consume_token(OBJECT);
+      jj_consume_token(OCCURRENCES_REGEX);
     } else if (jj_2_1092(2)) {
-      jj_consume_token(OF);
+      jj_consume_token(OLD);
     } else if (jj_2_1093(2)) {
-      jj_consume_token(ONE);
+      jj_consume_token(ONLY);
     } else if (jj_2_1094(2)) {
-      jj_consume_token(OPTION);
+      jj_consume_token(OR);
     } else if (jj_2_1095(2)) {
-      jj_consume_token(OUT);
+      jj_consume_token(OUTPUT);
     } else if (jj_2_1096(2)) {
-      jj_consume_token(OVERLAY);
+      jj_consume_token(PAD);
     } else if (jj_2_1097(2)) {
-      jj_consume_token(PARTIAL);
+      jj_consume_token(PATH);
     } else if (jj_2_1098(2)) {
-      jj_consume_token(PERCENT);
+      jj_consume_token(PERCENTILE_CONT);
     } else if (jj_2_1099(2)) {
-      jj_consume_token(PERCENT_RANK);
+      jj_consume_token(PERIOD);
     } else if (jj_2_1100(2)) {
-      jj_consume_token(PORTION);
+      jj_consume_token(POSITION);
     } else if (jj_2_1101(2)) {
-      jj_consume_token(POWER);
+      jj_consume_token(PRECEDES);
     } else if (jj_2_1102(2)) {
-      jj_consume_token(PREPARE);
+      jj_consume_token(PRESERVE);
     } else if (jj_2_1103(2)) {
-      jj_consume_token(PRIOR);
+      jj_consume_token(PRIVILEGES);
     } else if (jj_2_1104(2)) {
-      jj_consume_token(PUBLIC);
+      jj_consume_token(QUARTER);
     } else if (jj_2_1105(2)) {
-      jj_consume_token(READ);
+      jj_consume_token(READS);
     } else if (jj_2_1106(2)) {
-      jj_consume_token(RECURSIVE);
+      jj_consume_token(REF);
     } else if (jj_2_1107(2)) {
-      jj_consume_token(REFERENCING);
+      jj_consume_token(REGR_AVGX);
     } else if (jj_2_1108(2)) {
-      jj_consume_token(REGR_COUNT);
+      jj_consume_token(REGR_INTERCEPT);
     } else if (jj_2_1109(2)) {
-      jj_consume_token(REGR_SLOPE);
+      jj_consume_token(REGR_SXX);
     } else if (jj_2_1110(2)) {
-      jj_consume_token(REGR_SYY);
+      jj_consume_token(RELATIVE);
     } else if (jj_2_1111(2)) {
-      jj_consume_token(REPLACE);
+      jj_consume_token(RESET);
     } else if (jj_2_1112(2)) {
-      jj_consume_token(RESULT);
+      jj_consume_token(RETURN);
     } else if (jj_2_1113(2)) {
-      jj_consume_token(REVOKE);
+      jj_consume_token(ROLE);
     } else if (jj_2_1114(2)) {
-      jj_consume_token(ROUTINE);
+      jj_consume_token(ROW_NUMBER);
     } else if (jj_2_1115(2)) {
-      jj_consume_token(SAVEPOINT);
+      jj_consume_token(SCHEMA);
     } else if (jj_2_1116(2)) {
-      jj_consume_token(SCROLL);
+      jj_consume_token(SEARCH);
     } else if (jj_2_1117(2)) {
-      jj_consume_token(SECTION);
+      jj_consume_token(SEEK);
     } else if (jj_2_1118(2)) {
-      jj_consume_token(SESSION);
+      jj_consume_token(SESSION_USER);
     } else if (jj_2_1119(2)) {
-      jj_consume_token(SIMILAR);
+      jj_consume_token(SIZE);
     } else if (jj_2_1120(2)) {
-      jj_consume_token(SPACE);
+      jj_consume_token(SPECIFIC);
     } else if (jj_2_1121(2)) {
-      jj_consume_token(SQL);
+      jj_consume_token(SQLEXCEPTION);
     } else if (jj_2_1122(2)) {
-      jj_consume_token(SQLWARNING);
+      jj_consume_token(SQRT);
     } else if (jj_2_1123(2)) {
-      jj_consume_token(STATE);
+      jj_consume_token(STATIC);
     } else if (jj_2_1124(2)) {
-      jj_consume_token(STDDEV_SAMP);
+      jj_consume_token(SUBMULTISET);
     } else if (jj_2_1125(2)) {
-      jj_consume_token(SUBSTRING);
+      jj_consume_token(SUBSTRING_REGEX);
     } else if (jj_2_1126(2)) {
-      jj_consume_token(SUM);
+      jj_consume_token(SYMMETRIC);
     } else if (jj_2_1127(2)) {
-      jj_consume_token(SYSTEM_TIME);
+      jj_consume_token(SYSTEM_USER);
     } else if (jj_2_1128(2)) {
-      jj_consume_token(TIMEZONE_HOUR);
+      jj_consume_token(TEMPORARY);
     } else if (jj_2_1129(2)) {
-      jj_consume_token(TO);
+      jj_consume_token(TINYINT);
     } else if (jj_2_1130(2)) {
-      jj_consume_token(TRANSLATE);
+      jj_consume_token(TRANSACTION);
     } else if (jj_2_1131(2)) {
-      jj_consume_token(TREAT);
+      jj_consume_token(TRANSLATION);
     } else if (jj_2_1132(2)) {
-      jj_consume_token(TRIM_ARRAY);
+      jj_consume_token(TRIM);
     } else if (jj_2_1133(2)) {
-      jj_consume_token(UESCAPE);
+      jj_consume_token(TRUNCATE);
     } else if (jj_2_1134(2)) {
-      jj_consume_token(UNKNOWN);
+      jj_consume_token(UNIQUE);
     } else if (jj_2_1135(2)) {
-      jj_consume_token(USAGE);
+      jj_consume_token(UPSERT);
     } else if (jj_2_1136(2)) {
-      jj_consume_token(VALUE_OF);
+      jj_consume_token(VALUE);
     } else if (jj_2_1137(2)) {
-      jj_consume_token(VARYING);
+      jj_consume_token(VARCHAR);
     } else if (jj_2_1138(2)) {
-      jj_consume_token(VERSION);
+      jj_consume_token(VAR_SAMP);
     } else if (jj_2_1139(2)) {
-      jj_consume_token(WEEK);
+      jj_consume_token(VIEW);
     } else if (jj_2_1140(2)) {
-      jj_consume_token(WITHIN);
+      jj_consume_token(WIDTH_BUCKET);
     } else if (jj_2_1141(2)) {
-      jj_consume_token(WRITE);
+      jj_consume_token(WORK);
+    } else if (jj_2_1142(2)) {
+      jj_consume_token(ZONE);
     } else {
       jj_consume_token(-1);
       throw new ParseException();
@@ -9171,268 +9173,268 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
 
 /** @see #NonReservedKeyWord */
   final public void NonReservedKeyWord1of3() throws ParseException {
-    if (jj_2_1142(2)) {
+    if (jj_2_1143(2)) {
       jj_consume_token(TEMPLATE);
-    } else if (jj_2_1143(2)) {
-      jj_consume_token(ATOMICITY);
     } else if (jj_2_1144(2)) {
-      jj_consume_token(CACHE_NAME);
+      jj_consume_token(ATOMICITY);
     } else if (jj_2_1145(2)) {
-      jj_consume_token(ENCRYPTED);
+      jj_consume_token(CACHE_NAME);
     } else if (jj_2_1146(2)) {
-      jj_consume_token(LOGGING);
+      jj_consume_token(ENCRYPTED);
     } else if (jj_2_1147(2)) {
-      jj_consume_token(KILL);
+      jj_consume_token(LOGGING);
     } else if (jj_2_1148(2)) {
-      jj_consume_token(SERVICE);
+      jj_consume_token(KILL);
     } else if (jj_2_1149(2)) {
-      jj_consume_token(QUERY);
+      jj_consume_token(SERVICE);
     } else if (jj_2_1150(2)) {
-      jj_consume_token(ABS);
+      jj_consume_token(QUERY);
     } else if (jj_2_1151(2)) {
-      jj_consume_token(ADD);
+      jj_consume_token(ABSOLUTE);
     } else if (jj_2_1152(2)) {
-      jj_consume_token(ALLOW);
+      jj_consume_token(AFTER);
     } else if (jj_2_1153(2)) {
-      jj_consume_token(ARE);
+      jj_consume_token(ALTER);
     } else if (jj_2_1154(2)) {
-      jj_consume_token(AS);
+      jj_consume_token(ARRAY);
     } else if (jj_2_1155(2)) {
-      jj_consume_token(ASSERTION);
+      jj_consume_token(ASC);
     } else if (jj_2_1156(2)) {
-      jj_consume_token(ATOMIC);
+      jj_consume_token(ASYMMETRIC);
     } else if (jj_2_1157(2)) {
-      jj_consume_token(BEFORE);
+      jj_consume_token(AUTHORIZATION);
     } else if (jj_2_1158(2)) {
-      jj_consume_token(BEGIN_PARTITION);
+      jj_consume_token(BEGIN);
     } else if (jj_2_1159(2)) {
-      jj_consume_token(BINARY);
+      jj_consume_token(BETWEEN);
     } else if (jj_2_1160(2)) {
-      jj_consume_token(BOOLEAN);
+      jj_consume_token(BIT);
     } else if (jj_2_1161(2)) {
-      jj_consume_token(BY);
+      jj_consume_token(BOTH);
     } else if (jj_2_1162(2)) {
-      jj_consume_token(CARDINALITY);
+      jj_consume_token(C);
     } else if (jj_2_1163(2)) {
-      jj_consume_token(CAST);
+      jj_consume_token(CASCADE);
     } else if (jj_2_1164(2)) {
-      jj_consume_token(CEILING);
+      jj_consume_token(CATALOG);
     } else if (jj_2_1165(2)) {
-      jj_consume_token(CHARACTER_LENGTH);
+      jj_consume_token(CHAR);
     } else if (jj_2_1166(2)) {
-      jj_consume_token(CLASSIFIER);
+      jj_consume_token(CHAR_LENGTH);
     } else if (jj_2_1167(2)) {
-      jj_consume_token(COALESCE);
+      jj_consume_token(CLOB);
     } else if (jj_2_1168(2)) {
-      jj_consume_token(COLLECT);
+      jj_consume_token(COLLATE);
     } else if (jj_2_1169(2)) {
-      jj_consume_token(CONDITION);
+      jj_consume_token(COLUMN);
     } else if (jj_2_1170(2)) {
-      jj_consume_token(CONSTRAINTS);
+      jj_consume_token(CONNECT);
     } else if (jj_2_1171(2)) {
-      jj_consume_token(CONTINUE);
+      jj_consume_token(CONSTRUCTOR);
     } else if (jj_2_1172(2)) {
-      jj_consume_token(CORRESPONDING);
+      jj_consume_token(CONVERT);
     } else if (jj_2_1173(2)) {
-      jj_consume_token(COVAR_SAMP);
+      jj_consume_token(COUNT);
     } else if (jj_2_1174(2)) {
-      jj_consume_token(CURRENT_CATALOG);
+      jj_consume_token(CUBE);
     } else if (jj_2_1175(2)) {
-      jj_consume_token(CURRENT_PATH);
+      jj_consume_token(CURRENT_DATE);
     } else if (jj_2_1176(2)) {
-      jj_consume_token(CURRENT_SCHEMA);
+      jj_consume_token(CURRENT_ROLE);
     } else if (jj_2_1177(2)) {
-      jj_consume_token(CURRENT_TRANSFORM_GROUP_FOR_TYPE);
+      jj_consume_token(CURRENT_TIME);
     } else if (jj_2_1178(2)) {
-      jj_consume_token(DATA);
+      jj_consume_token(CURRENT_USER);
     } else if (jj_2_1179(2)) {
-      jj_consume_token(DEALLOCATE);
+      jj_consume_token(DATE);
     } else if (jj_2_1180(2)) {
-      jj_consume_token(DECLARE);
+      jj_consume_token(DEC);
     } else if (jj_2_1181(2)) {
-      jj_consume_token(DENSE_RANK);
+      jj_consume_token(DEFERRABLE);
     } else if (jj_2_1182(2)) {
-      jj_consume_token(DESC);
+      jj_consume_token(DEPTH);
     } else if (jj_2_1183(2)) {
-      jj_consume_token(DIAGNOSTICS);
+      jj_consume_token(DESCRIPTOR);
     } else if (jj_2_1184(2)) {
-      jj_consume_token(DOMAIN);
+      jj_consume_token(DISALLOW);
     } else if (jj_2_1185(2)) {
-      jj_consume_token(EACH);
+      jj_consume_token(DOUBLE);
     } else if (jj_2_1186(2)) {
-      jj_consume_token(EMPTY);
+      jj_consume_token(ELEMENT);
     } else if (jj_2_1187(2)) {
-      jj_consume_token(END_PARTITION);
+      jj_consume_token(END);
     } else if (jj_2_1188(2)) {
-      jj_consume_token(EVERY);
+      jj_consume_token(EQUALS);
     } else if (jj_2_1189(2)) {
-      jj_consume_token(EXECUTE);
+      jj_consume_token(EXCEPTION);
     } else if (jj_2_1190(2)) {
-      jj_consume_token(EXTEND);
+      jj_consume_token(EXISTS);
     } else if (jj_2_1191(2)) {
-      jj_consume_token(FALSE);
+      jj_consume_token(EXTERNAL);
     } else if (jj_2_1192(2)) {
-      jj_consume_token(FIRST_VALUE);
+      jj_consume_token(FILTER);
     } else if (jj_2_1193(2)) {
-      jj_consume_token(FOR);
+      jj_consume_token(FLOAT);
     } else if (jj_2_1194(2)) {
-      jj_consume_token(FRAME_ROW);
+      jj_consume_token(FOREIGN);
     } else if (jj_2_1195(2)) {
-      jj_consume_token(FUSION);
+      jj_consume_token(FREE);
     } else if (jj_2_1196(2)) {
-      jj_consume_token(GET);
+      jj_consume_token(G);
     } else if (jj_2_1197(2)) {
-      jj_consume_token(GOTO);
+      jj_consume_token(GLOBAL);
     } else if (jj_2_1198(2)) {
-      jj_consume_token(HOUR);
+      jj_consume_token(GROUPS);
     } else if (jj_2_1199(2)) {
-      jj_consume_token(IMMEDIATELY);
+      jj_consume_token(IDENTITY);
     } else if (jj_2_1200(2)) {
-      jj_consume_token(INITIAL);
+      jj_consume_token(IMPORT);
     } else if (jj_2_1201(2)) {
-      jj_consume_token(INPUT);
+      jj_consume_token(INITIALLY);
     } else if (jj_2_1202(2)) {
-      jj_consume_token(INTEGER);
+      jj_consume_token(INSENSITIVE);
     } else if (jj_2_1203(2)) {
-      jj_consume_token(ISOLATION);
+      jj_consume_token(INTERSECTION);
     } else if (jj_2_1204(2)) {
-      jj_consume_token(JSON_EXISTS);
+      jj_consume_token(JSON_ARRAY);
     } else if (jj_2_1205(2)) {
-      jj_consume_token(JSON_QUERY);
+      jj_consume_token(JSON_OBJECT);
     } else if (jj_2_1206(2)) {
-      jj_consume_token(KEY);
+      jj_consume_token(JSON_VALUE);
     } else if (jj_2_1207(2)) {
-      jj_consume_token(LARGE);
+      jj_consume_token(LAG);
     } else if (jj_2_1208(2)) {
-      jj_consume_token(LEAD);
+      jj_consume_token(LAST);
     } else if (jj_2_1209(2)) {
-      jj_consume_token(LEVEL);
+      jj_consume_token(LEADING);
     } else if (jj_2_1210(2)) {
-      jj_consume_token(LN);
+      jj_consume_token(LIKE);
     } else if (jj_2_1211(2)) {
-      jj_consume_token(LOCALTIMESTAMP);
+      jj_consume_token(LOCAL);
     } else if (jj_2_1212(2)) {
-      jj_consume_token(M);
+      jj_consume_token(LOCATOR);
     } else if (jj_2_1213(2)) {
-      jj_consume_token(MATCHES);
+      jj_consume_token(MAP);
     } else if (jj_2_1214(2)) {
-      jj_consume_token(MEASURES);
+      jj_consume_token(MATCH_NUMBER);
     } else if (jj_2_1215(2)) {
-      jj_consume_token(MIN);
+      jj_consume_token(MEMBER);
     } else if (jj_2_1216(2)) {
-      jj_consume_token(MODIFIES);
+      jj_consume_token(MINUTE);
     } else if (jj_2_1217(2)) {
-      jj_consume_token(MULTISET);
+      jj_consume_token(MODULE);
     } else if (jj_2_1218(2)) {
-      jj_consume_token(NATIONAL);
+      jj_consume_token(NAME);
     } else if (jj_2_1219(2)) {
-      jj_consume_token(NO);
+      jj_consume_token(NCHAR);
     } else if (jj_2_1220(2)) {
-      jj_consume_token(NOT);
+      jj_consume_token(NONE);
     } else if (jj_2_1221(2)) {
-      jj_consume_token(NULLIF);
+      jj_consume_token(NTH_VALUE);
     } else if (jj_2_1222(2)) {
-      jj_consume_token(OCCURRENCES_REGEX);
+      jj_consume_token(NUMERIC);
     } else if (jj_2_1223(2)) {
-      jj_consume_token(OLD);
+      jj_consume_token(OCTET_LENGTH);
     } else if (jj_2_1224(2)) {
-      jj_consume_token(ONLY);
+      jj_consume_token(OMIT);
     } else if (jj_2_1225(2)) {
-      jj_consume_token(OR);
+      jj_consume_token(OPEN);
     } else if (jj_2_1226(2)) {
-      jj_consume_token(OUTPUT);
+      jj_consume_token(ORDINALITY);
     } else if (jj_2_1227(2)) {
-      jj_consume_token(PAD);
+      jj_consume_token(OVERLAPS);
     } else if (jj_2_1228(2)) {
-      jj_consume_token(PATH);
+      jj_consume_token(PARAMETER);
     } else if (jj_2_1229(2)) {
-      jj_consume_token(PERCENTILE_CONT);
+      jj_consume_token(PER);
     } else if (jj_2_1230(2)) {
-      jj_consume_token(PERIOD);
+      jj_consume_token(PERCENTILE_DISC);
     } else if (jj_2_1231(2)) {
-      jj_consume_token(POSITION);
+      jj_consume_token(PERMUTE);
     } else if (jj_2_1232(2)) {
-      jj_consume_token(PRECEDES);
+      jj_consume_token(POSITION_REGEX);
     } else if (jj_2_1233(2)) {
-      jj_consume_token(PRESERVE);
+      jj_consume_token(PRECISION);
     } else if (jj_2_1234(2)) {
-      jj_consume_token(PRIVILEGES);
+      jj_consume_token(PREV);
     } else if (jj_2_1235(2)) {
-      jj_consume_token(QUARTER);
+      jj_consume_token(PROCEDURE);
     } else if (jj_2_1236(2)) {
-      jj_consume_token(READS);
+      jj_consume_token(RANK);
     } else if (jj_2_1237(2)) {
-      jj_consume_token(REF);
+      jj_consume_token(REAL);
     } else if (jj_2_1238(2)) {
-      jj_consume_token(REGR_AVGX);
+      jj_consume_token(REFERENCES);
     } else if (jj_2_1239(2)) {
-      jj_consume_token(REGR_INTERCEPT);
+      jj_consume_token(REGR_AVGY);
     } else if (jj_2_1240(2)) {
-      jj_consume_token(REGR_SXX);
+      jj_consume_token(REGR_R2);
     } else if (jj_2_1241(2)) {
-      jj_consume_token(RELATIVE);
+      jj_consume_token(REGR_SXY);
     } else if (jj_2_1242(2)) {
-      jj_consume_token(RESET);
+      jj_consume_token(RELEASE);
     } else if (jj_2_1243(2)) {
-      jj_consume_token(RETURN);
+      jj_consume_token(RESTRICT);
     } else if (jj_2_1244(2)) {
-      jj_consume_token(ROLE);
+      jj_consume_token(RETURNS);
     } else if (jj_2_1245(2)) {
-      jj_consume_token(ROW_NUMBER);
+      jj_consume_token(ROLLBACK);
     } else if (jj_2_1246(2)) {
-      jj_consume_token(SCHEMA);
+      jj_consume_token(RUNNING);
     } else if (jj_2_1247(2)) {
-      jj_consume_token(SEARCH);
+      jj_consume_token(SCOPE);
     } else if (jj_2_1248(2)) {
-      jj_consume_token(SEEK);
+      jj_consume_token(SECOND);
     } else if (jj_2_1249(2)) {
-      jj_consume_token(SESSION_USER);
+      jj_consume_token(SENSITIVE);
     } else if (jj_2_1250(2)) {
-      jj_consume_token(SIZE);
+      jj_consume_token(SHOW);
     } else if (jj_2_1251(2)) {
-      jj_consume_token(SPECIFIC);
+      jj_consume_token(SMALLINT);
     } else if (jj_2_1252(2)) {
-      jj_consume_token(SQLEXCEPTION);
+      jj_consume_token(SPECIFICTYPE);
     } else if (jj_2_1253(2)) {
-      jj_consume_token(SQRT);
+      jj_consume_token(SQLSTATE);
     } else if (jj_2_1254(2)) {
-      jj_consume_token(STATIC);
+      jj_consume_token(START);
     } else if (jj_2_1255(2)) {
-      jj_consume_token(SUBMULTISET);
+      jj_consume_token(STDDEV_POP);
     } else if (jj_2_1256(2)) {
-      jj_consume_token(SUBSTRING_REGEX);
+      jj_consume_token(SUBSET);
     } else if (jj_2_1257(2)) {
-      jj_consume_token(SYMMETRIC);
+      jj_consume_token(SUCCEEDS);
     } else if (jj_2_1258(2)) {
-      jj_consume_token(SYSTEM_USER);
+      jj_consume_token(SYSTEM);
     } else if (jj_2_1259(2)) {
-      jj_consume_token(TIMEZONE_MINUTE);
+      jj_consume_token(STRING_AGG);
     } else if (jj_2_1260(2)) {
-      jj_consume_token(TRAILING);
+      jj_consume_token(TIMEZONE_HOUR);
     } else if (jj_2_1261(2)) {
-      jj_consume_token(TRANSLATE_REGEX);
+      jj_consume_token(TO);
     } else if (jj_2_1262(2)) {
-      jj_consume_token(TRIGGER);
+      jj_consume_token(TRANSLATE);
     } else if (jj_2_1263(2)) {
-      jj_consume_token(TRUE);
+      jj_consume_token(TREAT);
     } else if (jj_2_1264(2)) {
-      jj_consume_token(UNDER);
+      jj_consume_token(TRIM_ARRAY);
     } else if (jj_2_1265(2)) {
-      jj_consume_token(UPPER);
+      jj_consume_token(UESCAPE);
     } else if (jj_2_1266(2)) {
-      jj_consume_token(USER);
+      jj_consume_token(UNKNOWN);
     } else if (jj_2_1267(2)) {
-      jj_consume_token(VARBINARY);
+      jj_consume_token(USAGE);
     } else if (jj_2_1268(2)) {
-      jj_consume_token(VAR_POP);
+      jj_consume_token(VALUE_OF);
     } else if (jj_2_1269(2)) {
-      jj_consume_token(VERSIONING);
+      jj_consume_token(VARYING);
     } else if (jj_2_1270(2)) {
-      jj_consume_token(WHENEVER);
+      jj_consume_token(VERSION);
     } else if (jj_2_1271(2)) {
-      jj_consume_token(WITHOUT);
+      jj_consume_token(WEEK);
     } else if (jj_2_1272(2)) {
-      jj_consume_token(YEAR);
+      jj_consume_token(WITHIN);
+    } else if (jj_2_1273(2)) {
+      jj_consume_token(WRITE);
     } else {
       jj_consume_token(-1);
       throw new ParseException();
@@ -9441,268 +9443,268 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
 
 /** @see #NonReservedKeyWord */
   final public void NonReservedKeyWord2of3() throws ParseException {
-    if (jj_2_1273(2)) {
+    if (jj_2_1274(2)) {
       jj_consume_token(BACKUPS);
-    } else if (jj_2_1274(2)) {
-      jj_consume_token(WRITE_SYNCHRONIZATION_MODE);
     } else if (jj_2_1275(2)) {
-      jj_consume_token(DATA_REGION);
+      jj_consume_token(WRITE_SYNCHRONIZATION_MODE);
     } else if (jj_2_1276(2)) {
-      jj_consume_token(PARALLEL);
+      jj_consume_token(DATA_REGION);
     } else if (jj_2_1277(2)) {
-      jj_consume_token(NOLOGGING);
+      jj_consume_token(PARALLEL);
     } else if (jj_2_1278(2)) {
-      jj_consume_token(SCAN);
+      jj_consume_token(NOLOGGING);
     } else if (jj_2_1279(2)) {
-      jj_consume_token(COMPUTE);
+      jj_consume_token(SCAN);
     } else if (jj_2_1280(2)) {
-      jj_consume_token(UUID);
+      jj_consume_token(COMPUTE);
     } else if (jj_2_1281(2)) {
-      jj_consume_token(ABSOLUTE);
+      jj_consume_token(A);
     } else if (jj_2_1282(2)) {
-      jj_consume_token(AFTER);
+      jj_consume_token(ACTION);
     } else if (jj_2_1283(2)) {
-      jj_consume_token(ALTER);
+      jj_consume_token(ALLOCATE);
     } else if (jj_2_1284(2)) {
-      jj_consume_token(ARRAY);
+      jj_consume_token(AND);
     } else if (jj_2_1285(2)) {
-      jj_consume_token(ASC);
+      jj_consume_token(ARRAY_MAX_CARDINALITY);
     } else if (jj_2_1286(2)) {
-      jj_consume_token(ASYMMETRIC);
+      jj_consume_token(ASENSITIVE);
     } else if (jj_2_1287(2)) {
-      jj_consume_token(AUTHORIZATION);
+      jj_consume_token(AT);
     } else if (jj_2_1288(2)) {
-      jj_consume_token(BEGIN);
+      jj_consume_token(AVG);
     } else if (jj_2_1289(2)) {
-      jj_consume_token(BETWEEN);
+      jj_consume_token(BEGIN_FRAME);
     } else if (jj_2_1290(2)) {
-      jj_consume_token(BIT);
+      jj_consume_token(BIGINT);
     } else if (jj_2_1291(2)) {
-      jj_consume_token(BOTH);
+      jj_consume_token(BLOB);
     } else if (jj_2_1292(2)) {
-      jj_consume_token(C);
+      jj_consume_token(BREADTH);
     } else if (jj_2_1293(2)) {
-      jj_consume_token(CASCADE);
+      jj_consume_token(CALLED);
     } else if (jj_2_1294(2)) {
-      jj_consume_token(CATALOG);
+      jj_consume_token(CASCADED);
     } else if (jj_2_1295(2)) {
-      jj_consume_token(CHAR);
+      jj_consume_token(CEIL);
     } else if (jj_2_1296(2)) {
-      jj_consume_token(CHAR_LENGTH);
+      jj_consume_token(CHARACTER);
     } else if (jj_2_1297(2)) {
-      jj_consume_token(CLOB);
+      jj_consume_token(CHECK);
     } else if (jj_2_1298(2)) {
-      jj_consume_token(COLLATE);
+      jj_consume_token(CLOSE);
     } else if (jj_2_1299(2)) {
-      jj_consume_token(COLUMN);
+      jj_consume_token(COLLATION);
     } else if (jj_2_1300(2)) {
-      jj_consume_token(CONNECT);
+      jj_consume_token(COMMIT);
     } else if (jj_2_1301(2)) {
-      jj_consume_token(CONSTRUCTOR);
+      jj_consume_token(CONNECTION);
     } else if (jj_2_1302(2)) {
-      jj_consume_token(CONVERT);
+      jj_consume_token(CONTAINS);
     } else if (jj_2_1303(2)) {
-      jj_consume_token(COUNT);
+      jj_consume_token(CORR);
     } else if (jj_2_1304(2)) {
-      jj_consume_token(CUBE);
+      jj_consume_token(COVAR_POP);
     } else if (jj_2_1305(2)) {
-      jj_consume_token(CURRENT_DATE);
+      jj_consume_token(CUME_DIST);
     } else if (jj_2_1306(2)) {
-      jj_consume_token(CURRENT_ROLE);
+      jj_consume_token(CURRENT_DEFAULT_TRANSFORM_GROUP);
     } else if (jj_2_1307(2)) {
-      jj_consume_token(CURRENT_TIME);
+      jj_consume_token(CURRENT_ROW);
     } else if (jj_2_1308(2)) {
-      jj_consume_token(CURRENT_USER);
+      jj_consume_token(CURRENT_TIMESTAMP);
     } else if (jj_2_1309(2)) {
-      jj_consume_token(DATE);
+      jj_consume_token(CYCLE);
     } else if (jj_2_1310(2)) {
-      jj_consume_token(DEC);
+      jj_consume_token(DAY);
     } else if (jj_2_1311(2)) {
-      jj_consume_token(DEFERRABLE);
+      jj_consume_token(DECIMAL);
     } else if (jj_2_1312(2)) {
-      jj_consume_token(DEPTH);
+      jj_consume_token(DEFERRED);
     } else if (jj_2_1313(2)) {
-      jj_consume_token(DESCRIPTOR);
+      jj_consume_token(DEREF);
     } else if (jj_2_1314(2)) {
-      jj_consume_token(DISALLOW);
+      jj_consume_token(DETERMINISTIC);
     } else if (jj_2_1315(2)) {
-      jj_consume_token(DOUBLE);
+      jj_consume_token(DISCONNECT);
     } else if (jj_2_1316(2)) {
-      jj_consume_token(ELEMENT);
+      jj_consume_token(DYNAMIC);
     } else if (jj_2_1317(2)) {
-      jj_consume_token(END);
+      jj_consume_token(ELSE);
     } else if (jj_2_1318(2)) {
-      jj_consume_token(EQUALS);
+      jj_consume_token(END_FRAME);
     } else if (jj_2_1319(2)) {
-      jj_consume_token(EXCEPTION);
+      jj_consume_token(ESCAPE);
     } else if (jj_2_1320(2)) {
-      jj_consume_token(EXISTS);
+      jj_consume_token(EXEC);
     } else if (jj_2_1321(2)) {
-      jj_consume_token(EXTERNAL);
+      jj_consume_token(EXP);
     } else if (jj_2_1322(2)) {
-      jj_consume_token(FILTER);
+      jj_consume_token(EXTRACT);
     } else if (jj_2_1323(2)) {
-      jj_consume_token(FLOAT);
+      jj_consume_token(FIRST);
     } else if (jj_2_1324(2)) {
-      jj_consume_token(FOREIGN);
+      jj_consume_token(FLOOR);
     } else if (jj_2_1325(2)) {
-      jj_consume_token(FREE);
+      jj_consume_token(FOUND);
     } else if (jj_2_1326(2)) {
-      jj_consume_token(G);
+      jj_consume_token(FUNCTION);
     } else if (jj_2_1327(2)) {
-      jj_consume_token(GLOBAL);
+      jj_consume_token(GENERAL);
     } else if (jj_2_1328(2)) {
-      jj_consume_token(GROUPS);
+      jj_consume_token(GO);
     } else if (jj_2_1329(2)) {
-      jj_consume_token(IDENTITY);
+      jj_consume_token(HOLD);
     } else if (jj_2_1330(2)) {
-      jj_consume_token(IMPORT);
+      jj_consume_token(IMMEDIATE);
     } else if (jj_2_1331(2)) {
-      jj_consume_token(INITIALLY);
+      jj_consume_token(INDICATOR);
     } else if (jj_2_1332(2)) {
-      jj_consume_token(INSENSITIVE);
+      jj_consume_token(INOUT);
     } else if (jj_2_1333(2)) {
-      jj_consume_token(INTERSECTION);
+      jj_consume_token(INT);
     } else if (jj_2_1334(2)) {
-      jj_consume_token(JSON_ARRAY);
+      jj_consume_token(IS);
     } else if (jj_2_1335(2)) {
-      jj_consume_token(JSON_OBJECT);
+      jj_consume_token(JSON_ARRAYAGG);
     } else if (jj_2_1336(2)) {
-      jj_consume_token(JSON_VALUE);
+      jj_consume_token(JSON_OBJECTAGG);
     } else if (jj_2_1337(2)) {
-      jj_consume_token(LAG);
+      jj_consume_token(K);
     } else if (jj_2_1338(2)) {
-      jj_consume_token(LAST);
+      jj_consume_token(LANGUAGE);
     } else if (jj_2_1339(2)) {
-      jj_consume_token(LEADING);
+      jj_consume_token(LAST_VALUE);
     } else if (jj_2_1340(2)) {
-      jj_consume_token(LIKE);
+      jj_consume_token(LENGTH);
     } else if (jj_2_1341(2)) {
-      jj_consume_token(LOCAL);
+      jj_consume_token(LIKE_REGEX);
     } else if (jj_2_1342(2)) {
-      jj_consume_token(LOCATOR);
+      jj_consume_token(LOCALTIME);
     } else if (jj_2_1343(2)) {
-      jj_consume_token(MAP);
+      jj_consume_token(LOWER);
     } else if (jj_2_1344(2)) {
-      jj_consume_token(MATCH_NUMBER);
+      jj_consume_token(MATCH);
     } else if (jj_2_1345(2)) {
-      jj_consume_token(MEMBER);
+      jj_consume_token(MAX);
     } else if (jj_2_1346(2)) {
-      jj_consume_token(MINUTE);
+      jj_consume_token(METHOD);
     } else if (jj_2_1347(2)) {
-      jj_consume_token(MODULE);
+      jj_consume_token(MOD);
     } else if (jj_2_1348(2)) {
-      jj_consume_token(NAME);
+      jj_consume_token(MONTH);
     } else if (jj_2_1349(2)) {
-      jj_consume_token(NCHAR);
+      jj_consume_token(NAMES);
     } else if (jj_2_1350(2)) {
-      jj_consume_token(NONE);
+      jj_consume_token(NCLOB);
     } else if (jj_2_1351(2)) {
-      jj_consume_token(NTH_VALUE);
+      jj_consume_token(NORMALIZE);
     } else if (jj_2_1352(2)) {
-      jj_consume_token(NUMERIC);
+      jj_consume_token(NTILE);
     } else if (jj_2_1353(2)) {
-      jj_consume_token(OCTET_LENGTH);
+      jj_consume_token(OBJECT);
     } else if (jj_2_1354(2)) {
-      jj_consume_token(OMIT);
+      jj_consume_token(OF);
     } else if (jj_2_1355(2)) {
-      jj_consume_token(OPEN);
+      jj_consume_token(ONE);
     } else if (jj_2_1356(2)) {
-      jj_consume_token(ORDINALITY);
+      jj_consume_token(OPTION);
     } else if (jj_2_1357(2)) {
-      jj_consume_token(OVERLAPS);
+      jj_consume_token(OUT);
     } else if (jj_2_1358(2)) {
-      jj_consume_token(PARAMETER);
+      jj_consume_token(OVERLAY);
     } else if (jj_2_1359(2)) {
-      jj_consume_token(PER);
+      jj_consume_token(PARTIAL);
     } else if (jj_2_1360(2)) {
-      jj_consume_token(PERCENTILE_DISC);
+      jj_consume_token(PERCENT);
     } else if (jj_2_1361(2)) {
-      jj_consume_token(PERMUTE);
+      jj_consume_token(PERCENT_RANK);
     } else if (jj_2_1362(2)) {
-      jj_consume_token(POSITION_REGEX);
+      jj_consume_token(PORTION);
     } else if (jj_2_1363(2)) {
-      jj_consume_token(PRECISION);
+      jj_consume_token(POWER);
     } else if (jj_2_1364(2)) {
-      jj_consume_token(PREV);
+      jj_consume_token(PREPARE);
     } else if (jj_2_1365(2)) {
-      jj_consume_token(PROCEDURE);
+      jj_consume_token(PRIOR);
     } else if (jj_2_1366(2)) {
-      jj_consume_token(RANK);
+      jj_consume_token(PUBLIC);
     } else if (jj_2_1367(2)) {
-      jj_consume_token(REAL);
+      jj_consume_token(READ);
     } else if (jj_2_1368(2)) {
-      jj_consume_token(REFERENCES);
+      jj_consume_token(RECURSIVE);
     } else if (jj_2_1369(2)) {
-      jj_consume_token(REGR_AVGY);
+      jj_consume_token(REFERENCING);
     } else if (jj_2_1370(2)) {
-      jj_consume_token(REGR_R2);
+      jj_consume_token(REGR_COUNT);
     } else if (jj_2_1371(2)) {
-      jj_consume_token(REGR_SXY);
+      jj_consume_token(REGR_SLOPE);
     } else if (jj_2_1372(2)) {
-      jj_consume_token(RELEASE);
+      jj_consume_token(REGR_SYY);
     } else if (jj_2_1373(2)) {
-      jj_consume_token(RESTRICT);
+      jj_consume_token(REPLACE);
     } else if (jj_2_1374(2)) {
-      jj_consume_token(RETURNS);
+      jj_consume_token(RESULT);
     } else if (jj_2_1375(2)) {
-      jj_consume_token(ROLLBACK);
+      jj_consume_token(REVOKE);
     } else if (jj_2_1376(2)) {
-      jj_consume_token(RUNNING);
+      jj_consume_token(ROUTINE);
     } else if (jj_2_1377(2)) {
-      jj_consume_token(SCOPE);
+      jj_consume_token(SAVEPOINT);
     } else if (jj_2_1378(2)) {
-      jj_consume_token(SECOND);
+      jj_consume_token(SCROLL);
     } else if (jj_2_1379(2)) {
-      jj_consume_token(SENSITIVE);
+      jj_consume_token(SECTION);
     } else if (jj_2_1380(2)) {
-      jj_consume_token(SHOW);
+      jj_consume_token(SESSION);
     } else if (jj_2_1381(2)) {
-      jj_consume_token(SMALLINT);
+      jj_consume_token(SIMILAR);
     } else if (jj_2_1382(2)) {
-      jj_consume_token(SPECIFICTYPE);
+      jj_consume_token(SPACE);
     } else if (jj_2_1383(2)) {
-      jj_consume_token(SQLSTATE);
+      jj_consume_token(SQL);
     } else if (jj_2_1384(2)) {
-      jj_consume_token(START);
+      jj_consume_token(SQLWARNING);
     } else if (jj_2_1385(2)) {
-      jj_consume_token(STDDEV_POP);
+      jj_consume_token(STATE);
     } else if (jj_2_1386(2)) {
-      jj_consume_token(SUBSET);
+      jj_consume_token(STDDEV_SAMP);
     } else if (jj_2_1387(2)) {
-      jj_consume_token(SUCCEEDS);
+      jj_consume_token(SUBSTRING);
     } else if (jj_2_1388(2)) {
-      jj_consume_token(SYSTEM);
+      jj_consume_token(SUM);
     } else if (jj_2_1389(2)) {
-      jj_consume_token(TEMPORARY);
+      jj_consume_token(SYSTEM_TIME);
     } else if (jj_2_1390(2)) {
-      jj_consume_token(TINYINT);
+      jj_consume_token(GROUP_CONCAT);
     } else if (jj_2_1391(2)) {
-      jj_consume_token(TRANSACTION);
+      jj_consume_token(TIMEZONE_MINUTE);
     } else if (jj_2_1392(2)) {
-      jj_consume_token(TRANSLATION);
+      jj_consume_token(TRAILING);
     } else if (jj_2_1393(2)) {
-      jj_consume_token(TRIM);
+      jj_consume_token(TRANSLATE_REGEX);
     } else if (jj_2_1394(2)) {
-      jj_consume_token(TRUNCATE);
+      jj_consume_token(TRIGGER);
     } else if (jj_2_1395(2)) {
-      jj_consume_token(UNIQUE);
+      jj_consume_token(TRUE);
     } else if (jj_2_1396(2)) {
-      jj_consume_token(UPSERT);
+      jj_consume_token(UNDER);
     } else if (jj_2_1397(2)) {
-      jj_consume_token(VALUE);
+      jj_consume_token(UPPER);
     } else if (jj_2_1398(2)) {
-      jj_consume_token(VARCHAR);
+      jj_consume_token(USER);
     } else if (jj_2_1399(2)) {
-      jj_consume_token(VAR_SAMP);
+      jj_consume_token(VARBINARY);
     } else if (jj_2_1400(2)) {
-      jj_consume_token(VIEW);
+      jj_consume_token(VAR_POP);
     } else if (jj_2_1401(2)) {
-      jj_consume_token(WIDTH_BUCKET);
+      jj_consume_token(VERSIONING);
     } else if (jj_2_1402(2)) {
-      jj_consume_token(WORK);
+      jj_consume_token(WHENEVER);
     } else if (jj_2_1403(2)) {
-      jj_consume_token(ZONE);
+      jj_consume_token(WITHOUT);
+    } else if (jj_2_1404(2)) {
+      jj_consume_token(YEAR);
     } else {
       jj_consume_token(-1);
       throw new ParseException();
@@ -19547,6 +19549,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     finally { jj_save(1402, xla); }
   }
 
+  final private boolean jj_2_1404(int xla) {
+    jj_la = xla; jj_lastpos = jj_scanpos = token;
+    try { return !jj_3_1404(); }
+    catch(LookaheadSuccess ls) { return true; }
+    finally { jj_save(1403, xla); }
+  }
+
   final private boolean jj_3R_312() {
     if (jj_3R_66()) return true;
     return false;
@@ -20371,13 +20380,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1403() {
-    if (jj_scan_token(ZONE)) return true;
+  final private boolean jj_3_1404() {
+    if (jj_scan_token(YEAR)) return true;
     return false;
   }
 
-  final private boolean jj_3_1402() {
-    if (jj_scan_token(WORK)) return true;
+  final private boolean jj_3_1403() {
+    if (jj_scan_token(WITHOUT)) return true;
     return false;
   }
 
@@ -20386,13 +20395,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1401() {
-    if (jj_scan_token(WIDTH_BUCKET)) return true;
+  final private boolean jj_3_1402() {
+    if (jj_scan_token(WHENEVER)) return true;
     return false;
   }
 
-  final private boolean jj_3_1400() {
-    if (jj_scan_token(VIEW)) return true;
+  final private boolean jj_3_1401() {
+    if (jj_scan_token(VERSIONING)) return true;
     return false;
   }
 
@@ -20401,8 +20410,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1399() {
-    if (jj_scan_token(VAR_SAMP)) return true;
+  final private boolean jj_3_1400() {
+    if (jj_scan_token(VAR_POP)) return true;
     return false;
   }
 
@@ -20411,23 +20420,23 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1398() {
-    if (jj_scan_token(VARCHAR)) return true;
+  final private boolean jj_3_1399() {
+    if (jj_scan_token(VARBINARY)) return true;
     return false;
   }
 
-  final private boolean jj_3_1397() {
-    if (jj_scan_token(VALUE)) return true;
+  final private boolean jj_3_1398() {
+    if (jj_scan_token(USER)) return true;
     return false;
   }
 
-  final private boolean jj_3_1396() {
-    if (jj_scan_token(UPSERT)) return true;
+  final private boolean jj_3_1397() {
+    if (jj_scan_token(UPPER)) return true;
     return false;
   }
 
-  final private boolean jj_3_1395() {
-    if (jj_scan_token(UNIQUE)) return true;
+  final private boolean jj_3_1396() {
+    if (jj_scan_token(UNDER)) return true;
     return false;
   }
 
@@ -20449,28 +20458,28 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1395() {
+    if (jj_scan_token(TRUE)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1394() {
-    if (jj_scan_token(TRUNCATE)) return true;
+    if (jj_scan_token(TRIGGER)) return true;
     return false;
   }
 
   final private boolean jj_3_1393() {
-    if (jj_scan_token(TRIM)) return true;
+    if (jj_scan_token(TRANSLATE_REGEX)) return true;
     return false;
   }
 
   final private boolean jj_3_1392() {
-    if (jj_scan_token(TRANSLATION)) return true;
+    if (jj_scan_token(TRAILING)) return true;
     return false;
   }
 
   final private boolean jj_3_1391() {
-    if (jj_scan_token(TRANSACTION)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1390() {
-    if (jj_scan_token(TINYINT)) return true;
+    if (jj_scan_token(TIMEZONE_MINUTE)) return true;
     return false;
   }
 
@@ -20485,43 +20494,43 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1390() {
+    if (jj_scan_token(GROUP_CONCAT)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1389() {
-    if (jj_scan_token(TEMPORARY)) return true;
+    if (jj_scan_token(SYSTEM_TIME)) return true;
     return false;
   }
 
   final private boolean jj_3_1388() {
-    if (jj_scan_token(SYSTEM)) return true;
+    if (jj_scan_token(SUM)) return true;
     return false;
   }
 
   final private boolean jj_3_1387() {
-    if (jj_scan_token(SUCCEEDS)) return true;
+    if (jj_scan_token(SUBSTRING)) return true;
     return false;
   }
 
   final private boolean jj_3_1386() {
-    if (jj_scan_token(SUBSET)) return true;
+    if (jj_scan_token(STDDEV_SAMP)) return true;
     return false;
   }
 
   final private boolean jj_3_1385() {
-    if (jj_scan_token(STDDEV_POP)) return true;
+    if (jj_scan_token(STATE)) return true;
     return false;
   }
 
   final private boolean jj_3_1384() {
-    if (jj_scan_token(START)) return true;
+    if (jj_scan_token(SQLWARNING)) return true;
     return false;
   }
 
   final private boolean jj_3_1383() {
-    if (jj_scan_token(SQLSTATE)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1382() {
-    if (jj_scan_token(SPECIFICTYPE)) return true;
+    if (jj_scan_token(SQL)) return true;
     return false;
   }
 
@@ -20530,18 +20539,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1381() {
-    if (jj_scan_token(SMALLINT)) return true;
+  final private boolean jj_3_1382() {
+    if (jj_scan_token(SPACE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1380() {
-    if (jj_scan_token(SHOW)) return true;
+  final private boolean jj_3_1381() {
+    if (jj_scan_token(SIMILAR)) return true;
     return false;
   }
 
-  final private boolean jj_3_1379() {
-    if (jj_scan_token(SENSITIVE)) return true;
+  final private boolean jj_3_1380() {
+    if (jj_scan_token(SESSION)) return true;
     return false;
   }
 
@@ -20551,23 +20560,23 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1378() {
-    if (jj_scan_token(SECOND)) return true;
+  final private boolean jj_3_1379() {
+    if (jj_scan_token(SECTION)) return true;
     return false;
   }
 
-  final private boolean jj_3_1377() {
-    if (jj_scan_token(SCOPE)) return true;
+  final private boolean jj_3_1378() {
+    if (jj_scan_token(SCROLL)) return true;
     return false;
   }
 
-  final private boolean jj_3_1376() {
-    if (jj_scan_token(RUNNING)) return true;
+  final private boolean jj_3_1377() {
+    if (jj_scan_token(SAVEPOINT)) return true;
     return false;
   }
 
-  final private boolean jj_3_1375() {
-    if (jj_scan_token(ROLLBACK)) return true;
+  final private boolean jj_3_1376() {
+    if (jj_scan_token(ROUTINE)) return true;
     return false;
   }
 
@@ -20581,13 +20590,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1374() {
-    if (jj_scan_token(RETURNS)) return true;
+  final private boolean jj_3_1375() {
+    if (jj_scan_token(REVOKE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1373() {
-    if (jj_scan_token(RESTRICT)) return true;
+  final private boolean jj_3_1374() {
+    if (jj_scan_token(RESULT)) return true;
     return false;
   }
 
@@ -20602,73 +20611,73 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1373() {
+    if (jj_scan_token(REPLACE)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1372() {
-    if (jj_scan_token(RELEASE)) return true;
+    if (jj_scan_token(REGR_SYY)) return true;
     return false;
   }
 
   final private boolean jj_3_1371() {
-    if (jj_scan_token(REGR_SXY)) return true;
+    if (jj_scan_token(REGR_SLOPE)) return true;
     return false;
   }
 
   final private boolean jj_3_1370() {
-    if (jj_scan_token(REGR_R2)) return true;
+    if (jj_scan_token(REGR_COUNT)) return true;
     return false;
   }
 
   final private boolean jj_3_1369() {
-    if (jj_scan_token(REGR_AVGY)) return true;
+    if (jj_scan_token(REFERENCING)) return true;
     return false;
   }
 
   final private boolean jj_3_1368() {
-    if (jj_scan_token(REFERENCES)) return true;
+    if (jj_scan_token(RECURSIVE)) return true;
     return false;
   }
 
   final private boolean jj_3_1367() {
-    if (jj_scan_token(REAL)) return true;
+    if (jj_scan_token(READ)) return true;
     return false;
   }
 
   final private boolean jj_3_1366() {
-    if (jj_scan_token(RANK)) return true;
+    if (jj_scan_token(PUBLIC)) return true;
     return false;
   }
 
   final private boolean jj_3_1365() {
-    if (jj_scan_token(PROCEDURE)) return true;
+    if (jj_scan_token(PRIOR)) return true;
     return false;
   }
 
   final private boolean jj_3_1364() {
-    if (jj_scan_token(PREV)) return true;
+    if (jj_scan_token(PREPARE)) return true;
     return false;
   }
 
   final private boolean jj_3_1363() {
-    if (jj_scan_token(PRECISION)) return true;
+    if (jj_scan_token(POWER)) return true;
     return false;
   }
 
   final private boolean jj_3_1362() {
-    if (jj_scan_token(POSITION_REGEX)) return true;
+    if (jj_scan_token(PORTION)) return true;
     return false;
   }
 
   final private boolean jj_3_1361() {
-    if (jj_scan_token(PERMUTE)) return true;
+    if (jj_scan_token(PERCENT_RANK)) return true;
     return false;
   }
 
   final private boolean jj_3_1360() {
-    if (jj_scan_token(PERCENTILE_DISC)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1359() {
-    if (jj_scan_token(PER)) return true;
+    if (jj_scan_token(PERCENT)) return true;
     return false;
   }
 
@@ -20677,8 +20686,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1358() {
-    if (jj_scan_token(PARAMETER)) return true;
+  final private boolean jj_3_1359() {
+    if (jj_scan_token(PARTIAL)) return true;
     return false;
   }
 
@@ -20687,8 +20696,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1357() {
-    if (jj_scan_token(OVERLAPS)) return true;
+  final private boolean jj_3_1358() {
+    if (jj_scan_token(OVERLAY)) return true;
     return false;
   }
 
@@ -20698,8 +20707,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1356() {
-    if (jj_scan_token(ORDINALITY)) return true;
+  final private boolean jj_3_1357() {
+    if (jj_scan_token(OUT)) return true;
     return false;
   }
 
@@ -20713,23 +20722,23 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1355() {
-    if (jj_scan_token(OPEN)) return true;
+  final private boolean jj_3_1356() {
+    if (jj_scan_token(OPTION)) return true;
     return false;
   }
 
-  final private boolean jj_3_1354() {
-    if (jj_scan_token(OMIT)) return true;
+  final private boolean jj_3_1355() {
+    if (jj_scan_token(ONE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1353() {
-    if (jj_scan_token(OCTET_LENGTH)) return true;
+  final private boolean jj_3_1354() {
+    if (jj_scan_token(OF)) return true;
     return false;
   }
 
-  final private boolean jj_3_1352() {
-    if (jj_scan_token(NUMERIC)) return true;
+  final private boolean jj_3_1353() {
+    if (jj_scan_token(OBJECT)) return true;
     return false;
   }
 
@@ -20738,13 +20747,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1351() {
-    if (jj_scan_token(NTH_VALUE)) return true;
+  final private boolean jj_3_1352() {
+    if (jj_scan_token(NTILE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1350() {
-    if (jj_scan_token(NONE)) return true;
+  final private boolean jj_3_1351() {
+    if (jj_scan_token(NORMALIZE)) return true;
     return false;
   }
 
@@ -20754,13 +20763,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1349() {
-    if (jj_scan_token(NCHAR)) return true;
+  final private boolean jj_3_1350() {
+    if (jj_scan_token(NCLOB)) return true;
     return false;
   }
 
-  final private boolean jj_3_1348() {
-    if (jj_scan_token(NAME)) return true;
+  final private boolean jj_3_1349() {
+    if (jj_scan_token(NAMES)) return true;
     return false;
   }
 
@@ -20774,18 +20783,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1347() {
-    if (jj_scan_token(MODULE)) return true;
+  final private boolean jj_3_1348() {
+    if (jj_scan_token(MONTH)) return true;
     return false;
   }
 
-  final private boolean jj_3_1346() {
-    if (jj_scan_token(MINUTE)) return true;
+  final private boolean jj_3_1347() {
+    if (jj_scan_token(MOD)) return true;
     return false;
   }
 
-  final private boolean jj_3_1345() {
-    if (jj_scan_token(MEMBER)) return true;
+  final private boolean jj_3_1346() {
+    if (jj_scan_token(METHOD)) return true;
     return false;
   }
 
@@ -20794,13 +20803,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1344() {
-    if (jj_scan_token(MATCH_NUMBER)) return true;
+  final private boolean jj_3_1345() {
+    if (jj_scan_token(MAX)) return true;
     return false;
   }
 
-  final private boolean jj_3_1343() {
-    if (jj_scan_token(MAP)) return true;
+  final private boolean jj_3_1344() {
+    if (jj_scan_token(MATCH)) return true;
     return false;
   }
 
@@ -20814,43 +20823,43 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1343() {
+    if (jj_scan_token(LOWER)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1342() {
-    if (jj_scan_token(LOCATOR)) return true;
+    if (jj_scan_token(LOCALTIME)) return true;
     return false;
   }
 
   final private boolean jj_3_1341() {
-    if (jj_scan_token(LOCAL)) return true;
+    if (jj_scan_token(LIKE_REGEX)) return true;
     return false;
   }
 
   final private boolean jj_3_1340() {
-    if (jj_scan_token(LIKE)) return true;
+    if (jj_scan_token(LENGTH)) return true;
     return false;
   }
 
   final private boolean jj_3_1339() {
-    if (jj_scan_token(LEADING)) return true;
+    if (jj_scan_token(LAST_VALUE)) return true;
     return false;
   }
 
   final private boolean jj_3_1338() {
-    if (jj_scan_token(LAST)) return true;
+    if (jj_scan_token(LANGUAGE)) return true;
     return false;
   }
 
   final private boolean jj_3_1337() {
-    if (jj_scan_token(LAG)) return true;
+    if (jj_scan_token(K)) return true;
     return false;
   }
 
   final private boolean jj_3_1336() {
-    if (jj_scan_token(JSON_VALUE)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1335() {
-    if (jj_scan_token(JSON_OBJECT)) return true;
+    if (jj_scan_token(JSON_OBJECTAGG)) return true;
     return false;
   }
 
@@ -20859,8 +20868,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1334() {
-    if (jj_scan_token(JSON_ARRAY)) return true;
+  final private boolean jj_3_1335() {
+    if (jj_scan_token(JSON_ARRAYAGG)) return true;
     return false;
   }
 
@@ -20873,23 +20882,23 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1333() {
-    if (jj_scan_token(INTERSECTION)) return true;
+  final private boolean jj_3_1334() {
+    if (jj_scan_token(IS)) return true;
     return false;
   }
 
-  final private boolean jj_3_1332() {
-    if (jj_scan_token(INSENSITIVE)) return true;
+  final private boolean jj_3_1333() {
+    if (jj_scan_token(INT)) return true;
     return false;
   }
 
-  final private boolean jj_3_1331() {
-    if (jj_scan_token(INITIALLY)) return true;
+  final private boolean jj_3_1332() {
+    if (jj_scan_token(INOUT)) return true;
     return false;
   }
 
-  final private boolean jj_3_1330() {
-    if (jj_scan_token(IMPORT)) return true;
+  final private boolean jj_3_1331() {
+    if (jj_scan_token(INDICATOR)) return true;
     return false;
   }
 
@@ -20899,13 +20908,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1329() {
-    if (jj_scan_token(IDENTITY)) return true;
+  final private boolean jj_3_1330() {
+    if (jj_scan_token(IMMEDIATE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1328() {
-    if (jj_scan_token(GROUPS)) return true;
+  final private boolean jj_3_1329() {
+    if (jj_scan_token(HOLD)) return true;
     return false;
   }
 
@@ -20914,23 +20923,23 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1327() {
-    if (jj_scan_token(GLOBAL)) return true;
+  final private boolean jj_3_1328() {
+    if (jj_scan_token(GO)) return true;
     return false;
   }
 
-  final private boolean jj_3_1326() {
-    if (jj_scan_token(G)) return true;
+  final private boolean jj_3_1327() {
+    if (jj_scan_token(GENERAL)) return true;
     return false;
   }
 
-  final private boolean jj_3_1325() {
-    if (jj_scan_token(FREE)) return true;
+  final private boolean jj_3_1326() {
+    if (jj_scan_token(FUNCTION)) return true;
     return false;
   }
 
-  final private boolean jj_3_1324() {
-    if (jj_scan_token(FOREIGN)) return true;
+  final private boolean jj_3_1325() {
+    if (jj_scan_token(FOUND)) return true;
     return false;
   }
 
@@ -20939,8 +20948,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1323() {
-    if (jj_scan_token(FLOAT)) return true;
+  final private boolean jj_3_1324() {
+    if (jj_scan_token(FLOOR)) return true;
     return false;
   }
 
@@ -20950,48 +20959,48 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1323() {
+    if (jj_scan_token(FIRST)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1322() {
-    if (jj_scan_token(FILTER)) return true;
+    if (jj_scan_token(EXTRACT)) return true;
     return false;
   }
 
   final private boolean jj_3_1321() {
-    if (jj_scan_token(EXTERNAL)) return true;
+    if (jj_scan_token(EXP)) return true;
     return false;
   }
 
   final private boolean jj_3_1320() {
-    if (jj_scan_token(EXISTS)) return true;
+    if (jj_scan_token(EXEC)) return true;
     return false;
   }
 
   final private boolean jj_3_1319() {
-    if (jj_scan_token(EXCEPTION)) return true;
+    if (jj_scan_token(ESCAPE)) return true;
     return false;
   }
 
   final private boolean jj_3_1318() {
-    if (jj_scan_token(EQUALS)) return true;
+    if (jj_scan_token(END_FRAME)) return true;
     return false;
   }
 
   final private boolean jj_3_1317() {
-    if (jj_scan_token(END)) return true;
+    if (jj_scan_token(ELSE)) return true;
     return false;
   }
 
   final private boolean jj_3_1316() {
-    if (jj_scan_token(ELEMENT)) return true;
+    if (jj_scan_token(DYNAMIC)) return true;
     return false;
   }
 
   final private boolean jj_3_1315() {
-    if (jj_scan_token(DOUBLE)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1314() {
-    if (jj_scan_token(DISALLOW)) return true;
+    if (jj_scan_token(DISCONNECT)) return true;
     return false;
   }
 
@@ -21001,33 +21010,33 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1314() {
+    if (jj_scan_token(DETERMINISTIC)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1313() {
-    if (jj_scan_token(DESCRIPTOR)) return true;
+    if (jj_scan_token(DEREF)) return true;
     return false;
   }
 
   final private boolean jj_3_1312() {
-    if (jj_scan_token(DEPTH)) return true;
+    if (jj_scan_token(DEFERRED)) return true;
     return false;
   }
 
   final private boolean jj_3_1311() {
-    if (jj_scan_token(DEFERRABLE)) return true;
+    if (jj_scan_token(DECIMAL)) return true;
     return false;
   }
 
   final private boolean jj_3_1310() {
-    if (jj_scan_token(DEC)) return true;
+    if (jj_scan_token(DAY)) return true;
     return false;
   }
 
   final private boolean jj_3_1309() {
-    if (jj_scan_token(DATE)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1308() {
-    if (jj_scan_token(CURRENT_USER)) return true;
+    if (jj_scan_token(CYCLE)) return true;
     return false;
   }
 
@@ -21036,13 +21045,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1307() {
-    if (jj_scan_token(CURRENT_TIME)) return true;
+  final private boolean jj_3_1308() {
+    if (jj_scan_token(CURRENT_TIMESTAMP)) return true;
     return false;
   }
 
-  final private boolean jj_3_1306() {
-    if (jj_scan_token(CURRENT_ROLE)) return true;
+  final private boolean jj_3_1307() {
+    if (jj_scan_token(CURRENT_ROW)) return true;
     return false;
   }
 
@@ -21051,8 +21060,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1305() {
-    if (jj_scan_token(CURRENT_DATE)) return true;
+  final private boolean jj_3_1306() {
+    if (jj_scan_token(CURRENT_DEFAULT_TRANSFORM_GROUP)) return true;
     return false;
   }
 
@@ -21064,13 +21073,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1304() {
-    if (jj_scan_token(CUBE)) return true;
+  final private boolean jj_3_1305() {
+    if (jj_scan_token(CUME_DIST)) return true;
     return false;
   }
 
-  final private boolean jj_3_1303() {
-    if (jj_scan_token(COUNT)) return true;
+  final private boolean jj_3_1304() {
+    if (jj_scan_token(COVAR_POP)) return true;
     return false;
   }
 
@@ -21079,28 +21088,28 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1303() {
+    if (jj_scan_token(CORR)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1302() {
-    if (jj_scan_token(CONVERT)) return true;
+    if (jj_scan_token(CONTAINS)) return true;
     return false;
   }
 
   final private boolean jj_3_1301() {
-    if (jj_scan_token(CONSTRUCTOR)) return true;
+    if (jj_scan_token(CONNECTION)) return true;
     return false;
   }
 
   final private boolean jj_3_1300() {
-    if (jj_scan_token(CONNECT)) return true;
+    if (jj_scan_token(COMMIT)) return true;
     return false;
   }
 
   final private boolean jj_3_1299() {
-    if (jj_scan_token(COLUMN)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1298() {
-    if (jj_scan_token(COLLATE)) return true;
+    if (jj_scan_token(COLLATION)) return true;
     return false;
   }
 
@@ -21110,28 +21119,28 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1298() {
+    if (jj_scan_token(CLOSE)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1297() {
-    if (jj_scan_token(CLOB)) return true;
+    if (jj_scan_token(CHECK)) return true;
     return false;
   }
 
   final private boolean jj_3_1296() {
-    if (jj_scan_token(CHAR_LENGTH)) return true;
+    if (jj_scan_token(CHARACTER)) return true;
     return false;
   }
 
   final private boolean jj_3_1295() {
-    if (jj_scan_token(CHAR)) return true;
+    if (jj_scan_token(CEIL)) return true;
     return false;
   }
 
   final private boolean jj_3_1294() {
-    if (jj_scan_token(CATALOG)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1293() {
-    if (jj_scan_token(CASCADE)) return true;
+    if (jj_scan_token(CASCADED)) return true;
     return false;
   }
 
@@ -21140,13 +21149,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1292() {
-    if (jj_scan_token(C)) return true;
+  final private boolean jj_3_1293() {
+    if (jj_scan_token(CALLED)) return true;
     return false;
   }
 
-  final private boolean jj_3_1291() {
-    if (jj_scan_token(BOTH)) return true;
+  final private boolean jj_3_1292() {
+    if (jj_scan_token(BREADTH)) return true;
     return false;
   }
 
@@ -21158,38 +21167,38 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1291() {
+    if (jj_scan_token(BLOB)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1290() {
-    if (jj_scan_token(BIT)) return true;
+    if (jj_scan_token(BIGINT)) return true;
     return false;
   }
 
   final private boolean jj_3_1289() {
-    if (jj_scan_token(BETWEEN)) return true;
+    if (jj_scan_token(BEGIN_FRAME)) return true;
     return false;
   }
 
   final private boolean jj_3_1288() {
-    if (jj_scan_token(BEGIN)) return true;
+    if (jj_scan_token(AVG)) return true;
     return false;
   }
 
   final private boolean jj_3_1287() {
-    if (jj_scan_token(AUTHORIZATION)) return true;
+    if (jj_scan_token(AT)) return true;
     return false;
   }
 
   final private boolean jj_3_1286() {
-    if (jj_scan_token(ASYMMETRIC)) return true;
+    if (jj_scan_token(ASENSITIVE)) return true;
     return false;
   }
 
   final private boolean jj_3_1285() {
-    if (jj_scan_token(ASC)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1284() {
-    if (jj_scan_token(ARRAY)) return true;
+    if (jj_scan_token(ARRAY_MAX_CARDINALITY)) return true;
     return false;
   }
 
@@ -21212,8 +21221,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1283() {
-    if (jj_scan_token(ALTER)) return true;
+  final private boolean jj_3_1284() {
+    if (jj_scan_token(AND)) return true;
     return false;
   }
 
@@ -21222,18 +21231,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1282() {
-    if (jj_scan_token(AFTER)) return true;
+  final private boolean jj_3_1283() {
+    if (jj_scan_token(ALLOCATE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1281() {
-    if (jj_scan_token(ABSOLUTE)) return true;
+  final private boolean jj_3_1282() {
+    if (jj_scan_token(ACTION)) return true;
     return false;
   }
 
-  final private boolean jj_3_1280() {
-    if (jj_scan_token(UUID)) return true;
+  final private boolean jj_3_1281() {
+    if (jj_scan_token(A)) return true;
     return false;
   }
 
@@ -21256,37 +21265,37 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1279() {
+  final private boolean jj_3_1280() {
     if (jj_scan_token(COMPUTE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1278() {
+  final private boolean jj_3_1279() {
     if (jj_scan_token(SCAN)) return true;
     return false;
   }
 
-  final private boolean jj_3_1277() {
+  final private boolean jj_3_1278() {
     if (jj_scan_token(NOLOGGING)) return true;
     return false;
   }
 
-  final private boolean jj_3_1276() {
+  final private boolean jj_3_1277() {
     if (jj_scan_token(PARALLEL)) return true;
     return false;
   }
 
-  final private boolean jj_3_1275() {
+  final private boolean jj_3_1276() {
     if (jj_scan_token(DATA_REGION)) return true;
     return false;
   }
 
-  final private boolean jj_3_1274() {
+  final private boolean jj_3_1275() {
     if (jj_scan_token(WRITE_SYNCHRONIZATION_MODE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1273() {
+  final private boolean jj_3_1274() {
     if (jj_scan_token(BACKUPS)) return true;
     return false;
   }
@@ -21300,8 +21309,6 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
   final private boolean jj_3R_268() {
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3_1273()) {
-    jj_scanpos = xsp;
     if (jj_3_1274()) {
     jj_scanpos = xsp;
     if (jj_3_1275()) {
@@ -21560,7 +21567,9 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     jj_scanpos = xsp;
     if (jj_3_1402()) {
     jj_scanpos = xsp;
-    if (jj_3_1403()) return true;
+    if (jj_3_1403()) {
+    jj_scanpos = xsp;
+    if (jj_3_1404()) return true;
     }
     }
     }
@@ -21704,123 +21713,123 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1273() {
+    if (jj_scan_token(WRITE)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1272() {
-    if (jj_scan_token(YEAR)) return true;
+    if (jj_scan_token(WITHIN)) return true;
     return false;
   }
 
   final private boolean jj_3_1271() {
-    if (jj_scan_token(WITHOUT)) return true;
+    if (jj_scan_token(WEEK)) return true;
     return false;
   }
 
   final private boolean jj_3_1270() {
-    if (jj_scan_token(WHENEVER)) return true;
+    if (jj_scan_token(VERSION)) return true;
     return false;
   }
 
   final private boolean jj_3_1269() {
-    if (jj_scan_token(VERSIONING)) return true;
+    if (jj_scan_token(VARYING)) return true;
     return false;
   }
 
   final private boolean jj_3_1268() {
-    if (jj_scan_token(VAR_POP)) return true;
+    if (jj_scan_token(VALUE_OF)) return true;
     return false;
   }
 
   final private boolean jj_3_1267() {
-    if (jj_scan_token(VARBINARY)) return true;
+    if (jj_scan_token(USAGE)) return true;
     return false;
   }
 
   final private boolean jj_3_1266() {
-    if (jj_scan_token(USER)) return true;
+    if (jj_scan_token(UNKNOWN)) return true;
     return false;
   }
 
   final private boolean jj_3_1265() {
-    if (jj_scan_token(UPPER)) return true;
+    if (jj_scan_token(UESCAPE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1264() {
-    if (jj_scan_token(UNDER)) return true;
+  final private boolean jj_3R_126() {
+    if (jj_3R_285()) return true;
     return false;
   }
 
-  final private boolean jj_3R_126() {
-    if (jj_3R_285()) return true;
+  final private boolean jj_3_1264() {
+    if (jj_scan_token(TRIM_ARRAY)) return true;
     return false;
   }
 
   final private boolean jj_3_1263() {
-    if (jj_scan_token(TRUE)) return true;
+    if (jj_scan_token(TREAT)) return true;
     return false;
   }
 
   final private boolean jj_3_1262() {
-    if (jj_scan_token(TRIGGER)) return true;
+    if (jj_scan_token(TRANSLATE)) return true;
     return false;
   }
 
   final private boolean jj_3_1261() {
-    if (jj_scan_token(TRANSLATE_REGEX)) return true;
+    if (jj_scan_token(TO)) return true;
     return false;
   }
 
   final private boolean jj_3_1260() {
-    if (jj_scan_token(TRAILING)) return true;
+    if (jj_scan_token(TIMEZONE_HOUR)) return true;
     return false;
   }
 
   final private boolean jj_3_1259() {
-    if (jj_scan_token(TIMEZONE_MINUTE)) return true;
+    if (jj_scan_token(STRING_AGG)) return true;
     return false;
   }
 
   final private boolean jj_3_1258() {
-    if (jj_scan_token(SYSTEM_USER)) return true;
+    if (jj_scan_token(SYSTEM)) return true;
     return false;
   }
 
   final private boolean jj_3_1257() {
-    if (jj_scan_token(SYMMETRIC)) return true;
+    if (jj_scan_token(SUCCEEDS)) return true;
     return false;
   }
 
   final private boolean jj_3_1256() {
-    if (jj_scan_token(SUBSTRING_REGEX)) return true;
+    if (jj_scan_token(SUBSET)) return true;
     return false;
   }
 
   final private boolean jj_3_1255() {
-    if (jj_scan_token(SUBMULTISET)) return true;
+    if (jj_scan_token(STDDEV_POP)) return true;
     return false;
   }
 
   final private boolean jj_3_1254() {
-    if (jj_scan_token(STATIC)) return true;
+    if (jj_scan_token(START)) return true;
     return false;
   }
 
   final private boolean jj_3_1253() {
-    if (jj_scan_token(SQRT)) return true;
+    if (jj_scan_token(SQLSTATE)) return true;
     return false;
   }
 
   final private boolean jj_3_1252() {
-    if (jj_scan_token(SQLEXCEPTION)) return true;
+    if (jj_scan_token(SPECIFICTYPE)) return true;
     return false;
   }
 
   final private boolean jj_3_1251() {
-    if (jj_scan_token(SPECIFIC)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1250() {
-    if (jj_scan_token(SIZE)) return true;
+    if (jj_scan_token(SMALLINT)) return true;
     return false;
   }
 
@@ -21830,28 +21839,28 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1250() {
+    if (jj_scan_token(SHOW)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1249() {
-    if (jj_scan_token(SESSION_USER)) return true;
+    if (jj_scan_token(SENSITIVE)) return true;
     return false;
   }
 
   final private boolean jj_3_1248() {
-    if (jj_scan_token(SEEK)) return true;
+    if (jj_scan_token(SECOND)) return true;
     return false;
   }
 
   final private boolean jj_3_1247() {
-    if (jj_scan_token(SEARCH)) return true;
+    if (jj_scan_token(SCOPE)) return true;
     return false;
   }
 
   final private boolean jj_3_1246() {
-    if (jj_scan_token(SCHEMA)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1245() {
-    if (jj_scan_token(ROW_NUMBER)) return true;
+    if (jj_scan_token(RUNNING)) return true;
     return false;
   }
 
@@ -21861,18 +21870,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1244() {
-    if (jj_scan_token(ROLE)) return true;
+  final private boolean jj_3_1245() {
+    if (jj_scan_token(ROLLBACK)) return true;
     return false;
   }
 
-  final private boolean jj_3_1243() {
-    if (jj_scan_token(RETURN)) return true;
+  final private boolean jj_3_1244() {
+    if (jj_scan_token(RETURNS)) return true;
     return false;
   }
 
-  final private boolean jj_3_1242() {
-    if (jj_scan_token(RESET)) return true;
+  final private boolean jj_3_1243() {
+    if (jj_scan_token(RESTRICT)) return true;
     return false;
   }
 
@@ -21882,13 +21891,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1242() {
+    if (jj_scan_token(RELEASE)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1241() {
-    if (jj_scan_token(RELATIVE)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1240() {
-    if (jj_scan_token(REGR_SXX)) return true;
+    if (jj_scan_token(REGR_SXY)) return true;
     return false;
   }
 
@@ -21904,63 +21913,63 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1240() {
+    if (jj_scan_token(REGR_R2)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1239() {
-    if (jj_scan_token(REGR_INTERCEPT)) return true;
+    if (jj_scan_token(REGR_AVGY)) return true;
     return false;
   }
 
   final private boolean jj_3_1238() {
-    if (jj_scan_token(REGR_AVGX)) return true;
+    if (jj_scan_token(REFERENCES)) return true;
     return false;
   }
 
   final private boolean jj_3_1237() {
-    if (jj_scan_token(REF)) return true;
+    if (jj_scan_token(REAL)) return true;
     return false;
   }
 
   final private boolean jj_3_1236() {
-    if (jj_scan_token(READS)) return true;
+    if (jj_scan_token(RANK)) return true;
     return false;
   }
 
   final private boolean jj_3_1235() {
-    if (jj_scan_token(QUARTER)) return true;
+    if (jj_scan_token(PROCEDURE)) return true;
     return false;
   }
 
   final private boolean jj_3_1234() {
-    if (jj_scan_token(PRIVILEGES)) return true;
+    if (jj_scan_token(PREV)) return true;
     return false;
   }
 
   final private boolean jj_3_1233() {
-    if (jj_scan_token(PRESERVE)) return true;
+    if (jj_scan_token(PRECISION)) return true;
     return false;
   }
 
   final private boolean jj_3_1232() {
-    if (jj_scan_token(PRECEDES)) return true;
+    if (jj_scan_token(POSITION_REGEX)) return true;
     return false;
   }
 
   final private boolean jj_3_1231() {
-    if (jj_scan_token(POSITION)) return true;
+    if (jj_scan_token(PERMUTE)) return true;
     return false;
   }
 
   final private boolean jj_3_1230() {
-    if (jj_scan_token(PERIOD)) return true;
+    if (jj_scan_token(PERCENTILE_DISC)) return true;
     return false;
   }
 
   final private boolean jj_3_1229() {
-    if (jj_scan_token(PERCENTILE_CONT)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1228() {
-    if (jj_scan_token(PATH)) return true;
+    if (jj_scan_token(PER)) return true;
     return false;
   }
 
@@ -21970,23 +21979,23 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1227() {
-    if (jj_scan_token(PAD)) return true;
+  final private boolean jj_3_1228() {
+    if (jj_scan_token(PARAMETER)) return true;
     return false;
   }
 
-  final private boolean jj_3_1226() {
-    if (jj_scan_token(OUTPUT)) return true;
+  final private boolean jj_3_1227() {
+    if (jj_scan_token(OVERLAPS)) return true;
     return false;
   }
 
-  final private boolean jj_3_1225() {
-    if (jj_scan_token(OR)) return true;
+  final private boolean jj_3_1226() {
+    if (jj_scan_token(ORDINALITY)) return true;
     return false;
   }
 
-  final private boolean jj_3_1224() {
-    if (jj_scan_token(ONLY)) return true;
+  final private boolean jj_3_1225() {
+    if (jj_scan_token(OPEN)) return true;
     return false;
   }
 
@@ -21995,18 +22004,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1223() {
-    if (jj_scan_token(OLD)) return true;
+  final private boolean jj_3_1224() {
+    if (jj_scan_token(OMIT)) return true;
     return false;
   }
 
-  final private boolean jj_3_1222() {
-    if (jj_scan_token(OCCURRENCES_REGEX)) return true;
+  final private boolean jj_3_1223() {
+    if (jj_scan_token(OCTET_LENGTH)) return true;
     return false;
   }
 
-  final private boolean jj_3_1221() {
-    if (jj_scan_token(NULLIF)) return true;
+  final private boolean jj_3_1222() {
+    if (jj_scan_token(NUMERIC)) return true;
     return false;
   }
 
@@ -22020,8 +22029,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1220() {
-    if (jj_scan_token(NOT)) return true;
+  final private boolean jj_3_1221() {
+    if (jj_scan_token(NTH_VALUE)) return true;
     return false;
   }
 
@@ -22030,18 +22039,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1219() {
-    if (jj_scan_token(NO)) return true;
+  final private boolean jj_3_1220() {
+    if (jj_scan_token(NONE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1218() {
-    if (jj_scan_token(NATIONAL)) return true;
+  final private boolean jj_3_1219() {
+    if (jj_scan_token(NCHAR)) return true;
     return false;
   }
 
-  final private boolean jj_3_1217() {
-    if (jj_scan_token(MULTISET)) return true;
+  final private boolean jj_3_1218() {
+    if (jj_scan_token(NAME)) return true;
     return false;
   }
 
@@ -22051,83 +22060,83 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1217() {
+    if (jj_scan_token(MODULE)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1216() {
-    if (jj_scan_token(MODIFIES)) return true;
+    if (jj_scan_token(MINUTE)) return true;
     return false;
   }
 
   final private boolean jj_3_1215() {
-    if (jj_scan_token(MIN)) return true;
+    if (jj_scan_token(MEMBER)) return true;
     return false;
   }
 
   final private boolean jj_3_1214() {
-    if (jj_scan_token(MEASURES)) return true;
+    if (jj_scan_token(MATCH_NUMBER)) return true;
     return false;
   }
 
   final private boolean jj_3_1213() {
-    if (jj_scan_token(MATCHES)) return true;
+    if (jj_scan_token(MAP)) return true;
     return false;
   }
 
   final private boolean jj_3_1212() {
-    if (jj_scan_token(M)) return true;
+    if (jj_scan_token(LOCATOR)) return true;
     return false;
   }
 
   final private boolean jj_3_1211() {
-    if (jj_scan_token(LOCALTIMESTAMP)) return true;
+    if (jj_scan_token(LOCAL)) return true;
     return false;
   }
 
   final private boolean jj_3_1210() {
-    if (jj_scan_token(LN)) return true;
+    if (jj_scan_token(LIKE)) return true;
     return false;
   }
 
   final private boolean jj_3_1209() {
-    if (jj_scan_token(LEVEL)) return true;
+    if (jj_scan_token(LEADING)) return true;
     return false;
   }
 
   final private boolean jj_3_1208() {
-    if (jj_scan_token(LEAD)) return true;
+    if (jj_scan_token(LAST)) return true;
     return false;
   }
 
   final private boolean jj_3_1207() {
-    if (jj_scan_token(LARGE)) return true;
+    if (jj_scan_token(LAG)) return true;
     return false;
   }
 
   final private boolean jj_3_1206() {
-    if (jj_scan_token(KEY)) return true;
+    if (jj_scan_token(JSON_VALUE)) return true;
     return false;
   }
 
   final private boolean jj_3_1205() {
-    if (jj_scan_token(JSON_QUERY)) return true;
+    if (jj_scan_token(JSON_OBJECT)) return true;
     return false;
   }
 
   final private boolean jj_3_1204() {
-    if (jj_scan_token(JSON_EXISTS)) return true;
+    if (jj_scan_token(JSON_ARRAY)) return true;
     return false;
   }
 
   final private boolean jj_3_1203() {
-    if (jj_scan_token(ISOLATION)) return true;
+    if (jj_scan_token(INTERSECTION)) return true;
     return false;
   }
 
   final private boolean jj_3_1202() {
-    if (jj_scan_token(INTEGER)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1201() {
-    if (jj_scan_token(INPUT)) return true;
+    if (jj_scan_token(INSENSITIVE)) return true;
     return false;
   }
 
@@ -22138,48 +22147,48 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1201() {
+    if (jj_scan_token(INITIALLY)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1200() {
-    if (jj_scan_token(INITIAL)) return true;
+    if (jj_scan_token(IMPORT)) return true;
     return false;
   }
 
   final private boolean jj_3_1199() {
-    if (jj_scan_token(IMMEDIATELY)) return true;
+    if (jj_scan_token(IDENTITY)) return true;
     return false;
   }
 
   final private boolean jj_3_1198() {
-    if (jj_scan_token(HOUR)) return true;
+    if (jj_scan_token(GROUPS)) return true;
     return false;
   }
 
   final private boolean jj_3_1197() {
-    if (jj_scan_token(GOTO)) return true;
+    if (jj_scan_token(GLOBAL)) return true;
     return false;
   }
 
   final private boolean jj_3_1196() {
-    if (jj_scan_token(GET)) return true;
+    if (jj_scan_token(G)) return true;
     return false;
   }
 
   final private boolean jj_3_1195() {
-    if (jj_scan_token(FUSION)) return true;
+    if (jj_scan_token(FREE)) return true;
     return false;
   }
 
   final private boolean jj_3_1194() {
-    if (jj_scan_token(FRAME_ROW)) return true;
+    if (jj_scan_token(FOREIGN)) return true;
     return false;
   }
 
   final private boolean jj_3_1193() {
-    if (jj_scan_token(FOR)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1192() {
-    if (jj_scan_token(FIRST_VALUE)) return true;
+    if (jj_scan_token(FLOAT)) return true;
     return false;
   }
 
@@ -22189,23 +22198,23 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1191() {
-    if (jj_scan_token(FALSE)) return true;
+  final private boolean jj_3_1192() {
+    if (jj_scan_token(FILTER)) return true;
     return false;
   }
 
-  final private boolean jj_3_1190() {
-    if (jj_scan_token(EXTEND)) return true;
+  final private boolean jj_3_1191() {
+    if (jj_scan_token(EXTERNAL)) return true;
     return false;
   }
 
-  final private boolean jj_3_1189() {
-    if (jj_scan_token(EXECUTE)) return true;
+  final private boolean jj_3_1190() {
+    if (jj_scan_token(EXISTS)) return true;
     return false;
   }
 
-  final private boolean jj_3_1188() {
-    if (jj_scan_token(EVERY)) return true;
+  final private boolean jj_3_1189() {
+    if (jj_scan_token(EXCEPTION)) return true;
     return false;
   }
 
@@ -22215,28 +22224,28 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1188() {
+    if (jj_scan_token(EQUALS)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1187() {
-    if (jj_scan_token(END_PARTITION)) return true;
+    if (jj_scan_token(END)) return true;
     return false;
   }
 
   final private boolean jj_3_1186() {
-    if (jj_scan_token(EMPTY)) return true;
+    if (jj_scan_token(ELEMENT)) return true;
     return false;
   }
 
   final private boolean jj_3_1185() {
-    if (jj_scan_token(EACH)) return true;
+    if (jj_scan_token(DOUBLE)) return true;
     return false;
   }
 
   final private boolean jj_3_1184() {
-    if (jj_scan_token(DOMAIN)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1183() {
-    if (jj_scan_token(DIAGNOSTICS)) return true;
+    if (jj_scan_token(DISALLOW)) return true;
     return false;
   }
 
@@ -22250,23 +22259,23 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1182() {
-    if (jj_scan_token(DESC)) return true;
+  final private boolean jj_3_1183() {
+    if (jj_scan_token(DESCRIPTOR)) return true;
     return false;
   }
 
-  final private boolean jj_3_1181() {
-    if (jj_scan_token(DENSE_RANK)) return true;
+  final private boolean jj_3_1182() {
+    if (jj_scan_token(DEPTH)) return true;
     return false;
   }
 
-  final private boolean jj_3_1180() {
-    if (jj_scan_token(DECLARE)) return true;
+  final private boolean jj_3_1181() {
+    if (jj_scan_token(DEFERRABLE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1179() {
-    if (jj_scan_token(DEALLOCATE)) return true;
+  final private boolean jj_3_1180() {
+    if (jj_scan_token(DEC)) return true;
     return false;
   }
 
@@ -22276,48 +22285,48 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1179() {
+    if (jj_scan_token(DATE)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1178() {
-    if (jj_scan_token(DATA)) return true;
+    if (jj_scan_token(CURRENT_USER)) return true;
     return false;
   }
 
   final private boolean jj_3_1177() {
-    if (jj_scan_token(CURRENT_TRANSFORM_GROUP_FOR_TYPE)) return true;
+    if (jj_scan_token(CURRENT_TIME)) return true;
     return false;
   }
 
   final private boolean jj_3_1176() {
-    if (jj_scan_token(CURRENT_SCHEMA)) return true;
+    if (jj_scan_token(CURRENT_ROLE)) return true;
     return false;
   }
 
   final private boolean jj_3_1175() {
-    if (jj_scan_token(CURRENT_PATH)) return true;
+    if (jj_scan_token(CURRENT_DATE)) return true;
     return false;
   }
 
   final private boolean jj_3_1174() {
-    if (jj_scan_token(CURRENT_CATALOG)) return true;
+    if (jj_scan_token(CUBE)) return true;
     return false;
   }
 
   final private boolean jj_3_1173() {
-    if (jj_scan_token(COVAR_SAMP)) return true;
+    if (jj_scan_token(COUNT)) return true;
     return false;
   }
 
   final private boolean jj_3_1172() {
-    if (jj_scan_token(CORRESPONDING)) return true;
+    if (jj_scan_token(CONVERT)) return true;
     return false;
   }
 
   final private boolean jj_3_1171() {
-    if (jj_scan_token(CONTINUE)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1170() {
-    if (jj_scan_token(CONSTRAINTS)) return true;
+    if (jj_scan_token(CONSTRUCTOR)) return true;
     return false;
   }
 
@@ -22328,93 +22337,93 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1170() {
+    if (jj_scan_token(CONNECT)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1169() {
-    if (jj_scan_token(CONDITION)) return true;
+    if (jj_scan_token(COLUMN)) return true;
     return false;
   }
 
   final private boolean jj_3_1168() {
-    if (jj_scan_token(COLLECT)) return true;
+    if (jj_scan_token(COLLATE)) return true;
     return false;
   }
 
   final private boolean jj_3_1167() {
-    if (jj_scan_token(COALESCE)) return true;
+    if (jj_scan_token(CLOB)) return true;
     return false;
   }
 
-  final private boolean jj_3_1166() {
-    if (jj_scan_token(CLASSIFIER)) return true;
+  final private boolean jj_3R_283() {
+    if (jj_scan_token(QUOTED_STRING)) return true;
     return false;
   }
 
-  final private boolean jj_3R_283() {
-    if (jj_scan_token(QUOTED_STRING)) return true;
+  final private boolean jj_3_1166() {
+    if (jj_scan_token(CHAR_LENGTH)) return true;
     return false;
   }
 
   final private boolean jj_3_1165() {
-    if (jj_scan_token(CHARACTER_LENGTH)) return true;
+    if (jj_scan_token(CHAR)) return true;
     return false;
   }
 
   final private boolean jj_3_1164() {
-    if (jj_scan_token(CEILING)) return true;
+    if (jj_scan_token(CATALOG)) return true;
     return false;
   }
 
   final private boolean jj_3_1163() {
-    if (jj_scan_token(CAST)) return true;
+    if (jj_scan_token(CASCADE)) return true;
     return false;
   }
 
   final private boolean jj_3_1162() {
-    if (jj_scan_token(CARDINALITY)) return true;
+    if (jj_scan_token(C)) return true;
     return false;
   }
 
   final private boolean jj_3_1161() {
-    if (jj_scan_token(BY)) return true;
+    if (jj_scan_token(BOTH)) return true;
     return false;
   }
 
   final private boolean jj_3_1160() {
-    if (jj_scan_token(BOOLEAN)) return true;
+    if (jj_scan_token(BIT)) return true;
     return false;
   }
 
   final private boolean jj_3_1159() {
-    if (jj_scan_token(BINARY)) return true;
+    if (jj_scan_token(BETWEEN)) return true;
     return false;
   }
 
   final private boolean jj_3_1158() {
-    if (jj_scan_token(BEGIN_PARTITION)) return true;
+    if (jj_scan_token(BEGIN)) return true;
     return false;
   }
 
   final private boolean jj_3_1157() {
-    if (jj_scan_token(BEFORE)) return true;
+    if (jj_scan_token(AUTHORIZATION)) return true;
     return false;
   }
 
   final private boolean jj_3_1156() {
-    if (jj_scan_token(ATOMIC)) return true;
+    if (jj_scan_token(ASYMMETRIC)) return true;
     return false;
   }
 
   final private boolean jj_3_1155() {
-    if (jj_scan_token(ASSERTION)) return true;
+    if (jj_scan_token(ASC)) return true;
     return false;
   }
 
   final private boolean jj_3_1154() {
-    if (jj_scan_token(AS)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1153() {
-    if (jj_scan_token(ARE)) return true;
+    if (jj_scan_token(ARRAY)) return true;
     return false;
   }
 
@@ -22439,27 +22448,27 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1152() {
-    if (jj_scan_token(ALLOW)) return true;
+  final private boolean jj_3_1153() {
+    if (jj_scan_token(ALTER)) return true;
     return false;
   }
 
-  final private boolean jj_3_1151() {
-    if (jj_scan_token(ADD)) return true;
+  final private boolean jj_3_1152() {
+    if (jj_scan_token(AFTER)) return true;
     return false;
   }
 
-  final private boolean jj_3_1150() {
-    if (jj_scan_token(ABS)) return true;
+  final private boolean jj_3_1151() {
+    if (jj_scan_token(ABSOLUTE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1149() {
+  final private boolean jj_3_1150() {
     if (jj_scan_token(QUERY)) return true;
     return false;
   }
 
-  final private boolean jj_3_1148() {
+  final private boolean jj_3_1149() {
     if (jj_scan_token(SERVICE)) return true;
     return false;
   }
@@ -22469,32 +22478,32 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1147() {
+  final private boolean jj_3_1148() {
     if (jj_scan_token(KILL)) return true;
     return false;
   }
 
-  final private boolean jj_3_1146() {
+  final private boolean jj_3_1147() {
     if (jj_scan_token(LOGGING)) return true;
     return false;
   }
 
-  final private boolean jj_3_1145() {
+  final private boolean jj_3_1146() {
     if (jj_scan_token(ENCRYPTED)) return true;
     return false;
   }
 
-  final private boolean jj_3_1144() {
+  final private boolean jj_3_1145() {
     if (jj_scan_token(CACHE_NAME)) return true;
     return false;
   }
 
-  final private boolean jj_3_1143() {
+  final private boolean jj_3_1144() {
     if (jj_scan_token(ATOMICITY)) return true;
     return false;
   }
 
-  final private boolean jj_3_1142() {
+  final private boolean jj_3_1143() {
     if (jj_scan_token(TEMPLATE)) return true;
     return false;
   }
@@ -22507,8 +22516,6 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
   final private boolean jj_3R_267() {
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3_1142()) {
-    jj_scanpos = xsp;
     if (jj_3_1143()) {
     jj_scanpos = xsp;
     if (jj_3_1144()) {
@@ -22767,7 +22774,9 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     jj_scanpos = xsp;
     if (jj_3_1271()) {
     jj_scanpos = xsp;
-    if (jj_3_1272()) return true;
+    if (jj_3_1272()) {
+    jj_scanpos = xsp;
+    if (jj_3_1273()) return true;
     }
     }
     }
@@ -22901,143 +22910,143 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1142() {
+    if (jj_scan_token(ZONE)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1141() {
-    if (jj_scan_token(WRITE)) return true;
+    if (jj_scan_token(WORK)) return true;
     return false;
   }
 
   final private boolean jj_3_1140() {
-    if (jj_scan_token(WITHIN)) return true;
+    if (jj_scan_token(WIDTH_BUCKET)) return true;
     return false;
   }
 
   final private boolean jj_3_1139() {
-    if (jj_scan_token(WEEK)) return true;
+    if (jj_scan_token(VIEW)) return true;
     return false;
   }
 
   final private boolean jj_3_1138() {
-    if (jj_scan_token(VERSION)) return true;
+    if (jj_scan_token(VAR_SAMP)) return true;
     return false;
   }
 
   final private boolean jj_3_1137() {
-    if (jj_scan_token(VARYING)) return true;
+    if (jj_scan_token(VARCHAR)) return true;
     return false;
   }
 
   final private boolean jj_3_1136() {
-    if (jj_scan_token(VALUE_OF)) return true;
+    if (jj_scan_token(VALUE)) return true;
     return false;
   }
 
   final private boolean jj_3_1135() {
-    if (jj_scan_token(USAGE)) return true;
+    if (jj_scan_token(UPSERT)) return true;
     return false;
   }
 
   final private boolean jj_3_1134() {
-    if (jj_scan_token(UNKNOWN)) return true;
+    if (jj_scan_token(UNIQUE)) return true;
     return false;
   }
 
   final private boolean jj_3_1133() {
-    if (jj_scan_token(UESCAPE)) return true;
+    if (jj_scan_token(TRUNCATE)) return true;
     return false;
   }
 
   final private boolean jj_3_1132() {
-    if (jj_scan_token(TRIM_ARRAY)) return true;
+    if (jj_scan_token(TRIM)) return true;
     return false;
   }
 
   final private boolean jj_3_1131() {
-    if (jj_scan_token(TREAT)) return true;
+    if (jj_scan_token(TRANSLATION)) return true;
     return false;
   }
 
   final private boolean jj_3_1130() {
-    if (jj_scan_token(TRANSLATE)) return true;
+    if (jj_scan_token(TRANSACTION)) return true;
     return false;
   }
 
   final private boolean jj_3_1129() {
-    if (jj_scan_token(TO)) return true;
+    if (jj_scan_token(TINYINT)) return true;
     return false;
   }
 
   final private boolean jj_3_1128() {
-    if (jj_scan_token(TIMEZONE_HOUR)) return true;
+    if (jj_scan_token(TEMPORARY)) return true;
     return false;
   }
 
   final private boolean jj_3_1127() {
-    if (jj_scan_token(SYSTEM_TIME)) return true;
+    if (jj_scan_token(SYSTEM_USER)) return true;
     return false;
   }
 
   final private boolean jj_3_1126() {
-    if (jj_scan_token(SUM)) return true;
+    if (jj_scan_token(SYMMETRIC)) return true;
     return false;
   }
 
   final private boolean jj_3_1125() {
-    if (jj_scan_token(SUBSTRING)) return true;
+    if (jj_scan_token(SUBSTRING_REGEX)) return true;
     return false;
   }
 
   final private boolean jj_3_1124() {
-    if (jj_scan_token(STDDEV_SAMP)) return true;
+    if (jj_scan_token(SUBMULTISET)) return true;
     return false;
   }
 
   final private boolean jj_3_1123() {
-    if (jj_scan_token(STATE)) return true;
+    if (jj_scan_token(STATIC)) return true;
     return false;
   }
 
   final private boolean jj_3_1122() {
-    if (jj_scan_token(SQLWARNING)) return true;
+    if (jj_scan_token(SQRT)) return true;
     return false;
   }
 
-  final private boolean jj_3_1121() {
-    if (jj_scan_token(SQL)) return true;
+  final private boolean jj_3R_214() {
+    if (jj_3R_213()) return true;
     return false;
   }
 
-  final private boolean jj_3R_214() {
-    if (jj_3R_213()) return true;
+  final private boolean jj_3_1121() {
+    if (jj_scan_token(SQLEXCEPTION)) return true;
     return false;
   }
 
   final private boolean jj_3_1120() {
-    if (jj_scan_token(SPACE)) return true;
+    if (jj_scan_token(SPECIFIC)) return true;
     return false;
   }
 
   final private boolean jj_3_1119() {
-    if (jj_scan_token(SIMILAR)) return true;
+    if (jj_scan_token(SIZE)) return true;
     return false;
   }
 
   final private boolean jj_3_1118() {
-    if (jj_scan_token(SESSION)) return true;
+    if (jj_scan_token(SESSION_USER)) return true;
     return false;
   }
 
   final private boolean jj_3_1117() {
-    if (jj_scan_token(SECTION)) return true;
+    if (jj_scan_token(SEEK)) return true;
     return false;
   }
 
   final private boolean jj_3_1116() {
-    if (jj_scan_token(SCROLL)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1115() {
-    if (jj_scan_token(SAVEPOINT)) return true;
+    if (jj_scan_token(SEARCH)) return true;
     return false;
   }
 
@@ -23047,58 +23056,58 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1115() {
+    if (jj_scan_token(SCHEMA)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1114() {
-    if (jj_scan_token(ROUTINE)) return true;
+    if (jj_scan_token(ROW_NUMBER)) return true;
     return false;
   }
 
   final private boolean jj_3_1113() {
-    if (jj_scan_token(REVOKE)) return true;
+    if (jj_scan_token(ROLE)) return true;
     return false;
   }
 
   final private boolean jj_3_1112() {
-    if (jj_scan_token(RESULT)) return true;
+    if (jj_scan_token(RETURN)) return true;
     return false;
   }
 
   final private boolean jj_3_1111() {
-    if (jj_scan_token(REPLACE)) return true;
+    if (jj_scan_token(RESET)) return true;
     return false;
   }
 
   final private boolean jj_3_1110() {
-    if (jj_scan_token(REGR_SYY)) return true;
+    if (jj_scan_token(RELATIVE)) return true;
     return false;
   }
 
   final private boolean jj_3_1109() {
-    if (jj_scan_token(REGR_SLOPE)) return true;
+    if (jj_scan_token(REGR_SXX)) return true;
     return false;
   }
 
   final private boolean jj_3_1108() {
-    if (jj_scan_token(REGR_COUNT)) return true;
+    if (jj_scan_token(REGR_INTERCEPT)) return true;
     return false;
   }
 
   final private boolean jj_3_1107() {
-    if (jj_scan_token(REFERENCING)) return true;
+    if (jj_scan_token(REGR_AVGX)) return true;
     return false;
   }
 
   final private boolean jj_3_1106() {
-    if (jj_scan_token(RECURSIVE)) return true;
+    if (jj_scan_token(REF)) return true;
     return false;
   }
 
   final private boolean jj_3_1105() {
-    if (jj_scan_token(READ)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1104() {
-    if (jj_scan_token(PUBLIC)) return true;
+    if (jj_scan_token(READS)) return true;
     return false;
   }
 
@@ -23108,68 +23117,68 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1104() {
+    if (jj_scan_token(QUARTER)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1103() {
-    if (jj_scan_token(PRIOR)) return true;
+    if (jj_scan_token(PRIVILEGES)) return true;
     return false;
   }
 
   final private boolean jj_3_1102() {
-    if (jj_scan_token(PREPARE)) return true;
+    if (jj_scan_token(PRESERVE)) return true;
     return false;
   }
 
   final private boolean jj_3_1101() {
-    if (jj_scan_token(POWER)) return true;
+    if (jj_scan_token(PRECEDES)) return true;
     return false;
   }
 
   final private boolean jj_3_1100() {
-    if (jj_scan_token(PORTION)) return true;
+    if (jj_scan_token(POSITION)) return true;
     return false;
   }
 
-  final private boolean jj_3_1099() {
-    if (jj_scan_token(PERCENT_RANK)) return true;
+  final private boolean jj_3_575() {
+    if (jj_3R_212()) return true;
     return false;
   }
 
-  final private boolean jj_3_575() {
-    if (jj_3R_212()) return true;
+  final private boolean jj_3_1099() {
+    if (jj_scan_token(PERIOD)) return true;
     return false;
   }
 
   final private boolean jj_3_1098() {
-    if (jj_scan_token(PERCENT)) return true;
+    if (jj_scan_token(PERCENTILE_CONT)) return true;
     return false;
   }
 
   final private boolean jj_3_1097() {
-    if (jj_scan_token(PARTIAL)) return true;
+    if (jj_scan_token(PATH)) return true;
     return false;
   }
 
   final private boolean jj_3_1096() {
-    if (jj_scan_token(OVERLAY)) return true;
+    if (jj_scan_token(PAD)) return true;
     return false;
   }
 
   final private boolean jj_3_1095() {
-    if (jj_scan_token(OUT)) return true;
+    if (jj_scan_token(OUTPUT)) return true;
     return false;
   }
 
   final private boolean jj_3_1094() {
-    if (jj_scan_token(OPTION)) return true;
+    if (jj_scan_token(OR)) return true;
     return false;
   }
 
   final private boolean jj_3_1093() {
-    if (jj_scan_token(ONE)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1092() {
-    if (jj_scan_token(OF)) return true;
+    if (jj_scan_token(ONLY)) return true;
     return false;
   }
 
@@ -23178,8 +23187,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1091() {
-    if (jj_scan_token(OBJECT)) return true;
+  final private boolean jj_3_1092() {
+    if (jj_scan_token(OLD)) return true;
     return false;
   }
 
@@ -23194,18 +23203,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1090() {
-    if (jj_scan_token(NTILE)) return true;
+  final private boolean jj_3_1091() {
+    if (jj_scan_token(OCCURRENCES_REGEX)) return true;
     return false;
   }
 
-  final private boolean jj_3_1089() {
-    if (jj_scan_token(NORMALIZE)) return true;
+  final private boolean jj_3_1090() {
+    if (jj_scan_token(NULLIF)) return true;
     return false;
   }
 
-  final private boolean jj_3_1088() {
-    if (jj_scan_token(NCLOB)) return true;
+  final private boolean jj_3_1089() {
+    if (jj_scan_token(NOT)) return true;
     return false;
   }
 
@@ -23214,18 +23223,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1087() {
-    if (jj_scan_token(NAMES)) return true;
+  final private boolean jj_3_1088() {
+    if (jj_scan_token(NO)) return true;
     return false;
   }
 
-  final private boolean jj_3_1086() {
-    if (jj_scan_token(MONTH)) return true;
+  final private boolean jj_3_1087() {
+    if (jj_scan_token(NATIONAL)) return true;
     return false;
   }
 
-  final private boolean jj_3_1085() {
-    if (jj_scan_token(MOD)) return true;
+  final private boolean jj_3_1086() {
+    if (jj_scan_token(MULTISET)) return true;
     return false;
   }
 
@@ -23235,13 +23244,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1084() {
-    if (jj_scan_token(METHOD)) return true;
+  final private boolean jj_3_1085() {
+    if (jj_scan_token(MODIFIES)) return true;
     return false;
   }
 
-  final private boolean jj_3_1083() {
-    if (jj_scan_token(MAX)) return true;
+  final private boolean jj_3_1084() {
+    if (jj_scan_token(MIN)) return true;
     return false;
   }
 
@@ -23254,13 +23263,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1082() {
-    if (jj_scan_token(MATCH)) return true;
+  final private boolean jj_3_1083() {
+    if (jj_scan_token(MEASURES)) return true;
     return false;
   }
 
-  final private boolean jj_3_1081() {
-    if (jj_scan_token(LOWER)) return true;
+  final private boolean jj_3_1082() {
+    if (jj_scan_token(MATCHES)) return true;
     return false;
   }
 
@@ -23273,8 +23282,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1080() {
-    if (jj_scan_token(LOCALTIME)) return true;
+  final private boolean jj_3_1081() {
+    if (jj_scan_token(M)) return true;
     return false;
   }
 
@@ -23286,8 +23295,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1079() {
-    if (jj_scan_token(LIKE_REGEX)) return true;
+  final private boolean jj_3_1080() {
+    if (jj_scan_token(LOCALTIMESTAMP)) return true;
     return false;
   }
 
@@ -23300,13 +23309,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1078() {
-    if (jj_scan_token(LENGTH)) return true;
+  final private boolean jj_3_1079() {
+    if (jj_scan_token(LN)) return true;
     return false;
   }
 
-  final private boolean jj_3_1077() {
-    if (jj_scan_token(LAST_VALUE)) return true;
+  final private boolean jj_3_1078() {
+    if (jj_scan_token(LEVEL)) return true;
     return false;
   }
 
@@ -23316,43 +23325,43 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1077() {
+    if (jj_scan_token(LEAD)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1076() {
-    if (jj_scan_token(LANGUAGE)) return true;
+    if (jj_scan_token(LARGE)) return true;
     return false;
   }
 
   final private boolean jj_3_1075() {
-    if (jj_scan_token(K)) return true;
+    if (jj_scan_token(KEY)) return true;
     return false;
   }
 
-  final private boolean jj_3_1074() {
-    if (jj_scan_token(JSON_OBJECTAGG)) return true;
+  final private boolean jj_3_573() {
+    if (jj_scan_token(BRACKET_QUOTED_IDENTIFIER)) return true;
     return false;
   }
 
-  final private boolean jj_3_573() {
-    if (jj_scan_token(BRACKET_QUOTED_IDENTIFIER)) return true;
+  final private boolean jj_3_1074() {
+    if (jj_scan_token(JSON_QUERY)) return true;
     return false;
   }
 
   final private boolean jj_3_1073() {
-    if (jj_scan_token(JSON_ARRAYAGG)) return true;
+    if (jj_scan_token(JSON_EXISTS)) return true;
     return false;
   }
 
   final private boolean jj_3_1072() {
-    if (jj_scan_token(IS)) return true;
+    if (jj_scan_token(ISOLATION)) return true;
     return false;
   }
 
   final private boolean jj_3_1071() {
-    if (jj_scan_token(INT)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1070() {
-    if (jj_scan_token(INOUT)) return true;
+    if (jj_scan_token(INTEGER)) return true;
     return false;
   }
 
@@ -23361,8 +23370,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1069() {
-    if (jj_scan_token(INDICATOR)) return true;
+  final private boolean jj_3_1070() {
+    if (jj_scan_token(INPUT)) return true;
     return false;
   }
 
@@ -23388,8 +23397,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1068() {
-    if (jj_scan_token(IMMEDIATE)) return true;
+  final private boolean jj_3_1069() {
+    if (jj_scan_token(INITIAL)) return true;
     return false;
   }
 
@@ -23398,33 +23407,33 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1068() {
+    if (jj_scan_token(IMMEDIATELY)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1067() {
-    if (jj_scan_token(HOLD)) return true;
+    if (jj_scan_token(HOUR)) return true;
     return false;
   }
 
   final private boolean jj_3_1066() {
-    if (jj_scan_token(GO)) return true;
+    if (jj_scan_token(GOTO)) return true;
     return false;
   }
 
   final private boolean jj_3_1065() {
-    if (jj_scan_token(GENERAL)) return true;
+    if (jj_scan_token(GET)) return true;
     return false;
   }
 
   final private boolean jj_3_1064() {
-    if (jj_scan_token(FUNCTION)) return true;
+    if (jj_scan_token(FUSION)) return true;
     return false;
   }
 
   final private boolean jj_3_1063() {
-    if (jj_scan_token(FOUND)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1062() {
-    if (jj_scan_token(FLOOR)) return true;
+    if (jj_scan_token(FRAME_ROW)) return true;
     return false;
   }
 
@@ -23433,8 +23442,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1061() {
-    if (jj_scan_token(FIRST)) return true;
+  final private boolean jj_3_1062() {
+    if (jj_scan_token(FOR)) return true;
     return false;
   }
 
@@ -23442,13 +23451,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1060() {
-    if (jj_scan_token(EXTRACT)) return true;
+  final private boolean jj_3_1061() {
+    if (jj_scan_token(FIRST_VALUE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1059() {
-    if (jj_scan_token(EXP)) return true;
+  final private boolean jj_3_1060() {
+    if (jj_scan_token(FALSE)) return true;
     return false;
   }
 
@@ -23467,18 +23476,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1058() {
-    if (jj_scan_token(EXEC)) return true;
+  final private boolean jj_3_1059() {
+    if (jj_scan_token(EXTEND)) return true;
     return false;
   }
 
-  final private boolean jj_3_1057() {
-    if (jj_scan_token(ESCAPE)) return true;
+  final private boolean jj_3_1058() {
+    if (jj_scan_token(EXECUTE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1056() {
-    if (jj_scan_token(END_FRAME)) return true;
+  final private boolean jj_3_1057() {
+    if (jj_scan_token(EVERY)) return true;
     return false;
   }
 
@@ -23487,28 +23496,28 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1056() {
+    if (jj_scan_token(END_PARTITION)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1055() {
-    if (jj_scan_token(ELSE)) return true;
+    if (jj_scan_token(EMPTY)) return true;
     return false;
   }
 
   final private boolean jj_3_1054() {
-    if (jj_scan_token(DYNAMIC)) return true;
+    if (jj_scan_token(EACH)) return true;
     return false;
   }
 
   final private boolean jj_3_1053() {
-    if (jj_scan_token(DISCONNECT)) return true;
+    if (jj_scan_token(DOMAIN)) return true;
     return false;
   }
 
   final private boolean jj_3_1052() {
-    if (jj_scan_token(DETERMINISTIC)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1051() {
-    if (jj_scan_token(DEREF)) return true;
+    if (jj_scan_token(DIAGNOSTICS)) return true;
     return false;
   }
 
@@ -23517,8 +23526,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1050() {
-    if (jj_scan_token(DEFERRED)) return true;
+  final private boolean jj_3_1051() {
+    if (jj_scan_token(DESC)) return true;
     return false;
   }
 
@@ -23527,23 +23536,23 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1049() {
-    if (jj_scan_token(DECIMAL)) return true;
+  final private boolean jj_3_1050() {
+    if (jj_scan_token(DENSE_RANK)) return true;
     return false;
   }
 
-  final private boolean jj_3_1048() {
-    if (jj_scan_token(DAY)) return true;
+  final private boolean jj_3_1049() {
+    if (jj_scan_token(DECLARE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1047() {
-    if (jj_scan_token(CYCLE)) return true;
+  final private boolean jj_3_1048() {
+    if (jj_scan_token(DEALLOCATE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1046() {
-    if (jj_scan_token(CURRENT_TIMESTAMP)) return true;
+  final private boolean jj_3_1047() {
+    if (jj_scan_token(DATA)) return true;
     return false;
   }
 
@@ -23567,28 +23576,28 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1046() {
+    if (jj_scan_token(CURRENT_TRANSFORM_GROUP_FOR_TYPE)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1045() {
-    if (jj_scan_token(CURRENT_ROW)) return true;
+    if (jj_scan_token(CURRENT_SCHEMA)) return true;
     return false;
   }
 
   final private boolean jj_3_1044() {
-    if (jj_scan_token(CURRENT_DEFAULT_TRANSFORM_GROUP)) return true;
+    if (jj_scan_token(CURRENT_PATH)) return true;
     return false;
   }
 
   final private boolean jj_3_1043() {
-    if (jj_scan_token(CUME_DIST)) return true;
+    if (jj_scan_token(CURRENT_CATALOG)) return true;
     return false;
   }
 
   final private boolean jj_3_1042() {
-    if (jj_scan_token(COVAR_POP)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1041() {
-    if (jj_scan_token(CORR)) return true;
+    if (jj_scan_token(COVAR_SAMP)) return true;
     return false;
   }
 
@@ -23620,88 +23629,88 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1041() {
+    if (jj_scan_token(CORRESPONDING)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1040() {
-    if (jj_scan_token(CONTAINS)) return true;
+    if (jj_scan_token(CONTINUE)) return true;
     return false;
   }
 
   final private boolean jj_3_1039() {
-    if (jj_scan_token(CONNECTION)) return true;
+    if (jj_scan_token(CONSTRAINTS)) return true;
     return false;
   }
 
   final private boolean jj_3_1038() {
-    if (jj_scan_token(COMMIT)) return true;
+    if (jj_scan_token(CONDITION)) return true;
     return false;
   }
 
   final private boolean jj_3_1037() {
-    if (jj_scan_token(COLLATION)) return true;
+    if (jj_scan_token(COLLECT)) return true;
     return false;
   }
 
   final private boolean jj_3_1036() {
-    if (jj_scan_token(CLOSE)) return true;
+    if (jj_scan_token(COALESCE)) return true;
     return false;
   }
 
   final private boolean jj_3_1035() {
-    if (jj_scan_token(CHECK)) return true;
+    if (jj_scan_token(CLASSIFIER)) return true;
     return false;
   }
 
-  final private boolean jj_3_1034() {
-    if (jj_scan_token(CHARACTER)) return true;
+  final private boolean jj_3R_123() {
+    if (jj_3R_283()) return true;
     return false;
   }
 
-  final private boolean jj_3R_123() {
-    if (jj_3R_283()) return true;
+  final private boolean jj_3_1034() {
+    if (jj_scan_token(CHARACTER_LENGTH)) return true;
     return false;
   }
 
   final private boolean jj_3_1033() {
-    if (jj_scan_token(CEIL)) return true;
+    if (jj_scan_token(CEILING)) return true;
     return false;
   }
 
   final private boolean jj_3_1032() {
-    if (jj_scan_token(CASCADED)) return true;
+    if (jj_scan_token(CAST)) return true;
     return false;
   }
 
   final private boolean jj_3_1031() {
-    if (jj_scan_token(CALLED)) return true;
+    if (jj_scan_token(CARDINALITY)) return true;
     return false;
   }
 
-  final private boolean jj_3_1030() {
-    if (jj_scan_token(BREADTH)) return true;
+  final private boolean jj_3_184() {
+    if (jj_3R_66()) return true;
     return false;
   }
 
-  final private boolean jj_3_184() {
-    if (jj_3R_66()) return true;
+  final private boolean jj_3_1030() {
+    if (jj_scan_token(BY)) return true;
     return false;
   }
 
   final private boolean jj_3_1029() {
-    if (jj_scan_token(BLOB)) return true;
+    if (jj_scan_token(BOOLEAN)) return true;
     return false;
   }
 
   final private boolean jj_3_1028() {
-    if (jj_scan_token(BIGINT)) return true;
+    if (jj_scan_token(BINARY)) return true;
     return false;
   }
 
   final private boolean jj_3_1027() {
-    if (jj_scan_token(BEGIN_FRAME)) return true;
-    return false;
-  }
-
-  final private boolean jj_3_1026() {
-    if (jj_scan_token(AVG)) return true;
+    if (jj_scan_token(BEGIN_PARTITION)) return true;
     return false;
   }
 
@@ -23710,13 +23719,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1025() {
-    if (jj_scan_token(AT)) return true;
+  final private boolean jj_3_1026() {
+    if (jj_scan_token(BEFORE)) return true;
     return false;
   }
 
-  final private boolean jj_3_1024() {
-    if (jj_scan_token(ASENSITIVE)) return true;
+  final private boolean jj_3_1025() {
+    if (jj_scan_token(ATOMIC)) return true;
     return false;
   }
 
@@ -23732,18 +23741,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1023() {
-    if (jj_scan_token(ARRAY_MAX_CARDINALITY)) return true;
+  final private boolean jj_3_1024() {
+    if (jj_scan_token(ASSERTION)) return true;
     return false;
   }
 
-  final private boolean jj_3_1022() {
-    if (jj_scan_token(AND)) return true;
+  final private boolean jj_3_1023() {
+    if (jj_scan_token(AS)) return true;
     return false;
   }
 
-  final private boolean jj_3_1021() {
-    if (jj_scan_token(ALLOCATE)) return true;
+  final private boolean jj_3_1022() {
+    if (jj_scan_token(ARE)) return true;
     return false;
   }
 
@@ -23752,18 +23761,18 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1020() {
-    if (jj_scan_token(ACTION)) return true;
+  final private boolean jj_3_1021() {
+    if (jj_scan_token(ALLOW)) return true;
     return false;
   }
 
-  final private boolean jj_3_1019() {
-    if (jj_scan_token(A)) return true;
+  final private boolean jj_3_1020() {
+    if (jj_scan_token(ADD)) return true;
     return false;
   }
 
-  final private boolean jj_3_1018() {
-    if (jj_scan_token(ASYNC)) return true;
+  final private boolean jj_3_1019() {
+    if (jj_scan_token(ABS)) return true;
     return false;
   }
 
@@ -23772,6 +23781,11 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1018() {
+    if (jj_scan_token(ASYNC)) return true;
+    return false;
+  }
+
   final private boolean jj_3_1017() {
     if (jj_scan_token(CONTINUOUS)) return true;
     return false;
@@ -23817,6 +23831,11 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_564() {
+    if (jj_scan_token(SQL_TSI_QUARTER)) return true;
+    return false;
+  }
+
   final private boolean jj_3R_266() {
     Token xsp;
     xsp = jj_scanpos;
@@ -24080,7 +24099,10 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     jj_scanpos = xsp;
     if (jj_3_1140()) {
     jj_scanpos = xsp;
-    if (jj_3_1141()) return true;
+    if (jj_3_1141()) {
+    jj_scanpos = xsp;
+    if (jj_3_1142()) return true;
+    }
     }
     }
     }
@@ -24211,11 +24233,6 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     }
     }
     }
-    return false;
-  }
-
-  final private boolean jj_3_564() {
-    if (jj_scan_token(SQL_TSI_QUARTER)) return true;
     return false;
   }
 
@@ -24265,18 +24282,13 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1010() {
-    if (jj_3R_268()) return true;
-    return false;
-  }
-
   final private boolean jj_3_555() {
     if (jj_scan_token(HOUR)) return true;
     return false;
   }
 
-  final private boolean jj_3_1009() {
-    if (jj_3R_267()) return true;
+  final private boolean jj_3_1010() {
+    if (jj_3R_268()) return true;
     return false;
   }
 
@@ -24285,8 +24297,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_1008() {
-    if (jj_3R_266()) return true;
+  final private boolean jj_3_1009() {
+    if (jj_3R_267()) return true;
     return false;
   }
 
@@ -24295,6 +24307,11 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_1008() {
+    if (jj_3R_266()) return true;
+    return false;
+  }
+
   final private boolean jj_3_552() {
     if (jj_scan_token(SQL_TSI_SECOND)) return true;
     return false;
@@ -24310,6 +24327,11 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
+  final private boolean jj_3_549() {
+    if (jj_scan_token(SQL_TSI_FRAC_SECOND)) return true;
+    return false;
+  }
+
   final private boolean jj_3R_212() {
     Token xsp;
     xsp = jj_scanpos;
@@ -24323,11 +24345,6 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     return false;
   }
 
-  final private boolean jj_3_549() {
-    if (jj_scan_token(SQL_TSI_FRAC_SECOND)) return true;
-    return false;
-  }
-
   final private boolean jj_3_548() {
     if (jj_scan_token(NANOSECOND)) return true;
     return false;
@@ -25312,17 +25329,17 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
     xsp = jj_scanpos;
     if (jj_scan_token(600)) {
     jj_scanpos = xsp;
-    if (jj_scan_token(781)) {
-    jj_scanpos = xsp;
     if (jj_scan_token(780)) {
     jj_scanpos = xsp;
+    if (jj_scan_token(779)) {
+    jj_scanpos = xsp;
+    if (jj_scan_token(776)) {
+    jj_scanpos = xsp;
     if (jj_scan_token(777)) {
     jj_scanpos = xsp;
     if (jj_scan_token(778)) {
     jj_scanpos = xsp;
-    if (jj_scan_token(779)) {
-    jj_scanpos = xsp;
-    if (jj_scan_token(776)) return true;
+    if (jj_scan_token(775)) return true;
     }
     }
     }
@@ -31795,15 +31812,15 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
       jj_la1_21 = new int[] {0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,};
    }
    private static void jj_la1_22() {
-      jj_la1_22 = new int[] {0x0,0x10000,0x20000000,0x0,0x10000,0x10000,0x800000,0x800000,0x0,0x0,};
+      jj_la1_22 = new int[] {0x0,0x8000,0x10000000,0x0,0x8000,0x8000,0x400000,0x400000,0x0,0x0,};
    }
    private static void jj_la1_23() {
-      jj_la1_23 = new int[] {0x0,0x0,0xa080,0x6000,0x0,0x0,0x0,0x0,0x0,0x0,};
+      jj_la1_23 = new int[] {0x0,0x0,0x5040,0x3000,0x0,0x0,0x0,0x0,0x0,0x0,};
    }
    private static void jj_la1_24() {
       jj_la1_24 = new int[] {0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,};
    }
-  final private JJCalls[] jj_2_rtns = new JJCalls[1403];
+  final private JJCalls[] jj_2_rtns = new JJCalls[1404];
   private boolean jj_rescan = false;
   private int jj_gc = 0;
 
@@ -31978,8 +31995,8 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
 
   public ParseException generateParseException() {
     jj_expentries.removeAllElements();
-    boolean[] la1tokens = new boolean[787];
-    for (int i = 0; i < 787; i++) {
+    boolean[] la1tokens = new boolean[786];
+    for (int i = 0; i < 786; i++) {
       la1tokens[i] = false;
     }
     if (jj_kind >= 0) {
@@ -32067,7 +32084,7 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
         }
       }
     }
-    for (int i = 0; i < 787; i++) {
+    for (int i = 0; i < 786; i++) {
       if (la1tokens[i]) {
         jj_expentry = new int[1];
         jj_expentry[0] = i;
@@ -32092,7 +32109,7 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
 
   final private void jj_rescan_token() {
     jj_rescan = true;
-    for (int i = 0; i < 1403; i++) {
+    for (int i = 0; i < 1404; i++) {
     try {
       JJCalls p = jj_2_rtns[i];
       do {
@@ -33502,6 +33519,7 @@ public class IgniteSqlParserImpl extends SqlAbstractParserImpl implements Ignite
             case 1400: jj_3_1401(); break;
             case 1401: jj_3_1402(); break;
             case 1402: jj_3_1403(); break;
+            case 1403: jj_3_1404(); break;
           }
         }
         p = p.next;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImplConstants.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImplConstants.java
index 000001e0904..407f02231da 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImplConstants.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImplConstants.java
@@ -715,73 +715,72 @@ public interface IgniteSqlParserImplConstants {
   int COMPUTE = 709;
   int ASYNC = 710;
   int QUERY = 711;
-  int UUID = 712;
-  int UNSIGNED_INTEGER_LITERAL = 713;
-  int APPROX_NUMERIC_LITERAL = 714;
-  int DECIMAL_NUMERIC_LITERAL = 715;
-  int EXPONENT = 716;
-  int HEXDIGIT = 717;
-  int WHITESPACE = 718;
-  int BINARY_STRING_LITERAL = 719;
-  int QUOTED_STRING = 720;
-  int PREFIXED_STRING_LITERAL = 721;
-  int UNICODE_STRING_LITERAL = 722;
-  int CHARSETNAME = 723;
-  int BIG_QUERY_DOUBLE_QUOTED_STRING = 724;
-  int BIG_QUERY_QUOTED_STRING = 725;
-  int UNICODE_QUOTED_ESCAPE_CHAR = 726;
-  int LPAREN = 727;
-  int RPAREN = 728;
-  int LBRACE_D = 729;
-  int LBRACE_T = 730;
-  int LBRACE_TS = 731;
-  int LBRACE_FN = 732;
-  int LBRACE = 733;
-  int RBRACE = 734;
-  int LBRACKET = 735;
-  int RBRACKET = 736;
-  int SEMICOLON = 737;
-  int DOT = 738;
-  int COMMA = 739;
-  int EQ = 740;
-  int GT = 741;
-  int LT = 742;
-  int HOOK = 743;
-  int COLON = 744;
-  int LE = 745;
-  int GE = 746;
-  int NE = 747;
-  int NE2 = 748;
-  int PLUS = 749;
-  int MINUS = 750;
-  int STAR = 751;
-  int SLASH = 752;
-  int PERCENT_REMAINDER = 753;
-  int CONCAT = 754;
-  int NAMED_ARGUMENT_ASSIGNMENT = 755;
-  int DOUBLE_PERIOD = 756;
-  int QUOTE = 757;
-  int DOUBLE_QUOTE = 758;
-  int VERTICAL_BAR = 759;
-  int CARET = 760;
-  int DOLLAR = 761;
-  int INFIX_CAST = 762;
-  int HINT_BEG = 768;
-  int COMMENT_END = 769;
-  int SINGLE_LINE_COMMENT = 772;
-  int FORMAL_COMMENT = 773;
-  int MULTI_LINE_COMMENT = 774;
-  int BRACKET_QUOTED_IDENTIFIER = 776;
-  int QUOTED_IDENTIFIER = 777;
-  int BACK_QUOTED_IDENTIFIER = 778;
-  int BIG_QUERY_BACK_QUOTED_IDENTIFIER = 779;
-  int HYPHENATED_IDENTIFIER = 780;
-  int IDENTIFIER = 781;
-  int COLLATION_ID = 782;
-  int UNICODE_QUOTED_IDENTIFIER = 783;
-  int LETTER = 784;
-  int DIGIT = 785;
-  int BEL = 786;
+  int UNSIGNED_INTEGER_LITERAL = 712;
+  int APPROX_NUMERIC_LITERAL = 713;
+  int DECIMAL_NUMERIC_LITERAL = 714;
+  int EXPONENT = 715;
+  int HEXDIGIT = 716;
+  int WHITESPACE = 717;
+  int BINARY_STRING_LITERAL = 718;
+  int QUOTED_STRING = 719;
+  int PREFIXED_STRING_LITERAL = 720;
+  int UNICODE_STRING_LITERAL = 721;
+  int CHARSETNAME = 722;
+  int BIG_QUERY_DOUBLE_QUOTED_STRING = 723;
+  int BIG_QUERY_QUOTED_STRING = 724;
+  int UNICODE_QUOTED_ESCAPE_CHAR = 725;
+  int LPAREN = 726;
+  int RPAREN = 727;
+  int LBRACE_D = 728;
+  int LBRACE_T = 729;
+  int LBRACE_TS = 730;
+  int LBRACE_FN = 731;
+  int LBRACE = 732;
+  int RBRACE = 733;
+  int LBRACKET = 734;
+  int RBRACKET = 735;
+  int SEMICOLON = 736;
+  int DOT = 737;
+  int COMMA = 738;
+  int EQ = 739;
+  int GT = 740;
+  int LT = 741;
+  int HOOK = 742;
+  int COLON = 743;
+  int LE = 744;
+  int GE = 745;
+  int NE = 746;
+  int NE2 = 747;
+  int PLUS = 748;
+  int MINUS = 749;
+  int STAR = 750;
+  int SLASH = 751;
+  int PERCENT_REMAINDER = 752;
+  int CONCAT = 753;
+  int NAMED_ARGUMENT_ASSIGNMENT = 754;
+  int DOUBLE_PERIOD = 755;
+  int QUOTE = 756;
+  int DOUBLE_QUOTE = 757;
+  int VERTICAL_BAR = 758;
+  int CARET = 759;
+  int DOLLAR = 760;
+  int INFIX_CAST = 761;
+  int HINT_BEG = 767;
+  int COMMENT_END = 768;
+  int SINGLE_LINE_COMMENT = 771;
+  int FORMAL_COMMENT = 772;
+  int MULTI_LINE_COMMENT = 773;
+  int BRACKET_QUOTED_IDENTIFIER = 775;
+  int QUOTED_IDENTIFIER = 776;
+  int BACK_QUOTED_IDENTIFIER = 777;
+  int BIG_QUERY_BACK_QUOTED_IDENTIFIER = 778;
+  int HYPHENATED_IDENTIFIER = 779;
+  int IDENTIFIER = 780;
+  int COLLATION_ID = 781;
+  int UNICODE_QUOTED_IDENTIFIER = 782;
+  int LETTER = 783;
+  int DIGIT = 784;
+  int BEL = 785;
 
   int DEFAULT = 0;
   int DQID = 1;
@@ -1504,7 +1503,6 @@ public interface IgniteSqlParserImplConstants {
     "\"COMPUTE\"",
     "\"ASYNC\"",
     "\"QUERY\"",
-    "\"UUID\"",
     "<UNSIGNED_INTEGER_LITERAL>",
     "<APPROX_NUMERIC_LITERAL>",
     "<DECIMAL_NUMERIC_LITERAL>",
@@ -1562,12 +1560,12 @@ public interface IgniteSqlParserImplConstants {
     "\"\\f\"",
     "\"/*+\"",
     "\"*/\"",
-    "<token of kind 770>",
+    "<token of kind 769>",
     "\"/*\"",
     "<SINGLE_LINE_COMMENT>",
     "<FORMAL_COMMENT>",
     "<MULTI_LINE_COMMENT>",
-    "<token of kind 775>",
+    "<token of kind 774>",
     "<BRACKET_QUOTED_IDENTIFIER>",
     "<QUOTED_IDENTIFIER>",
     "<BACK_QUOTED_IDENTIFIER>",
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImplTokenManager.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImplTokenManager.java
index 4770f9e18a8..93c5fcac778 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImplTokenManager.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/generated/IgniteSqlParserImplTokenManager.java
@@ -143,62 +143,62 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
    switch (pos)
    {
       case 0:
-         if ((active5 & 0x1fffffc00000L) != 0L || (active10 & 0x8000000000000000L) != 0L)
-         {
-            jjmatchedKind = 781;
-            return 86;
-         }
          if ((active10 & 0x80000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             return 1;
          }
-         if ((active11 & 0x40000000000000L) != 0L)
-            return 87;
-         if ((active11 & 0x20000000L) != 0L)
-            return 88;
-         if ((active9 & 0xf000000000000000L) != 0L || (active10 & 0xfffffL) != 0L || (active11 & 0x100L) != 0L)
-         {
-            jjmatchedKind = 781;
-            return 58;
-         }
          if ((active11 & 0x20000000000000L) != 0L)
+            return 86;
+         if ((active11 & 0x10000000L) != 0L)
+            return 87;
+         if ((active11 & 0x10000000000000L) != 0L)
             return 55;
-         if ((active11 & 0x1000000000000L) != 0L || (active12 & 0x9L) != 0L)
+         if ((active11 & 0x8000800000000000L) != 0L || (active12 & 0x4L) != 0L)
             return 84;
-         if ((active0 & 0x7ffe000000000L) != 0L || (active2 & 0xfffffffffffffff0L) != 0L || (active3 & 0xffffe0007fffffffL) != 0L || (active4 & 0x7ffffe1fffffffL) != 0L || (active5 & 0xffffe00000000000L) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0x7fffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xfffffffffffffffL) != 0L || (active10 & 0x7f27f7fffff00000L) != 0L || (active11 & 0x95L) != 0L)
+         if ((active5 & 0x1fffffc00000L) != 0L || (active10 & 0x8000000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
-            return 89;
+            jjmatchedKind = 780;
+            return 88;
          }
-         if ((active0 & 0xfff8001ffffffff8L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfL) != 0L || (active3 & 0x1fff80000000L) != 0L || (active4 & 0xff800001e0000000L) != 0L || (active5 & 0x3fffffL) != 0L || (active7 & 0x8000000000000000L) != 0L || (active10 & 0xd8000000000000L) != 0L || (active11 & 0x20000000000006aL) != 0L)
+         if ((active0 & 0xfff8001ffffffff8L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfL) != 0L || (active3 & 0x1fff80000000L) != 0L || (active4 & 0xff800001e0000000L) != 0L || (active5 & 0x3fffffL) != 0L || (active7 & 0x8000000000000000L) != 0L || (active10 & 0xd8000000000000L) != 0L || (active11 & 0x10000000000006aL) != 0L)
             return 89;
-         if ((active11 & 0x400000000000L) != 0L)
+         if ((active11 & 0x200000000000L) != 0L)
             return 15;
-         if ((active11 & 0x10000400000000L) != 0L)
+         if ((active0 & 0x7ffe000000000L) != 0L || (active2 & 0xfffffffffffffff0L) != 0L || (active3 & 0xffffe0007fffffffL) != 0L || (active4 & 0x7ffffe1fffffffL) != 0L || (active5 & 0xffffe00000000000L) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0x7fffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xfffffffffffffffL) != 0L || (active10 & 0x7f27f7fffff00000L) != 0L || (active11 & 0x95L) != 0L)
+         {
+            jjmatchedKind = 780;
+            return 89;
+         }
+         if ((active9 & 0xf000000000000000L) != 0L || (active10 & 0xfffffL) != 0L)
+         {
+            jjmatchedKind = 780;
+            return 58;
+         }
+         if ((active11 & 0x8000200000000L) != 0L)
             return 90;
          return -1;
       case 1:
-         if ((active12 & 0x9L) != 0L)
-            return 82;
-         if ((active0 & 0x40007fe000000L) != 0L || (active3 & 0xfc00006000000000L) != 0L || (active4 & 0x200000007ffffL) != 0L || (active5 & 0x78e6001f00000000L) != 0L || (active9 & 0x6000000000L) != 0L || (active10 & 0xa811000000000000L) != 0L || (active11 & 0x40L) != 0L)
-            return 89;
-         if ((active0 & 0xfff3fff801fffff0L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xffffffffffffffffL) != 0L || (active3 & 0x3ffff9f7fffffffL) != 0L || (active4 & 0xff7dffffdff80000L) != 0L || (active5 & 0x8719ffe0ffffffffL) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0xffffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xffffff9fffffffffL) != 0L || (active10 & 0x57eeffffffffffffL) != 0L || (active11 & 0x1bfL) != 0L)
+         if ((active0 & 0xfff3fff801fffff0L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xffffffffffffffffL) != 0L || (active3 & 0x3ffff9f7fffffffL) != 0L || (active4 & 0xff7dffffdff80000L) != 0L || (active5 & 0x8719ffe0ffffffffL) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0xffffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xffffff9fffffffffL) != 0L || (active10 & 0x57eeffffffffffffL) != 0L || (active11 & 0xbfL) != 0L)
          {
             if (jjmatchedPos != 1)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 1;
             }
             return 89;
          }
+         if ((active11 & 0x8000000000000000L) != 0L || (active12 & 0x4L) != 0L)
+            return 82;
+         if ((active0 & 0x40007fe000000L) != 0L || (active3 & 0xfc00006000000000L) != 0L || (active4 & 0x200000007ffffL) != 0L || (active5 & 0x78e6001f00000000L) != 0L || (active9 & 0x6000000000L) != 0L || (active10 & 0xa811000000000000L) != 0L || (active11 & 0x40L) != 0L)
+            return 89;
          return -1;
       case 2:
-         if ((active0 & 0xfff3dfef79e98c80L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfe1ff2ffffff09ffL) != 0L || (active3 & 0xfbfeffd77f87ff9fL) != 0L || (active4 & 0x3e7dfffa1fff40ffL) != 0L || (active5 & 0xf795ffeebffe03ffL) != 0L || (active6 & 0xf8fffffdc07fff78L) != 0L || (active7 & 0x1ffffffc3fffffffL) != 0L || (active8 & 0xfffL) != 0L || (active9 & 0xffffffdffff7fffeL) != 0L || (active10 & 0xfffef7ffffffffffL) != 0L || (active11 & 0x1ffL) != 0L)
+         if ((active0 & 0xfff3dfef79e98c80L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfe1ff2ffffff09ffL) != 0L || (active3 & 0xfbfeffd77f87ff9fL) != 0L || (active4 & 0x3e7dfffa1fff40ffL) != 0L || (active5 & 0xf795ffeebffe03ffL) != 0L || (active6 & 0xf8fffffdc07fff78L) != 0L || (active7 & 0x1ffffffc3fffffffL) != 0L || (active8 & 0xfffL) != 0L || (active9 & 0xffffffdffff7fffeL) != 0L || (active10 & 0xfffef7ffffffffffL) != 0L || (active11 & 0xffL) != 0L)
          {
             if (jjmatchedPos != 2)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 2;
             }
             return 89;
@@ -213,26 +213,26 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 3)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 3;
             }
             return 89;
          }
-         if ((active0 & 0x6631400000000000L) != 0L || (active1 & 0x83000000023feL) != 0L || (active2 & 0x51001e00005f0L) != 0L || (active3 & 0x680401c00000cL) != 0L || (active4 & 0xc7601ff82000L) != 0L || (active5 & 0x190078280c80000L) != 0L || (active6 & 0x78080100300078L) != 0L || (active7 & 0x4014000200800000L) != 0L || (active8 & 0x59L) != 0L || (active9 & 0x9c0000ff0000002L) != 0L || (active10 & 0x100f1e3c002f800L) != 0L || (active11 & 0x106L) != 0L)
+         if ((active0 & 0x6631400000000000L) != 0L || (active1 & 0x83000000023feL) != 0L || (active2 & 0x51001e00005f0L) != 0L || (active3 & 0x680401c00000cL) != 0L || (active4 & 0xc7601ff82000L) != 0L || (active5 & 0x190078280c80000L) != 0L || (active6 & 0x78080100300078L) != 0L || (active7 & 0x4014000200800000L) != 0L || (active8 & 0x59L) != 0L || (active9 & 0x9c0000ff0000002L) != 0L || (active10 & 0x100f1e3c002f800L) != 0L || (active11 & 0x6L) != 0L)
             return 89;
          return -1;
       case 4:
+         if ((active2 & 0x40000000000000L) != 0L)
+            return 91;
          if ((active0 & 0xd9e29e2f780120e0L) != 0L || (active1 & 0xfff3affffffe9bfcL) != 0L || (active2 & 0xd592e2ffd3ffe9a7L) != 0L || (active3 & 0xfbd86017637413dbL) != 0L || (active4 & 0x8020aac99fc75ef1L) != 0L || (active5 & 0xc605fb2c2711bbfbL) != 0L || (active6 & 0xfe83d7dcbf4fff74L) != 0L || (active7 & 0x1febf0ddbf1ffeffL) != 0L || (active8 & 0xffffffffffffeea6L) != 0L || (active9 & 0xf6bdffdf88f7f7e1L) != 0L || (active10 & 0xf5de02da3f80f2fcL) != 0L || (active11 & 0x39L) != 0L)
          {
             if (jjmatchedPos != 4)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 4;
             }
             return 89;
          }
-         if ((active2 & 0x40000000000000L) != 0L)
-            return 91;
          if ((active0 & 0x1c001e8cc00L) != 0L || (active1 & 0x4400000014401L) != 0L || (active2 & 0x2a0800000c000008L) != 0L || (active3 & 0x241f800083ec00L) != 0L || (active4 & 0x3e5d10120000000eL) != 0L || (active5 & 0x3000044018a60004L) != 0L || (active6 & 0x24202040000002L) != 0L || (active7 & 0x80000f2000600100L) != 0L || (active8 & 0x100L) != 0L || (active9 & 0x200000700081cL) != 0L || (active10 & 0xa202404007d0503L) != 0L || (active11 & 0xc0L) != 0L)
             return 89;
          return -1;
@@ -243,7 +243,7 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 5)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 5;
             }
             return 89;
@@ -254,17 +254,17 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
       case 6:
          if ((active2 & 0x40000000000000L) != 0L)
             return 91;
+         if ((active0 & 0xd982840000000000L) != 0L || (active1 & 0xffe0080700420000L) != 0L || (active2 & 0x2e0001063c001L) != 0L || (active3 & 0x8800010100600249L) != 0L || (active4 & 0xc20228000024201L) != 0L || (active5 & 0x400100024001000L) != 0L || (active6 & 0x80021204af448020L) != 0L || (active7 & 0x180180043c0806a0L) != 0L || (active8 & 0x40002L) != 0L || (active9 & 0x103c001000000000L) != 0L || (active10 & 0x400402803c800038L) != 0L || (active11 & 0x30L) != 0L)
+            return 89;
          if ((active0 & 0x40038e79c02040L) != 0L || (active1 & 0x11a7f8f73c9bfcL) != 0L || (active2 & 0x8190007fc20c09a4L) != 0L || (active3 & 0x71c84c0623048102L) != 0L || (active4 & 0xb01880499fc41cd0L) != 0L || (active5 & 0x6000c12c0310a3b9L) != 0L || (active6 & 0x7e80c5d800097f50L) != 0L || (active7 & 0x5e20e898107c85fL) != 0L || (active8 & 0xfffffffffffbee20L) != 0L || (active9 & 0xe281ffcf8ed7b6d1L) != 0L || (active10 & 0xb7fa000a0340f000L) != 0L || (active11 & 0x9L) != 0L)
          {
             if (jjmatchedPos != 6)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 6;
             }
             return 89;
          }
-         if ((active0 & 0xd982840000000000L) != 0L || (active1 & 0xffe0080700420000L) != 0L || (active2 & 0x2e0001063c001L) != 0L || (active3 & 0x8800010100600249L) != 0L || (active4 & 0xc20228000024201L) != 0L || (active5 & 0x400100024001000L) != 0L || (active6 & 0x80021204af448020L) != 0L || (active7 & 0x180180043c0806a0L) != 0L || (active8 & 0x40002L) != 0L || (active9 & 0x103c001000000000L) != 0L || (active10 & 0x400402803c800038L) != 0L || (active11 & 0x30L) != 0L)
-            return 89;
          return -1;
       case 7:
          if ((active2 & 0x40000000000000L) != 0L)
@@ -273,7 +273,7 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 7)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 7;
             }
             return 89;
@@ -282,37 +282,37 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
             return 89;
          return -1;
       case 8:
-         if ((active0 & 0x20610400000L) != 0L || (active1 & 0x108000f03c03f8L) != 0L || (active2 & 0x8080000000000000L) != 0L || (active3 & 0x70c0400202040002L) != 0L || (active4 & 0x18000000040c01L) != 0L || (active5 & 0x2c00000000L) != 0L || (active6 & 0x1880818000017f00L) != 0L || (active7 & 0x60000880040000L) != 0L || (active8 & 0x300004000000L) != 0L || (active9 & 0x2000f00388130410L) != 0L || (active10 & 0x8410000002000000L) != 0L)
-            return 89;
          if ((active0 & 0x1040018869800000L) != 0L || (active1 & 0xffc121fe07001804L) != 0L || (active2 & 0x100c01782040985L) != 0L || (active3 & 0x100040001008000L) != 0L || (active4 & 0x300080409fc00090L) != 0L || (active5 & 0x4000c000010003b8L) != 0L || (active6 & 0x660040100e080040L) != 0L || (active7 & 0x15000e813803c041L) != 0L || (active8 & 0xfffacffffb13ac00L) != 0L || (active9 & 0xc0b90f4c06c0b2c1L) != 0L || (active10 & 0x23e800082000f000L) != 0L || (active11 & 0x8L) != 0L)
          {
             if (jjmatchedPos != 8)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 8;
             }
             return 89;
          }
+         if ((active0 & 0x20610400000L) != 0L || (active1 & 0x108000f03c03f8L) != 0L || (active2 & 0x8080000000000000L) != 0L || (active3 & 0x70c0400202040002L) != 0L || (active4 & 0x18000000040c01L) != 0L || (active5 & 0x2c00000000L) != 0L || (active6 & 0x1880818000017f00L) != 0L || (active7 & 0x60000880040000L) != 0L || (active8 & 0x300004000000L) != 0L || (active9 & 0x2000f00388130410L) != 0L || (active10 & 0x8410000002000000L) != 0L)
+            return 89;
          return -1;
       case 9:
-         if ((active0 & 0x468000000L) != 0L || (active1 & 0x100fe00000a00L) != 0L || (active2 & 0x1102040800L) != 0L || (active4 & 0x804098c00000L) != 0L || (active5 & 0x4000000801000200L) != 0L || (active6 & 0x2200400000000040L) != 0L || (active7 & 0x40100000041L) != 0L || (active8 & 0x2038000L) != 0L || (active9 & 0x80200002008240L) != 0L || (active10 & 0x280000020000000L) != 0L || (active11 & 0x8L) != 0L)
-            return 89;
          if ((active0 & 0x1040018801800000L) != 0L || (active1 & 0xffc02100c73811f4L) != 0L || (active2 & 0x100c00680000185L) != 0L || (active3 & 0x180040001008000L) != 0L || (active4 & 0x3010000007000890L) != 0L || (active5 & 0xc000000001b8L) != 0L || (active6 & 0x440000100e087e00L) != 0L || (active7 & 0x15000a803803c000L) != 0L || (active8 & 0xfffacffff9102c00L) != 0L || (active9 & 0xc0398f4f04c23081L) != 0L || (active10 & 0x216800080000f000L) != 0L)
          {
             if (jjmatchedPos != 9)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 9;
             }
             return 89;
          }
+         if ((active0 & 0x468000000L) != 0L || (active1 & 0x100fe00000a00L) != 0L || (active2 & 0x1102040800L) != 0L || (active4 & 0x804098c00000L) != 0L || (active5 & 0x4000000801000200L) != 0L || (active6 & 0x2200400000000040L) != 0L || (active7 & 0x40100000041L) != 0L || (active8 & 0x2038000L) != 0L || (active9 & 0x80200002008240L) != 0L || (active10 & 0x280000020000000L) != 0L || (active11 & 0x8L) != 0L)
+            return 89;
          return -1;
       case 10:
          if ((active0 & 0x1000010801800000L) != 0L || (active1 & 0xf7c02074863811f0L) != 0L || (active2 & 0x100c00200000181L) != 0L || (active3 & 0x100040000000000L) != 0L || (active4 & 0x3010000000800880L) != 0L || (active5 & 0xc00000000038L) != 0L || (active6 & 0x400000100e007e00L) != 0L || (active7 & 0x11000a003803c000L) != 0L || (active8 & 0xffca8ffff0002c00L) != 0L || (active9 & 0x8038804f00021000L) != 0L || (active10 & 0x2800080000f000L) != 0L)
          {
             if (jjmatchedPos != 10)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 10;
             }
             return 89;
@@ -321,24 +321,24 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
             return 89;
          return -1;
       case 11:
-         if ((active0 & 0x1000000000000000L) != 0L || (active1 & 0x6680000000001000L) != 0L || (active2 & 0x1L) != 0L || (active3 & 0x40000000000L) != 0L || (active4 & 0x1000000000000880L) != 0L || (active5 & 0x800000000020L) != 0L || (active6 & 0x8000000L) != 0L || (active7 & 0x1100080010000000L) != 0L || (active8 & 0x6082800000002800L) != 0L || (active9 & 0x10000100000000L) != 0L || (active10 & 0x8000800000000L) != 0L)
-            return 89;
          if ((active0 & 0x10801800000L) != 0L || (active1 & 0x91402074863801f0L) != 0L || (active2 & 0x100c00200000180L) != 0L || (active3 & 0x100000000000000L) != 0L || (active4 & 0x2010000004800000L) != 0L || (active5 & 0x400000000018L) != 0L || (active6 & 0x4000001006007e00L) != 0L || (active7 & 0x2002803c000L) != 0L || (active8 & 0x9f480ffff0000400L) != 0L || (active9 & 0x80288e4e00021000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
             if (jjmatchedPos != 11)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 11;
             }
             return 89;
          }
+         if ((active0 & 0x1000000000000000L) != 0L || (active1 & 0x6680000000001000L) != 0L || (active2 & 0x1L) != 0L || (active3 & 0x40000000000L) != 0L || (active4 & 0x1000000000000880L) != 0L || (active5 & 0x800000000020L) != 0L || (active6 & 0x8000000L) != 0L || (active7 & 0x1100080010000000L) != 0L || (active8 & 0x6082800000002800L) != 0L || (active9 & 0x10000100000000L) != 0L || (active10 & 0x8000800000000L) != 0L)
+            return 89;
          return -1;
       case 12:
          if ((active0 & 0x800000000L) != 0L || (active1 & 0x200000000000L) != 0L || (active2 & 0x100000200000000L) != 0L || (active4 & 0x800000L) != 0L || (active8 & 0x8408000000000400L) != 0L || (active9 & 0x8000000600000000L) != 0L)
             return 89;
          if ((active0 & 0x10001800000L) != 0L || (active1 & 0xd1400074863801f0L) != 0L || (active2 & 0xc00000000180L) != 0L || (active3 & 0x100000000000000L) != 0L || (active4 & 0x2010000004000000L) != 0L || (active5 & 0x400000000018L) != 0L || (active6 & 0x4000001006007e00L) != 0L || (active7 & 0x2002803c000L) != 0L || (active8 & 0x1b400ffff0000000L) != 0L || (active9 & 0x288e4800021000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 12;
             return 89;
          }
@@ -346,7 +346,7 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
       case 13:
          if ((active0 & 0x10001800000L) != 0L || (active1 & 0xc1400074862801f0L) != 0L || (active2 & 0xc00000000180L) != 0L || (active4 & 0x2000000000000000L) != 0L || (active5 & 0x400000000010L) != 0L || (active6 & 0x6007800L) != 0L || (active7 & 0x803c000L) != 0L || (active8 & 0x9400ffff0000000L) != 0L || (active9 & 0x88e4800021000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 13;
             return 89;
          }
@@ -356,7 +356,7 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
       case 14:
          if ((active0 & 0x1800000L) != 0L || (active1 & 0xc1000050862800f0L) != 0L || (active2 & 0xc00000000180L) != 0L || (active5 & 0x400000000010L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x34000L) != 0L || (active8 & 0x1400bfff0000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 14;
             return 89;
          }
@@ -364,35 +364,35 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
             return 89;
          return -1;
       case 15:
+         if ((active0 & 0x800000L) != 0L || (active1 & 0x86200010L) != 0L || (active2 & 0xc00000000000L) != 0L || (active8 & 0x800f0000000L) != 0L)
+            return 89;
          if ((active0 & 0x1000000L) != 0L || (active1 & 0xc1000050000800e0L) != 0L || (active2 & 0x180L) != 0L || (active5 & 0x400000000010L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x34000L) != 0L || (active8 & 0x14003ff00000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
             if (jjmatchedPos != 15)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 15;
             }
             return 89;
          }
-         if ((active0 & 0x800000L) != 0L || (active1 & 0x86200010L) != 0L || (active2 & 0xc00000000000L) != 0L || (active8 & 0x800f0000000L) != 0L)
-            return 89;
          return -1;
       case 16:
+         if ((active1 & 0x4000004000080000L) != 0L || (active5 & 0x400000000000L) != 0L || (active7 & 0x20000L) != 0L || (active8 & 0x38700000000L) != 0L)
+            return 89;
          if ((active0 & 0x1000000L) != 0L || (active1 & 0x81000010040000e0L) != 0L || (active2 & 0x800000000180L) != 0L || (active5 & 0x10L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x14000L) != 0L || (active8 & 0x1400078e0000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
             if (jjmatchedPos != 16)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 16;
             }
             return 89;
          }
-         if ((active1 & 0x4000004000080000L) != 0L || (active5 & 0x400000000000L) != 0L || (active7 & 0x20000L) != 0L || (active8 & 0x38700000000L) != 0L)
-            return 89;
          return -1;
       case 17:
          if ((active0 & 0x1000000L) != 0L || (active1 & 0x81000000040000a0L) != 0L || (active2 & 0x800000000180L) != 0L || (active5 & 0x10L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x14000L) != 0L || (active8 & 0x140015ee0000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 17;
             return 89;
          }
@@ -404,7 +404,7 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 18)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 18;
             }
             return 89;
@@ -413,19 +413,19 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
             return 89;
          return -1;
       case 19:
-         if ((active1 & 0x80L) != 0L || (active5 & 0x10L) != 0L || (active7 & 0x4000L) != 0L)
-            return 89;
          if ((active0 & 0x1000000L) != 0L || (active1 & 0x8100000004000020L) != 0L || (active2 & 0x800000000180L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x10000L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0xc0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 19;
             return 89;
          }
+         if ((active1 & 0x80L) != 0L || (active5 & 0x10L) != 0L || (active7 & 0x4000L) != 0L)
+            return 89;
          return -1;
       case 20:
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x180L) != 0L || (active6 & 0x7800L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0xc0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 20;
             return 89;
          }
@@ -433,21 +433,21 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
             return 89;
          return -1;
       case 21:
-         if ((active2 & 0x80L) != 0L || (active9 & 0x40000000000L) != 0L || (active10 & 0x6000L) != 0L)
-            return 89;
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x7800L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0x80000000000L) != 0L || (active10 & 0x20000000009000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 21;
             return 89;
          }
+         if ((active2 & 0x80L) != 0L || (active9 & 0x40000000000L) != 0L || (active10 & 0x6000L) != 0L)
+            return 89;
          return -1;
       case 22:
          if ((active6 & 0x2000L) != 0L)
             return 89;
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x5800L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0x80000000000L) != 0L || (active10 & 0x20000000009000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 22;
             return 89;
          }
@@ -455,7 +455,7 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
       case 23:
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x5800L) != 0L || (active8 & 0x116c0000000L) != 0L || (active10 & 0x20000000001000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 23;
             return 89;
          }
@@ -465,7 +465,7 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
       case 24:
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x1800L) != 0L || (active8 & 0x116c0000000L) != 0L || (active10 & 0x20000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 24;
             return 89;
          }
@@ -473,29 +473,29 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
             return 89;
          return -1;
       case 25:
+         if ((active6 & 0x1800L) != 0L || (active8 & 0x100c0000000L) != 0L || (active10 & 0x20000000000000L) != 0L)
+            return 89;
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active8 & 0x1600000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 25;
             return 89;
          }
-         if ((active6 & 0x1800L) != 0L || (active8 & 0x100c0000000L) != 0L || (active10 & 0x20000000000000L) != 0L)
-            return 89;
          return -1;
       case 26:
+         if ((active2 & 0x100L) != 0L || (active8 & 0x600000000L) != 0L)
+            return 89;
          if ((active1 & 0x8100000000000000L) != 0L || (active8 & 0x1000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 26;
             return 89;
          }
-         if ((active2 & 0x100L) != 0L || (active8 & 0x600000000L) != 0L)
-            return 89;
          return -1;
       case 27:
          if ((active1 & 0x8100000000000000L) != 0L || (active8 & 0x1000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 27;
             return 89;
          }
@@ -503,7 +503,7 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
       case 28:
          if ((active1 & 0x8100000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 28;
             return 89;
          }
@@ -513,7 +513,7 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
       case 29:
          if ((active1 & 0x8100000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 29;
             return 89;
          }
@@ -521,7 +521,7 @@ private final int jjStopStringLiteralDfa_1(int pos, long active0, long active1,
       case 30:
          if ((active1 & 0x8000000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 30;
             return 89;
          }
@@ -556,56 +556,56 @@ private final int jjMoveStringLiteralDfa0_1()
    {
       case 33:
          jjmatchedKind = 1;
-         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x100000000000L, 0x0L);
+         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x80000000000L, 0x0L);
       case 34:
-         return jjStartNfaWithStates_1(0, 758, 87);
+         return jjStartNfaWithStates_1(0, 757, 86);
       case 36:
-         return jjStartNfaWithStates_1(0, 761, 89);
+         return jjStartNfaWithStates_1(0, 760, 89);
       case 37:
-         return jjStopAtPos(0, 753);
+         return jjStopAtPos(0, 752);
       case 39:
-         return jjStartNfaWithStates_1(0, 757, 55);
+         return jjStartNfaWithStates_1(0, 756, 55);
       case 40:
-         return jjStopAtPos(0, 727);
+         return jjStopAtPos(0, 726);
       case 41:
-         return jjStopAtPos(0, 728);
+         return jjStopAtPos(0, 727);
       case 42:
-         jjmatchedKind = 751;
-         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x2L);
+         jjmatchedKind = 750;
+         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x1L);
       case 43:
-         return jjStopAtPos(0, 749);
+         return jjStopAtPos(0, 748);
       case 44:
-         return jjStopAtPos(0, 739);
+         return jjStopAtPos(0, 738);
       case 45:
-         return jjStartNfaWithStates_1(0, 750, 15);
+         return jjStartNfaWithStates_1(0, 749, 15);
       case 46:
-         jjmatchedKind = 738;
-         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x10000000000000L, 0x0L);
+         jjmatchedKind = 737;
+         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x8000000000000L, 0x0L);
       case 47:
-         jjmatchedKind = 752;
-         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x9L);
+         jjmatchedKind = 751;
+         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x8000000000000000L, 0x4L);
       case 58:
-         jjmatchedKind = 744;
-         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x400000000000000L, 0x0L);
+         jjmatchedKind = 743;
+         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x200000000000000L, 0x0L);
       case 59:
-         return jjStopAtPos(0, 737);
+         return jjStopAtPos(0, 736);
       case 60:
-         jjmatchedKind = 742;
-         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xa0000000000L, 0x0L);
+         jjmatchedKind = 741;
+         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x50000000000L, 0x0L);
       case 61:
-         jjmatchedKind = 740;
-         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x8000000000000L, 0x0L);
+         jjmatchedKind = 739;
+         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x4000000000000L, 0x0L);
       case 62:
-         jjmatchedKind = 741;
-         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x40000000000L, 0x0L);
+         jjmatchedKind = 740;
+         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x20000000000L, 0x0L);
       case 63:
-         return jjStopAtPos(0, 743);
+         return jjStopAtPos(0, 742);
       case 91:
-         return jjStopAtPos(0, 735);
+         return jjStopAtPos(0, 734);
       case 93:
-         return jjStopAtPos(0, 736);
+         return jjStopAtPos(0, 735);
       case 94:
-         return jjStopAtPos(0, 760);
+         return jjStopAtPos(0, 759);
       case 65:
       case 97:
          jjmatchedKind = 3;
@@ -673,7 +673,7 @@ private final int jjMoveStringLiteralDfa0_1()
          return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xfffffffff000000L, 0x2000000000000L, 0x0L, 0x0L);
       case 85:
       case 117:
-         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xf000000000000000L, 0xfffffL, 0x100L, 0x0L);
+         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xf000000000000000L, 0xfffffL, 0x0L, 0x0L);
       case 86:
       case 118:
          return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x20000007ff00000L, 0x0L, 0x0L);
@@ -690,12 +690,12 @@ private final int jjMoveStringLiteralDfa0_1()
       case 122:
          return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x400000000000L, 0x0L, 0x0L);
       case 123:
-         return jjStartNfaWithStates_1(0, 733, 88);
+         return jjStartNfaWithStates_1(0, 732, 87);
       case 124:
-         jjmatchedKind = 759;
-         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x4000000000000L, 0x0L);
+         jjmatchedKind = 758;
+         return jjMoveStringLiteralDfa1_1(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x2000000000000L, 0x0L);
       case 125:
-         return jjStopAtPos(0, 734);
+         return jjStopAtPos(0, 733);
       case 126:
          return jjStopAtPos(0, 2);
       default :
@@ -712,37 +712,37 @@ private final int jjMoveStringLiteralDfa1_1(long active0, long active1, long act
    switch(curChar)
    {
       case 42:
-         if ((active12 & 0x8L) != 0L)
+         if ((active12 & 0x4L) != 0L)
          {
-            jjmatchedKind = 771;
+            jjmatchedKind = 770;
             jjmatchedPos = 1;
          }
-         return jjMoveStringLiteralDfa2_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0x1L);
+         return jjMoveStringLiteralDfa2_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0x8000000000000000L, active12, 0L);
       case 46:
-         if ((active11 & 0x10000000000000L) != 0L)
-            return jjStopAtPos(1, 756);
+         if ((active11 & 0x8000000000000L) != 0L)
+            return jjStopAtPos(1, 755);
          break;
       case 47:
-         if ((active12 & 0x2L) != 0L)
-            return jjStopAtPos(1, 769);
+         if ((active12 & 0x1L) != 0L)
+            return jjStopAtPos(1, 768);
          break;
       case 58:
-         if ((active11 & 0x400000000000000L) != 0L)
-            return jjStopAtPos(1, 762);
+         if ((active11 & 0x200000000000000L) != 0L)
+            return jjStopAtPos(1, 761);
          break;
       case 61:
-         if ((active11 & 0x20000000000L) != 0L)
+         if ((active11 & 0x10000000000L) != 0L)
+            return jjStopAtPos(1, 744);
+         else if ((active11 & 0x20000000000L) != 0L)
             return jjStopAtPos(1, 745);
-         else if ((active11 & 0x40000000000L) != 0L)
-            return jjStopAtPos(1, 746);
-         else if ((active11 & 0x100000000000L) != 0L)
-            return jjStopAtPos(1, 748);
+         else if ((active11 & 0x80000000000L) != 0L)
+            return jjStopAtPos(1, 747);
          break;
       case 62:
-         if ((active11 & 0x80000000000L) != 0L)
-            return jjStopAtPos(1, 747);
-         else if ((active11 & 0x8000000000000L) != 0L)
-            return jjStopAtPos(1, 755);
+         if ((active11 & 0x40000000000L) != 0L)
+            return jjStopAtPos(1, 746);
+         else if ((active11 & 0x4000000000000L) != 0L)
+            return jjStopAtPos(1, 754);
          break;
       case 65:
       case 97:
@@ -857,7 +857,7 @@ private final int jjMoveStringLiteralDfa1_1(long active0, long active1, long act
          return jjMoveStringLiteralDfa2_1(active0, 0x700000000L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x8000006000000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0xffcL, active10, 0x100000000e0000L, active11, 0L, active12, 0L);
       case 85:
       case 117:
-         return jjMoveStringLiteralDfa2_1(active0, 0x800000000L, active1, 0xfff8000000000000L, active2, 0x7L, active3, 0x70000000L, active4, 0L, active5, 0x1f8000300000L, active6, 0x3000000000007L, active7, 0x400000000L, active8, 0L, active9, 0x4000000000ff000L, active10, 0L, active11, 0x180L, active12, 0L);
+         return jjMoveStringLiteralDfa2_1(active0, 0x800000000L, active1, 0xfff8000000000000L, active2, 0x7L, active3, 0x70000000L, active4, 0L, active5, 0x1f8000300000L, active6, 0x3000000000007L, active7, 0x400000000L, active8, 0L, active9, 0x4000000000ff000L, active10, 0L, active11, 0x80L, active12, 0L);
       case 86:
       case 118:
          return jjMoveStringLiteralDfa2_1(active0, 0x1000000000L, active1, 0L, active2, 0x2000000000000000L, active3, 0L, active4, 0L, active5, 0L, active6, 0x78L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
@@ -870,8 +870,8 @@ private final int jjMoveStringLiteralDfa1_1(long active0, long active1, long act
             return jjStartNfaWithStates_1(1, 50, 89);
          return jjMoveStringLiteralDfa2_1(active0, 0L, active1, 0L, active2, 0xe00000000008L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0x800000000f00000L, active10, 0L, active11, 0L, active12, 0L);
       case 124:
-         if ((active11 & 0x4000000000000L) != 0L)
-            return jjStopAtPos(1, 754);
+         if ((active11 & 0x2000000000000L) != 0L)
+            return jjStopAtPos(1, 753);
          break;
       default :
          break;
@@ -884,23 +884,23 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
       return jjStartNfa_1(0, old0, old1, old2, old3, old4, old5, old6, old7, old8, old9, old10, old11, old12); 
    try { curChar = input_stream.readChar(); }
    catch(java.io.IOException e) {
-      jjStopStringLiteralDfa_1(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, active12);
+      jjStopStringLiteralDfa_1(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, 0L);
       return 2;
    }
    switch(curChar)
    {
       case 43:
-         if ((active12 & 0x1L) != 0L)
-            return jjStopAtPos(2, 768);
+         if ((active11 & 0x8000000000000000L) != 0L)
+            return jjStopAtPos(2, 767);
          break;
       case 65:
       case 97:
          if ((active0 & 0x100L) != 0L)
             return jjStartNfaWithStates_1(2, 8, 89);
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0x9bffL, active2, 0x400000800L, active3, 0x18003000000L, active4, 0x30000000000L, active5, 0x1L, active6, 0x72000180000000L, active7, 0x203000000000L, active8, 0x120L, active9, 0x1ff800000003cL, active10, 0x320000000400L, active11, 0x4L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0x9bffL, active2, 0x400000800L, active3, 0x18003000000L, active4, 0x30000000000L, active5, 0x1L, active6, 0x72000180000000L, active7, 0x203000000000L, active8, 0x120L, active9, 0x1ff800000003cL, active10, 0x320000000400L, active11, 0x4L);
       case 66:
       case 98:
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0x8000000010000L, active2, 0L, active3, 0L, active4, 0x200200000000L, active5, 0L, active6, 0x1000000000000L, active7, 0L, active8, 0L, active9, 0x200000000703f000L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0x8000000010000L, active2, 0L, active3, 0L, active4, 0x200200000000L, active5, 0L, active6, 0x1000000000000L, active7, 0L, active8, 0L, active9, 0x200000000703f000L, active10, 0L, active11, 0L);
       case 67:
       case 99:
          if ((active0 & 0x4000000L) != 0L)
@@ -910,7 +910,7 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             jjmatchedKind = 140;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0xd01100000000e008L, active3, 0x3800000000000003L, active4, 0x3c000000000000L, active5, 0x400000000080L, active6, 0x80000000000000L, active7, 0x3c00000000000L, active8, 0L, active9, 0xc000000000040000L, active10, 0x4c4000000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0xd01100000000e008L, active3, 0x3800000000000003L, active4, 0x3c000000000000L, active5, 0x400000000080L, active6, 0x80000000000000L, active7, 0x3c00000000000L, active8, 0L, active9, 0xc000000000040000L, active10, 0x4c4000000000000L, active11, 0L);
       case 68:
       case 100:
          if ((active0 & 0x200L) != 0L)
@@ -931,14 +931,14 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             return jjStartNfaWithStates_1(2, 371, 89);
          else if ((active6 & 0x80L) != 0L)
             return jjStartNfaWithStates_1(2, 391, 89);
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0x1c0000000000000L, active3, 0x4000000000000000L, active4, 0L, active5, 0x7000000000018000L, active6, 0L, active7, 0L, active8, 0L, active9, 0xc0L, active10, 0x800000800000081L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0x1c0000000000000L, active3, 0x4000000000000000L, active4, 0L, active5, 0x7000000000018000L, active6, 0L, active7, 0L, active8, 0L, active9, 0xc0L, active10, 0x800000800000081L, active11, 0L);
       case 69:
       case 101:
          if ((active0 & 0x100000L) != 0L)
             return jjStartNfaWithStates_1(2, 20, 89);
          else if ((active5 & 0x40000000000000L) != 0L)
             return jjStartNfaWithStates_1(2, 374, 89);
-         return jjMoveStringLiteralDfa3_1(active0, 0x2000008000000L, active1, 0x2000000000400L, active2, 0x2002000000000000L, active3, 0x840000400000cL, active4, 0L, active5, 0x100000000000000L, active6, 0xfc000000078L, active7, 0x4000000000000L, active8, 0xe00L, active9, 0x2000030000000L, active10, 0x7c000f800L, active11, 0x80L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x2000008000000L, active1, 0x2000000000400L, active2, 0x2002000000000000L, active3, 0x840000400000cL, active4, 0L, active5, 0x100000000000000L, active6, 0xfc000000078L, active7, 0x4000000000000L, active8, 0xe00L, active9, 0x2000030000000L, active10, 0x7c000f800L, active11, 0x80L);
       case 70:
       case 102:
          if ((active6 & 0x100000000000000L) != 0L)
@@ -946,28 +946,28 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             jjmatchedKind = 440;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_1(active0, 0x2000000000L, active1, 0L, active2, 0x7f0000L, active3, 0L, active4, 0x40000000000L, active5, 0x4000000000000L, active6, 0x600000000000000L, active7, 0L, active8, 0L, active9, 0L, active10, 0x80000000e0000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x2000000000L, active1, 0L, active2, 0x7f0000L, active3, 0L, active4, 0x40000000000L, active5, 0x4000000000000L, active6, 0x600000000000000L, active7, 0L, active8, 0L, active9, 0L, active10, 0x80000000e0000L, active11, 0L);
       case 71:
       case 103:
          if ((active0 & 0x1000000000L) != 0L)
             return jjStartNfaWithStates_1(2, 36, 89);
          else if ((active4 & 0x400000000L) != 0L)
             return jjStartNfaWithStates_1(2, 290, 89);
-         return jjMoveStringLiteralDfa3_1(active0, 0x9c000000000L, active1, 0L, active2, 0x800000L, active3, 0L, active4, 0L, active5, 0L, active6, 0xf800000000000000L, active7, 0x20000fL, active8, 0L, active9, 0L, active10, 0x4000000000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x9c000000000L, active1, 0L, active2, 0x800000L, active3, 0L, active4, 0L, active5, 0L, active6, 0xf800000000000000L, active7, 0x20000fL, active8, 0L, active9, 0L, active10, 0x4000000000000000L, active11, 0L);
       case 72:
       case 104:
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x8000002008000000L, active6, 0L, active7, 0xc000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x8000002008000000L, active6, 0L, active7, 0xc000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 73:
       case 105:
          if ((active6 & 0x200000000L) != 0L)
             return jjStartNfaWithStates_1(2, 417, 89);
-         return jjMoveStringLiteralDfa3_1(active0, 0x6000000000000000L, active1, 0L, active2, 0L, active3, 0x8020000000000010L, active4, 0x100001L, active5, 0x10004000000000L, active6, 0x700000000000L, active7, 0x400000L, active8, 0x10L, active9, 0xfc000000000000L, active10, 0x20040000010006L, active11, 0x100L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x6000000000000000L, active1, 0L, active2, 0L, active3, 0x8020000000000010L, active4, 0x100001L, active5, 0x10004000000000L, active6, 0x700000000000L, active7, 0x400000L, active8, 0x10L, active9, 0xfc000000000000L, active10, 0x20040000010006L, active11, 0L);
       case 74:
       case 106:
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x200000000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x200000000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 75:
       case 107:
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0xc00000000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0x8L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0xc00000000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0x8L, active11, 0L);
       case 76:
       case 108:
          if ((active0 & 0x1000L) != 0L)
@@ -982,12 +982,12 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
          }
          else if ((active10 & 0x80000000000L) != 0L)
             return jjStartNfaWithStates_1(2, 683, 89);
-         return jjMoveStringLiteralDfa3_1(active0, 0x30000000006000L, active1, 0x1fe0000L, active2, 0x1000000L, active3, 0x800010041400L, active4, 0L, active5, 0x80078010100300L, active6, 0L, active7, 0x18000003800030L, active8, 0xffffffffffffe000L, active9, 0x1L, active10, 0xa200000000700000L, active11, 0x2L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x30000000006000L, active1, 0x1fe0000L, active2, 0x1000000L, active3, 0x800010041400L, active4, 0L, active5, 0x80078010100300L, active6, 0L, active7, 0x18000003800030L, active8, 0xffffffffffffe000L, active9, 0x1L, active10, 0xa200000000700000L, active11, 0x2L);
       case 77:
       case 109:
          if ((active9 & 0x80000L) != 0L)
             return jjStartNfaWithStates_1(2, 595, 89);
-         return jjMoveStringLiteralDfa3_1(active0, 0x400L, active1, 0x1000001e000000L, active2, 0x8000000000L, active3, 0xc0000000000000L, active4, 0x1000000000000L, active5, 0x180000e00002L, active6, 0L, active7, 0L, active8, 0x46L, active9, 0x400000fc8100000L, active10, 0x2800000000000L, active11, 0x20L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x400L, active1, 0x1000001e000000L, active2, 0x8000000000L, active3, 0xc0000000000000L, active4, 0x1000000000000L, active5, 0x180000e00002L, active6, 0L, active7, 0L, active8, 0x46L, active9, 0x400000fc8100000L, active10, 0x2800000000000L, active11, 0x20L);
       case 78:
       case 110:
          if ((active5 & 0x400L) != 0L)
@@ -995,10 +995,10 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             jjmatchedKind = 330;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_1(active0, 0x8000100000000000L, active1, 0xfffe0000000L, active2, 0xe00002000000L, active3, 0x10000320002000L, active4, 0x80800000002L, active5, 0x201063800L, active6, 0xc000000000000L, active7, 0x8020000400000000L, active8, 0L, active9, 0x1000000000L, active10, 0x401000000060L, active11, 0x8L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x8000100000000000L, active1, 0xfffe0000000L, active2, 0xe00002000000L, active3, 0x10000320002000L, active4, 0x80800000002L, active5, 0x201063800L, active6, 0xc000000000000L, active7, 0x8020000400000000L, active8, 0L, active9, 0x1000000000L, active10, 0x401000000060L, active11, 0x8L);
       case 79:
       case 111:
-         return jjMoveStringLiteralDfa3_1(active0, 0xc00100000000L, active1, 0x4000000006000L, active2, 0x200100000000000L, active3, 0x1e1408030000L, active4, 0x1fe70004L, active5, 0L, active6, 0x800000000000L, active7, 0xf0000000000L, active8, 0x1L, active9, 0L, active10, 0x10000000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0xc00100000000L, active1, 0x4000000006000L, active2, 0x200100000000000L, active3, 0x1e1408030000L, active4, 0x1fe70004L, active5, 0L, active6, 0x800000000000L, active7, 0xf0000000000L, active8, 0x1L, active9, 0L, active10, 0x10000000000000L, active11, 0L);
       case 80:
       case 112:
          if ((active3 & 0x20L) != 0L)
@@ -1010,10 +1010,10 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             return jjStartNfaWithStates_1(2, 240, 89);
          else if ((active4 & 0x100000000000000L) != 0L)
             return jjStartNfaWithStates_1(2, 312, 89);
-         return jjMoveStringLiteralDfa3_1(active0, 0x80000L, active1, 0L, active2, 0x8000004000000L, active3, 0x300000000000040L, active4, 0x8L, active5, 0L, active6, 0L, active7, 0x400000000000c0L, active8, 0L, active9, 0x800004000000000L, active10, 0x110L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x80000L, active1, 0L, active2, 0x8000004000000L, active3, 0x300000000000040L, active4, 0x8L, active5, 0L, active6, 0L, active7, 0x400000000000c0L, active8, 0L, active9, 0x800004000000000L, active10, 0x110L, active11, 0L);
       case 81:
       case 113:
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0x80000000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0x80000000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 82:
       case 114:
          if ((active3 & 0x80000L) != 0L)
@@ -1026,7 +1026,7 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             jjmatchedKind = 407;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_1(active0, 0x40020001e00000L, active1, 0xffe0300000000000L, active2, 0x800000018000007L, active3, 0x70c000L, active4, 0x1000000000L, active5, 0xc00080004L, active6, 0x43f01ff00L, active7, 0x700100000000000L, active8, 0L, active9, 0x702L, active10, 0x100001003f800000L, active11, 0x10L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x40020001e00000L, active1, 0xffe0300000000000L, active2, 0x800000018000007L, active3, 0x70c000L, active4, 0x1000000000L, active5, 0xc00080004L, active6, 0x43f01ff00L, active7, 0x700100000000000L, active8, 0L, active9, 0x702L, active10, 0x100001003f800000L, active11, 0x10L);
       case 83:
       case 115:
          if ((active0 & 0x10L) != 0L)
@@ -1034,7 +1034,7 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             jjmatchedKind = 4;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_1(active0, 0x780000030000060L, active1, 0L, active2, 0x40079e0000000L, active3, 0x40000000L, active4, 0x60000000f0L, active5, 0x20000038L, active6, 0x18001e0000L, active7, 0x1800000000001f00L, active8, 0L, active9, 0x1000000000e00000L, active10, 0x200L, active11, 0x1L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x780000030000060L, active1, 0L, active2, 0x40079e0000000L, active3, 0x40000000L, active4, 0x60000000f0L, active5, 0x20000038L, active6, 0x18001e0000L, active7, 0x1800000000001f00L, active8, 0L, active9, 0x1000000000e00000L, active10, 0x200L, active11, 0x1L);
       case 84:
       case 116:
          if ((active0 & 0x200000000000L) != 0L)
@@ -1060,13 +1060,13 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             jjmatchedKind = 509;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_1(active0, 0x1801040e00008880L, active1, 0L, active2, 0x2000001f0L, active3, 0x4000000b80L, active4, 0x3e00008000003e00L, active5, 0x601800006000040L, active6, 0x600006L, active7, 0x40000000000fe000L, active8, 0L, active9, 0L, active10, 0x10000e000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x1801040e00008880L, active1, 0L, active2, 0x2000001f0L, active3, 0x4000000b80L, active4, 0x3e00008000003e00L, active5, 0x601800006000040L, active6, 0x600006L, active7, 0x40000000000fe000L, active8, 0L, active9, 0L, active10, 0x10000e000000000L, active11, 0L);
       case 85:
       case 117:
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0x400000000000L, active2, 0x400020000000000L, active3, 0x6000000800000L, active4, 0L, active5, 0L, active6, 0L, active7, 0x3c000000L, active8, 0x80L, active9, 0x300000000000000L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0x400000000000L, active2, 0x400020000000000L, active3, 0x6000000800000L, active4, 0L, active5, 0L, active6, 0L, active7, 0x3c000000L, active8, 0x80L, active9, 0x300000000000000L, active10, 0L, active11, 0L);
       case 86:
       case 118:
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0x1800000000000L, active2, 0L, active3, 0x200000000000L, active4, 0x100000084000L, active5, 0L, active6, 0x40000000L, active7, 0x800100000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0x1800000000000L, active2, 0L, active3, 0x200000000000L, active4, 0x100000084000L, active5, 0L, active6, 0x40000000L, active7, 0x800100000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 87:
       case 119:
          if ((active2 & 0x40000000000L) != 0L)
@@ -1078,7 +1078,7 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             jjmatchedKind = 478;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_1(active0, 0x10000L, active1, 0L, active2, 0L, active3, 0L, active4, 0x40000000000000L, active5, 0L, active6, 0x2000000000L, active7, 0x380000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x10000L, active1, 0L, active2, 0L, active3, 0L, active4, 0x40000000000000L, active5, 0L, active6, 0x2000000000L, active7, 0x380000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 88:
       case 120:
          if ((active4 & 0x4000000000000000L) != 0L)
@@ -1086,7 +1086,7 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             jjmatchedKind = 318;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0x8000000000000000L, active5, 0x80000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0x8000000000000000L, active5, 0x80000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 89:
       case 121:
          if ((active0 & 0x40000L) != 0L)
@@ -1103,19 +1103,19 @@ private final int jjMoveStringLiteralDfa2_1(long old0, long active0, long old1,
             jjmatchedKind = 286;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_1(active0, 0x40000000L, active1, 0L, active2, 0x400L, active3, 0L, active4, 0x180000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0x800L, active10, 0L, active11, 0x40L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0x40000000L, active1, 0L, active2, 0x400L, active3, 0L, active4, 0x180000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0x800L, active10, 0L, active11, 0x40L);
       case 90:
       case 122:
-         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0x8L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_1(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0x8L, active9, 0L, active10, 0L, active11, 0L);
       default :
          break;
    }
-   return jjStartNfa_1(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, active12);
+   return jjStartNfa_1(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, 0L);
 }
-private final int jjMoveStringLiteralDfa3_1(long old0, long active0, long old1, long active1, long old2, long active2, long old3, long active3, long old4, long active4, long old5, long active5, long old6, long active6, long old7, long active7, long old8, long active8, long old9, long active9, long old10, long active10, long old11, long active11, long old12, long active12)
+private final int jjMoveStringLiteralDfa3_1(long old0, long active0, long old1, long active1, long old2, long active2, long old3, long active3, long old4, long active4, long old5, long active5, long old6, long active6, long old7, long active7, long old8, long active8, long old9, long active9, long old10, long active10, long old11, long active11)
 {
-   if (((active0 &= old0) | (active1 &= old1) | (active2 &= old2) | (active3 &= old3) | (active4 &= old4) | (active5 &= old5) | (active6 &= old6) | (active7 &= old7) | (active8 &= old8) | (active9 &= old9) | (active10 &= old10) | (active11 &= old11) | (active12 &= old12)) == 0L)
-      return jjStartNfa_1(1, old0, old1, old2, old3, old4, old5, old6, old7, old8, old9, old10, old11, old12); 
+   if (((active0 &= old0) | (active1 &= old1) | (active2 &= old2) | (active3 &= old3) | (active4 &= old4) | (active5 &= old5) | (active6 &= old6) | (active7 &= old7) | (active8 &= old8) | (active9 &= old9) | (active10 &= old10) | (active11 &= old11)) == 0L)
+      return jjStartNfa_1(1, old0, old1, old2, old3, old4, old5, old6, old7, old8, old9, old10, old11, 0L);
    try { curChar = input_stream.readChar(); }
    catch(java.io.IOException e) {
       jjStopStringLiteralDfa_1(2, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, 0L);
@@ -1179,8 +1179,6 @@ private final int jjMoveStringLiteralDfa3_1(long old0, long active0, long old1,
             jjmatchedKind = 436;
             jjmatchedPos = 3;
          }
-         else if ((active11 & 0x100L) != 0L)
-            return jjStartNfaWithStates_1(3, 712, 89);
          return jjMoveStringLiteralDfa4_1(active0, 0x40000000000000L, active1, 0xe0000000L, active2, 0L, active3, 0L, active4, 0x20000010000L, active5, 0L, active6, 0x20000000000000L, active7, 0L, active8, 0L, active9, 0xc0L, active10, 0x1000000000L, active11, 0L);
       case 69:
       case 101:
@@ -4497,14 +4495,14 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x3ff001000000000L & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (curChar == 36)
                      jjCheckNAdd(31);
                   break;
-               case 86:
+               case 88:
                   if ((0x7ff601000000000L & l) != 0L)
                      jjCheckNAddTwoStates(29, 30);
                   else if (curChar == 39)
@@ -4513,14 +4511,14 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x3ff001000000000L & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (curChar == 36)
                      jjCheckNAdd(31);
                   break;
-               case 87:
+               case 86:
                   if ((0xfffffffbffffdbffL & l) != 0L)
                      jjCheckNAddStates(6, 8);
                   else if (curChar == 34)
@@ -4529,8 +4527,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 84:
                   if (curChar == 47)
                   {
-                     if (kind > 772)
-                        kind = 772;
+                     if (kind > 771)
+                        kind = 771;
                      jjCheckNAddStates(9, 11);
                   }
                   else if (curChar == 42)
@@ -4547,8 +4545,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 59;
                   if ((0x3ff001000000000L & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (curChar == 36)
@@ -4559,15 +4557,15 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjCheckNAddStates(12, 14);
                   else if (curChar == 39)
                   {
-                     if (kind > 720)
-                        kind = 720;
+                     if (kind > 719)
+                        kind = 719;
                   }
                   if ((0xfc00f7faffffc9ffL & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 56;
                   if (curChar == 39)
                      jjstateSet[jjnewStateCnt++] = 52;
                   break;
-               case 88:
+               case 87:
                   if (curChar == 32)
                      jjCheckNAddTwoStates(78, 79);
                   if (curChar == 32)
@@ -4586,8 +4584,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 90:
                   if ((0x3ff000000000000L & l) != 0L)
                   {
-                     if (kind > 715)
-                        kind = 715;
+                     if (kind > 714)
+                        kind = 714;
                      jjCheckNAdd(49);
                   }
                   if ((0x3ff000000000000L & l) != 0L)
@@ -4602,8 +4600,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x3ff001000000000L & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (curChar == 36)
@@ -4620,8 +4618,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjCheckNAddTwoStates(48, 49);
                   else if (curChar == 7)
                   {
-                     if (kind > 786)
-                        kind = 786;
+                     if (kind > 785)
+                        kind = 785;
                   }
                   else if (curChar == 34)
                      jjCheckNAddTwoStates(22, 24);
@@ -4629,14 +4627,14 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 15;
                   if ((0x3ff000000000000L & l) != 0L)
                   {
-                     if (kind > 713)
-                        kind = 713;
+                     if (kind > 712)
+                        kind = 712;
                      jjCheckNAddStates(24, 30);
                   }
                   else if (curChar == 36)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   break;
@@ -4653,8 +4651,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 3;
                   break;
                case 5:
-                  if (curChar == 39 && kind > 719)
-                     kind = 719;
+                  if (curChar == 39 && kind > 718)
+                     kind = 718;
                   break;
                case 7:
                   if ((0x3ff000000000000L & l) != 0L)
@@ -4678,30 +4676,30 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 11;
                   break;
                case 13:
-                  if (curChar == 39 && kind > 721)
-                     kind = 721;
+                  if (curChar == 39 && kind > 720)
+                     kind = 720;
                   break;
                case 15:
                   if (curChar != 45)
                      break;
-                  if (kind > 772)
-                     kind = 772;
+                  if (kind > 771)
+                     kind = 771;
                   jjCheckNAddStates(9, 11);
                   break;
                case 16:
                   if ((0xffffffffffffdbffL & l) == 0L)
                      break;
-                  if (kind > 772)
-                     kind = 772;
+                  if (kind > 771)
+                     kind = 771;
                   jjCheckNAddStates(9, 11);
                   break;
                case 17:
-                  if ((0x2400L & l) != 0L && kind > 772)
-                     kind = 772;
+                  if ((0x2400L & l) != 0L && kind > 771)
+                     kind = 771;
                   break;
                case 18:
-                  if (curChar == 10 && kind > 772)
-                     kind = 772;
+                  if (curChar == 10 && kind > 771)
+                     kind = 771;
                   break;
                case 19:
                   if (curChar == 13)
@@ -4728,21 +4726,21 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 23;
                   break;
                case 25:
-                  if (curChar == 34 && kind > 777)
-                     kind = 777;
+                  if (curChar == 34 && kind > 776)
+                     kind = 776;
                   break;
                case 26:
                   if (curChar != 36)
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 27:
                   if ((0x3ff001000000000L & l) == 0L)
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 28:
@@ -4760,8 +4758,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 31:
                   if (curChar != 36)
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjCheckNAddTwoStates(31, 32);
                   break;
                case 32:
@@ -4771,26 +4769,26 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 33:
                   if ((0x3ff001000000000L & l) == 0L)
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjCheckNAdd(33);
                   break;
                case 34:
-                  if (curChar == 7 && kind > 786)
-                     kind = 786;
+                  if (curChar == 7 && kind > 785)
+                     kind = 785;
                   break;
                case 35:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 713)
-                     kind = 713;
+                  if (kind > 712)
+                     kind = 712;
                   jjCheckNAddStates(24, 30);
                   break;
                case 36:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 713)
-                     kind = 713;
+                  if (kind > 712)
+                     kind = 712;
                   jjCheckNAdd(36);
                   break;
                case 37:
@@ -4804,8 +4802,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 40:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 714)
-                     kind = 714;
+                  if (kind > 713)
+                     kind = 713;
                   jjCheckNAdd(40);
                   break;
                case 41:
@@ -4819,22 +4817,22 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 43:
                   if (curChar != 46)
                      break;
-                  if (kind > 715)
-                     kind = 715;
+                  if (kind > 714)
+                     kind = 714;
                   jjCheckNAdd(44);
                   break;
                case 44:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 715)
-                     kind = 715;
+                  if (kind > 714)
+                     kind = 714;
                   jjCheckNAdd(44);
                   break;
                case 45:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 715)
-                     kind = 715;
+                  if (kind > 714)
+                     kind = 714;
                   jjCheckNAddStates(31, 33);
                   break;
                case 46:
@@ -4852,8 +4850,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 49:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 715)
-                     kind = 715;
+                  if (kind > 714)
+                     kind = 714;
                   jjCheckNAdd(49);
                   break;
                case 50:
@@ -4873,12 +4871,12 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 52;
                   break;
                case 54:
-                  if (curChar == 39 && kind > 720)
-                     kind = 720;
+                  if (curChar == 39 && kind > 719)
+                     kind = 719;
                   break;
                case 56:
-                  if (curChar == 39 && kind > 726)
-                     kind = 726;
+                  if (curChar == 39 && kind > 725)
+                     kind = 725;
                   break;
                case 59:
                case 61:
@@ -4894,8 +4892,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 61;
                   break;
                case 63:
-                  if (curChar == 39 && kind > 722)
-                     kind = 722;
+                  if (curChar == 39 && kind > 721)
+                     kind = 721;
                   break;
                case 64:
                   if (curChar == 38)
@@ -4918,8 +4916,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 67;
                   break;
                case 69:
-                  if (curChar == 34 && kind > 783)
-                     kind = 783;
+                  if (curChar == 34 && kind > 782)
+                     kind = 782;
                   break;
                case 71:
                   if (curChar == 32)
@@ -4946,14 +4944,14 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 83;
                   break;
                case 83:
-                  if ((0xffff7fffffffffffL & l) != 0L && kind > 770)
-                     kind = 770;
+                  if ((0xffff7fffffffffffL & l) != 0L && kind > 769)
+                     kind = 769;
                   break;
                case 85:
                   if (curChar != 47)
                      break;
-                  if (kind > 772)
-                     kind = 772;
+                  if (kind > 771)
+                     kind = 771;
                   jjCheckNAddStates(9, 11);
                   break;
                default : break;
@@ -4974,24 +4972,24 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   break;
-               case 86:
+               case 88:
                   if ((0x7fffffe87fffffeL & l) != 0L)
                      jjCheckNAddTwoStates(29, 30);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                      jjCheckNAddStates(0, 2);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   break;
-               case 87:
+               case 86:
                case 22:
                   jjCheckNAddStates(6, 8);
                   break;
@@ -5002,8 +5000,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   break;
@@ -5012,20 +5010,20 @@ private final int jjMoveNfa_1(int startState, int curPos)
                   if ((0xffffff81ffffff81L & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 56;
                   break;
-               case 88:
+               case 87:
                   if ((0x4000000040L & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 80;
                   else if ((0x10000000100000L & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 77;
                   else if ((0x1000000010L & l) != 0L)
                   {
-                     if (kind > 729)
-                        kind = 729;
+                     if (kind > 728)
+                        kind = 728;
                   }
                   if ((0x10000000100000L & l) != 0L)
                   {
-                     if (kind > 730)
-                        kind = 730;
+                     if (kind > 729)
+                        kind = 729;
                   }
                   break;
                case 91:
@@ -5040,8 +5038,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   break;
@@ -5052,8 +5050,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjAddStates(44, 51);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if ((0x20000000200000L & l) != 0L)
@@ -5088,22 +5086,22 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjCheckNAdd(9);
                   break;
                case 16:
-                  if (kind > 772)
-                     kind = 772;
+                  if (kind > 771)
+                     kind = 771;
                   jjAddStates(9, 11);
                   break;
                case 26:
                   if ((0x7fffffe87fffffeL & l) == 0L)
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 27:
                   if ((0x7fffffe87fffffeL & l) == 0L)
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 28:
@@ -5113,15 +5111,15 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 31:
                   if ((0x7fffffe87fffffeL & l) == 0L)
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjAddStates(54, 55);
                   break;
                case 33:
                   if ((0x7fffffe87fffffeL & l) == 0L)
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjstateSet[jjnewStateCnt++] = 33;
                   break;
                case 38:
@@ -5146,32 +5144,32 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjAddStates(44, 51);
                   break;
                case 72:
-                  if ((0x1000000010L & l) != 0L && kind > 729)
-                     kind = 729;
+                  if ((0x1000000010L & l) != 0L && kind > 728)
+                     kind = 728;
                   break;
                case 74:
-                  if ((0x10000000100000L & l) != 0L && kind > 730)
-                     kind = 730;
+                  if ((0x10000000100000L & l) != 0L && kind > 729)
+                     kind = 729;
                   break;
                case 76:
                   if ((0x10000000100000L & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 77;
                   break;
                case 77:
-                  if ((0x8000000080000L & l) != 0L && kind > 731)
-                     kind = 731;
+                  if ((0x8000000080000L & l) != 0L && kind > 730)
+                     kind = 730;
                   break;
                case 79:
                   if ((0x4000000040L & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 80;
                   break;
                case 80:
-                  if ((0x400000004000L & l) != 0L && kind > 732)
-                     kind = 732;
+                  if ((0x400000004000L & l) != 0L && kind > 731)
+                     kind = 731;
                   break;
                case 83:
-                  if (kind > 770)
-                     kind = 770;
+                  if (kind > 769)
+                     kind = 769;
                   break;
                default : break;
             }
@@ -5191,8 +5189,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 89:
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
@@ -5200,11 +5198,11 @@ private final int jjMoveNfa_1(int startState, int curPos)
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                      jjCheckNAddTwoStates(29, 30);
                   break;
-               case 86:
+               case 88:
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
@@ -5212,7 +5210,7 @@ private final int jjMoveNfa_1(int startState, int curPos)
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                      jjCheckNAddTwoStates(29, 30);
                   break;
-               case 87:
+               case 86:
                case 22:
                   if (jjCanMove_0(hiByte, i1, i2, l1, l2))
                      jjCheckNAddStates(6, 8);
@@ -5220,9 +5218,9 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 58:
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                   {
-                     if (kind > 781)
-                        kind = 781;
-                     jjCheckNAdd(27);
+                     if (kind > 780)
+                        kind = 780;
+                     jjCheckNAdd(27);
                   }
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                      jjCheckNAddStates(0, 2);
@@ -5243,8 +5241,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 1:
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
@@ -5255,8 +5253,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 0:
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
@@ -5273,22 +5271,22 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 16:
                   if (!jjCanMove_0(hiByte, i1, i2, l1, l2))
                      break;
-                  if (kind > 772)
-                     kind = 772;
+                  if (kind > 771)
+                     kind = 771;
                   jjAddStates(9, 11);
                   break;
                case 26:
                   if (!jjCanMove_1(hiByte, i1, i2, l1, l2))
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 27:
                   if (!jjCanMove_1(hiByte, i1, i2, l1, l2))
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 28:
@@ -5298,15 +5296,15 @@ private final int jjMoveNfa_1(int startState, int curPos)
                case 31:
                   if (!jjCanMove_1(hiByte, i1, i2, l1, l2))
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjAddStates(54, 55);
                   break;
                case 33:
                   if (!jjCanMove_1(hiByte, i1, i2, l1, l2))
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjstateSet[jjnewStateCnt++] = 33;
                   break;
                case 51:
@@ -5322,8 +5320,8 @@ private final int jjMoveNfa_1(int startState, int curPos)
                      jjAddStates(41, 43);
                   break;
                case 83:
-                  if (jjCanMove_0(hiByte, i1, i2, l1, l2) && kind > 770)
-                     kind = 770;
+                  if (jjCanMove_0(hiByte, i1, i2, l1, l2) && kind > 769)
+                     kind = 769;
                   break;
                default : break;
             }
@@ -5347,62 +5345,62 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
    switch (pos)
    {
       case 0:
-         if ((active5 & 0x1fffffc00000L) != 0L || (active10 & 0x8000000000000000L) != 0L)
-         {
-            jjmatchedKind = 781;
-            return 86;
-         }
          if ((active10 & 0x80000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             return 1;
          }
-         if ((active11 & 0x20000000L) != 0L)
+         if ((active11 & 0x10000000L) != 0L)
+            return 86;
+         if ((active11 & 0x40000000L) != 0L)
             return 87;
-         if ((active11 & 0x80000000L) != 0L)
-            return 88;
-         if ((active9 & 0xf000000000000000L) != 0L || (active10 & 0xfffffL) != 0L || (active11 & 0x100L) != 0L)
-         {
-            jjmatchedKind = 781;
-            return 58;
-         }
-         if ((active11 & 0x20000000000000L) != 0L)
+         if ((active11 & 0x10000000000000L) != 0L)
             return 55;
-         if ((active11 & 0x1000000000000L) != 0L || (active12 & 0x9L) != 0L)
+         if ((active11 & 0x8000800000000000L) != 0L || (active12 & 0x4L) != 0L)
             return 84;
-         if ((active0 & 0x7ffe000000000L) != 0L || (active2 & 0xfffffffffffffff0L) != 0L || (active3 & 0xffffe0007fffffffL) != 0L || (active4 & 0x7ffffe1fffffffL) != 0L || (active5 & 0xffffe00000000000L) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0x7fffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xfffffffffffffffL) != 0L || (active10 & 0x7f27f7fffff00000L) != 0L || (active11 & 0x95L) != 0L)
+         if ((active5 & 0x1fffffc00000L) != 0L || (active10 & 0x8000000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
-            return 89;
+            jjmatchedKind = 780;
+            return 88;
          }
-         if ((active11 & 0x400000000000L) != 0L)
+         if ((active11 & 0x200000000000L) != 0L)
             return 15;
-         if ((active0 & 0xfff8001ffffffff8L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfL) != 0L || (active3 & 0x1fff80000000L) != 0L || (active4 & 0xff800001e0000000L) != 0L || (active5 & 0x3fffffL) != 0L || (active7 & 0x8000000000000000L) != 0L || (active10 & 0xd8000000000000L) != 0L || (active11 & 0x20000000000006aL) != 0L)
+         if ((active0 & 0xfff8001ffffffff8L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfL) != 0L || (active3 & 0x1fff80000000L) != 0L || (active4 & 0xff800001e0000000L) != 0L || (active5 & 0x3fffffL) != 0L || (active7 & 0x8000000000000000L) != 0L || (active10 & 0xd8000000000000L) != 0L || (active11 & 0x10000000000006aL) != 0L)
+            return 89;
+         if ((active0 & 0x7ffe000000000L) != 0L || (active2 & 0xfffffffffffffff0L) != 0L || (active3 & 0xffffe0007fffffffL) != 0L || (active4 & 0x7ffffe1fffffffL) != 0L || (active5 & 0xffffe00000000000L) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0x7fffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xfffffffffffffffL) != 0L || (active10 & 0x7f27f7fffff00000L) != 0L || (active11 & 0x95L) != 0L)
+         {
+            jjmatchedKind = 780;
             return 89;
-         if ((active11 & 0x10000400000000L) != 0L)
+         }
+         if ((active9 & 0xf000000000000000L) != 0L || (active10 & 0xfffffL) != 0L)
+         {
+            jjmatchedKind = 780;
+            return 58;
+         }
+         if ((active11 & 0x8000200000000L) != 0L)
             return 90;
          return -1;
       case 1:
-         if ((active12 & 0x9L) != 0L)
-            return 82;
-         if ((active0 & 0x40007fe000000L) != 0L || (active3 & 0xfc00006000000000L) != 0L || (active4 & 0x200000007ffffL) != 0L || (active5 & 0x78e6001f00000000L) != 0L || (active9 & 0x6000000000L) != 0L || (active10 & 0xa811000000000000L) != 0L || (active11 & 0x40L) != 0L)
-            return 89;
-         if ((active0 & 0xfff3fff801fffff0L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xffffffffffffffffL) != 0L || (active3 & 0x3ffff9f7fffffffL) != 0L || (active4 & 0xff7dffffdff80000L) != 0L || (active5 & 0x8719ffe0ffffffffL) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0xffffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xffffff9fffffffffL) != 0L || (active10 & 0x57eeffffffffffffL) != 0L || (active11 & 0x1bfL) != 0L)
+         if ((active0 & 0xfff3fff801fffff0L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xffffffffffffffffL) != 0L || (active3 & 0x3ffff9f7fffffffL) != 0L || (active4 & 0xff7dffffdff80000L) != 0L || (active5 & 0x8719ffe0ffffffffL) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0xffffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xffffff9fffffffffL) != 0L || (active10 & 0x57eeffffffffffffL) != 0L || (active11 & 0xbfL) != 0L)
          {
             if (jjmatchedPos != 1)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 1;
             }
             return 89;
          }
+         if ((active11 & 0x8000000000000000L) != 0L || (active12 & 0x4L) != 0L)
+            return 82;
+         if ((active0 & 0x40007fe000000L) != 0L || (active3 & 0xfc00006000000000L) != 0L || (active4 & 0x200000007ffffL) != 0L || (active5 & 0x78e6001f00000000L) != 0L || (active9 & 0x6000000000L) != 0L || (active10 & 0xa811000000000000L) != 0L || (active11 & 0x40L) != 0L)
+            return 89;
          return -1;
       case 2:
-         if ((active0 & 0xfff3dfef79e98c80L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfe1ff2ffffff09ffL) != 0L || (active3 & 0xfbfeffd77f87ff9fL) != 0L || (active4 & 0x3e7dfffa1fff40ffL) != 0L || (active5 & 0xf795ffeebffe03ffL) != 0L || (active6 & 0xf8fffffdc07fff78L) != 0L || (active7 & 0x1ffffffc3fffffffL) != 0L || (active8 & 0xfffL) != 0L || (active9 & 0xffffffdffff7fffeL) != 0L || (active10 & 0xfffef7ffffffffffL) != 0L || (active11 & 0x1ffL) != 0L)
+         if ((active0 & 0xfff3dfef79e98c80L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfe1ff2ffffff09ffL) != 0L || (active3 & 0xfbfeffd77f87ff9fL) != 0L || (active4 & 0x3e7dfffa1fff40ffL) != 0L || (active5 & 0xf795ffeebffe03ffL) != 0L || (active6 & 0xf8fffffdc07fff78L) != 0L || (active7 & 0x1ffffffc3fffffffL) != 0L || (active8 & 0xfffL) != 0L || (active9 & 0xffffffdffff7fffeL) != 0L || (active10 & 0xfffef7ffffffffffL) != 0L || (active11 & 0xffL) != 0L)
          {
             if (jjmatchedPos != 2)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 2;
             }
             return 89;
@@ -5417,26 +5415,26 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 3)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 3;
             }
             return 89;
          }
-         if ((active0 & 0x6631400000000000L) != 0L || (active1 & 0x83000000023feL) != 0L || (active2 & 0x51001e00005f0L) != 0L || (active3 & 0x680401c00000cL) != 0L || (active4 & 0xc7601ff82000L) != 0L || (active5 & 0x190078280c80000L) != 0L || (active6 & 0x78080100300078L) != 0L || (active7 & 0x4014000200800000L) != 0L || (active8 & 0x59L) != 0L || (active9 & 0x9c0000ff0000002L) != 0L || (active10 & 0x100f1e3c002f800L) != 0L || (active11 & 0x106L) != 0L)
+         if ((active0 & 0x6631400000000000L) != 0L || (active1 & 0x83000000023feL) != 0L || (active2 & 0x51001e00005f0L) != 0L || (active3 & 0x680401c00000cL) != 0L || (active4 & 0xc7601ff82000L) != 0L || (active5 & 0x190078280c80000L) != 0L || (active6 & 0x78080100300078L) != 0L || (active7 & 0x4014000200800000L) != 0L || (active8 & 0x59L) != 0L || (active9 & 0x9c0000ff0000002L) != 0L || (active10 & 0x100f1e3c002f800L) != 0L || (active11 & 0x6L) != 0L)
             return 89;
          return -1;
       case 4:
+         if ((active2 & 0x40000000000000L) != 0L)
+            return 91;
          if ((active0 & 0xd9e29e2f780120e0L) != 0L || (active1 & 0xfff3affffffe9bfcL) != 0L || (active2 & 0xd592e2ffd3ffe9a7L) != 0L || (active3 & 0xfbd86017637413dbL) != 0L || (active4 & 0x8020aac99fc75ef1L) != 0L || (active5 & 0xc605fb2c2711bbfbL) != 0L || (active6 & 0xfe83d7dcbf4fff74L) != 0L || (active7 & 0x1febf0ddbf1ffeffL) != 0L || (active8 & 0xffffffffffffeea6L) != 0L || (active9 & 0xf6bdffdf88f7f7e1L) != 0L || (active10 & 0xf5de02da3f80f2fcL) != 0L || (active11 & 0x39L) != 0L)
          {
             if (jjmatchedPos != 4)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 4;
             }
             return 89;
          }
-         if ((active2 & 0x40000000000000L) != 0L)
-            return 91;
          if ((active0 & 0x1c001e8cc00L) != 0L || (active1 & 0x4400000014401L) != 0L || (active2 & 0x2a0800000c000008L) != 0L || (active3 & 0x241f800083ec00L) != 0L || (active4 & 0x3e5d10120000000eL) != 0L || (active5 & 0x3000044018a60004L) != 0L || (active6 & 0x24202040000002L) != 0L || (active7 & 0x80000f2000600100L) != 0L || (active8 & 0x100L) != 0L || (active9 & 0x200000700081cL) != 0L || (active10 & 0xa202404007d0503L) != 0L || (active11 & 0xc0L) != 0L)
             return 89;
          return -1;
@@ -5447,7 +5445,7 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 5)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 5;
             }
             return 89;
@@ -5458,17 +5456,17 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
       case 6:
          if ((active2 & 0x40000000000000L) != 0L)
             return 91;
+         if ((active0 & 0xd982840000000000L) != 0L || (active1 & 0xffe0080700420000L) != 0L || (active2 & 0x2e0001063c001L) != 0L || (active3 & 0x8800010100600249L) != 0L || (active4 & 0xc20228000024201L) != 0L || (active5 & 0x400100024001000L) != 0L || (active6 & 0x80021204af448020L) != 0L || (active7 & 0x180180043c0806a0L) != 0L || (active8 & 0x40002L) != 0L || (active9 & 0x103c001000000000L) != 0L || (active10 & 0x400402803c800038L) != 0L || (active11 & 0x30L) != 0L)
+            return 89;
          if ((active0 & 0x40038e79c02040L) != 0L || (active1 & 0x11a7f8f73c9bfcL) != 0L || (active2 & 0x8190007fc20c09a4L) != 0L || (active3 & 0x71c84c0623048102L) != 0L || (active4 & 0xb01880499fc41cd0L) != 0L || (active5 & 0x6000c12c0310a3b9L) != 0L || (active6 & 0x7e80c5d800097f50L) != 0L || (active7 & 0x5e20e898107c85fL) != 0L || (active8 & 0xfffffffffffbee20L) != 0L || (active9 & 0xe281ffcf8ed7b6d1L) != 0L || (active10 & 0xb7fa000a0340f000L) != 0L || (active11 & 0x9L) != 0L)
          {
             if (jjmatchedPos != 6)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 6;
             }
             return 89;
          }
-         if ((active0 & 0xd982840000000000L) != 0L || (active1 & 0xffe0080700420000L) != 0L || (active2 & 0x2e0001063c001L) != 0L || (active3 & 0x8800010100600249L) != 0L || (active4 & 0xc20228000024201L) != 0L || (active5 & 0x400100024001000L) != 0L || (active6 & 0x80021204af448020L) != 0L || (active7 & 0x180180043c0806a0L) != 0L || (active8 & 0x40002L) != 0L || (active9 & 0x103c001000000000L) != 0L || (active10 & 0x400402803c800038L) != 0L || (active11 & 0x30L) != 0L)
-            return 89;
          return -1;
       case 7:
          if ((active2 & 0x40000000000000L) != 0L)
@@ -5477,7 +5475,7 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 7)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 7;
             }
             return 89;
@@ -5486,37 +5484,37 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
             return 89;
          return -1;
       case 8:
-         if ((active0 & 0x20610400000L) != 0L || (active1 & 0x108000f03c03f8L) != 0L || (active2 & 0x8080000000000000L) != 0L || (active3 & 0x70c0400202040002L) != 0L || (active4 & 0x18000000040c01L) != 0L || (active5 & 0x2c00000000L) != 0L || (active6 & 0x1880818000017f00L) != 0L || (active7 & 0x60000880040000L) != 0L || (active8 & 0x300004000000L) != 0L || (active9 & 0x2000f00388130410L) != 0L || (active10 & 0x8410000002000000L) != 0L)
-            return 89;
          if ((active0 & 0x1040018869800000L) != 0L || (active1 & 0xffc121fe07001804L) != 0L || (active2 & 0x100c01782040985L) != 0L || (active3 & 0x100040001008000L) != 0L || (active4 & 0x300080409fc00090L) != 0L || (active5 & 0x4000c000010003b8L) != 0L || (active6 & 0x660040100e080040L) != 0L || (active7 & 0x15000e813803c041L) != 0L || (active8 & 0xfffacffffb13ac00L) != 0L || (active9 & 0xc0b90f4c06c0b2c1L) != 0L || (active10 & 0x23e800082000f000L) != 0L || (active11 & 0x8L) != 0L)
          {
             if (jjmatchedPos != 8)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 8;
             }
             return 89;
          }
+         if ((active0 & 0x20610400000L) != 0L || (active1 & 0x108000f03c03f8L) != 0L || (active2 & 0x8080000000000000L) != 0L || (active3 & 0x70c0400202040002L) != 0L || (active4 & 0x18000000040c01L) != 0L || (active5 & 0x2c00000000L) != 0L || (active6 & 0x1880818000017f00L) != 0L || (active7 & 0x60000880040000L) != 0L || (active8 & 0x300004000000L) != 0L || (active9 & 0x2000f00388130410L) != 0L || (active10 & 0x8410000002000000L) != 0L)
+            return 89;
          return -1;
       case 9:
-         if ((active0 & 0x468000000L) != 0L || (active1 & 0x100fe00000a00L) != 0L || (active2 & 0x1102040800L) != 0L || (active4 & 0x804098c00000L) != 0L || (active5 & 0x4000000801000200L) != 0L || (active6 & 0x2200400000000040L) != 0L || (active7 & 0x40100000041L) != 0L || (active8 & 0x2038000L) != 0L || (active9 & 0x80200002008240L) != 0L || (active10 & 0x280000020000000L) != 0L || (active11 & 0x8L) != 0L)
-            return 89;
          if ((active0 & 0x1040018801800000L) != 0L || (active1 & 0xffc02100c73811f4L) != 0L || (active2 & 0x100c00680000185L) != 0L || (active3 & 0x180040001008000L) != 0L || (active4 & 0x3010000007000890L) != 0L || (active5 & 0xc000000001b8L) != 0L || (active6 & 0x440000100e087e00L) != 0L || (active7 & 0x15000a803803c000L) != 0L || (active8 & 0xfffacffff9102c00L) != 0L || (active9 & 0xc0398f4f04c23081L) != 0L || (active10 & 0x216800080000f000L) != 0L)
          {
             if (jjmatchedPos != 9)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 9;
             }
             return 89;
          }
+         if ((active0 & 0x468000000L) != 0L || (active1 & 0x100fe00000a00L) != 0L || (active2 & 0x1102040800L) != 0L || (active4 & 0x804098c00000L) != 0L || (active5 & 0x4000000801000200L) != 0L || (active6 & 0x2200400000000040L) != 0L || (active7 & 0x40100000041L) != 0L || (active8 & 0x2038000L) != 0L || (active9 & 0x80200002008240L) != 0L || (active10 & 0x280000020000000L) != 0L || (active11 & 0x8L) != 0L)
+            return 89;
          return -1;
       case 10:
          if ((active0 & 0x1000010801800000L) != 0L || (active1 & 0xf7c02074863811f0L) != 0L || (active2 & 0x100c00200000181L) != 0L || (active3 & 0x100040000000000L) != 0L || (active4 & 0x3010000000800880L) != 0L || (active5 & 0xc00000000038L) != 0L || (active6 & 0x400000100e007e00L) != 0L || (active7 & 0x11000a003803c000L) != 0L || (active8 & 0xffca8ffff0002c00L) != 0L || (active9 & 0x8038804f00021000L) != 0L || (active10 & 0x2800080000f000L) != 0L)
          {
             if (jjmatchedPos != 10)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 10;
             }
             return 89;
@@ -5525,24 +5523,24 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
             return 89;
          return -1;
       case 11:
-         if ((active0 & 0x1000000000000000L) != 0L || (active1 & 0x6680000000001000L) != 0L || (active2 & 0x1L) != 0L || (active3 & 0x40000000000L) != 0L || (active4 & 0x1000000000000880L) != 0L || (active5 & 0x800000000020L) != 0L || (active6 & 0x8000000L) != 0L || (active7 & 0x1100080010000000L) != 0L || (active8 & 0x6082800000002800L) != 0L || (active9 & 0x10000100000000L) != 0L || (active10 & 0x8000800000000L) != 0L)
-            return 89;
          if ((active0 & 0x10801800000L) != 0L || (active1 & 0x91402074863801f0L) != 0L || (active2 & 0x100c00200000180L) != 0L || (active3 & 0x100000000000000L) != 0L || (active4 & 0x2010000004800000L) != 0L || (active5 & 0x400000000018L) != 0L || (active6 & 0x4000001006007e00L) != 0L || (active7 & 0x2002803c000L) != 0L || (active8 & 0x9f480ffff0000400L) != 0L || (active9 & 0x80288e4e00021000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
             if (jjmatchedPos != 11)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 11;
             }
             return 89;
          }
+         if ((active0 & 0x1000000000000000L) != 0L || (active1 & 0x6680000000001000L) != 0L || (active2 & 0x1L) != 0L || (active3 & 0x40000000000L) != 0L || (active4 & 0x1000000000000880L) != 0L || (active5 & 0x800000000020L) != 0L || (active6 & 0x8000000L) != 0L || (active7 & 0x1100080010000000L) != 0L || (active8 & 0x6082800000002800L) != 0L || (active9 & 0x10000100000000L) != 0L || (active10 & 0x8000800000000L) != 0L)
+            return 89;
          return -1;
       case 12:
          if ((active0 & 0x800000000L) != 0L || (active1 & 0x200000000000L) != 0L || (active2 & 0x100000200000000L) != 0L || (active4 & 0x800000L) != 0L || (active8 & 0x8408000000000400L) != 0L || (active9 & 0x8000000600000000L) != 0L)
             return 89;
          if ((active0 & 0x10001800000L) != 0L || (active1 & 0xd1400074863801f0L) != 0L || (active2 & 0xc00000000180L) != 0L || (active3 & 0x100000000000000L) != 0L || (active4 & 0x2010000004000000L) != 0L || (active5 & 0x400000000018L) != 0L || (active6 & 0x4000001006007e00L) != 0L || (active7 & 0x2002803c000L) != 0L || (active8 & 0x1b400ffff0000000L) != 0L || (active9 & 0x288e4800021000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 12;
             return 89;
          }
@@ -5550,7 +5548,7 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
       case 13:
          if ((active0 & 0x10001800000L) != 0L || (active1 & 0xc1400074862801f0L) != 0L || (active2 & 0xc00000000180L) != 0L || (active4 & 0x2000000000000000L) != 0L || (active5 & 0x400000000010L) != 0L || (active6 & 0x6007800L) != 0L || (active7 & 0x803c000L) != 0L || (active8 & 0x9400ffff0000000L) != 0L || (active9 & 0x88e4800021000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 13;
             return 89;
          }
@@ -5560,7 +5558,7 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
       case 14:
          if ((active0 & 0x1800000L) != 0L || (active1 & 0xc1000050862800f0L) != 0L || (active2 & 0xc00000000180L) != 0L || (active5 & 0x400000000010L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x34000L) != 0L || (active8 & 0x1400bfff0000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 14;
             return 89;
          }
@@ -5568,35 +5566,35 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
             return 89;
          return -1;
       case 15:
+         if ((active0 & 0x800000L) != 0L || (active1 & 0x86200010L) != 0L || (active2 & 0xc00000000000L) != 0L || (active8 & 0x800f0000000L) != 0L)
+            return 89;
          if ((active0 & 0x1000000L) != 0L || (active1 & 0xc1000050000800e0L) != 0L || (active2 & 0x180L) != 0L || (active5 & 0x400000000010L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x34000L) != 0L || (active8 & 0x14003ff00000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
             if (jjmatchedPos != 15)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 15;
             }
             return 89;
          }
-         if ((active0 & 0x800000L) != 0L || (active1 & 0x86200010L) != 0L || (active2 & 0xc00000000000L) != 0L || (active8 & 0x800f0000000L) != 0L)
-            return 89;
          return -1;
       case 16:
+         if ((active1 & 0x4000004000080000L) != 0L || (active5 & 0x400000000000L) != 0L || (active7 & 0x20000L) != 0L || (active8 & 0x38700000000L) != 0L)
+            return 89;
          if ((active0 & 0x1000000L) != 0L || (active1 & 0x81000010040000e0L) != 0L || (active2 & 0x800000000180L) != 0L || (active5 & 0x10L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x14000L) != 0L || (active8 & 0x1400078e0000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
             if (jjmatchedPos != 16)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 16;
             }
             return 89;
          }
-         if ((active1 & 0x4000004000080000L) != 0L || (active5 & 0x400000000000L) != 0L || (active7 & 0x20000L) != 0L || (active8 & 0x38700000000L) != 0L)
-            return 89;
          return -1;
       case 17:
          if ((active0 & 0x1000000L) != 0L || (active1 & 0x81000000040000a0L) != 0L || (active2 & 0x800000000180L) != 0L || (active5 & 0x10L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x14000L) != 0L || (active8 & 0x140015ee0000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 17;
             return 89;
          }
@@ -5608,7 +5606,7 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 18)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 18;
             }
             return 89;
@@ -5617,19 +5615,19 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
             return 89;
          return -1;
       case 19:
-         if ((active1 & 0x80L) != 0L || (active5 & 0x10L) != 0L || (active7 & 0x4000L) != 0L)
-            return 89;
          if ((active0 & 0x1000000L) != 0L || (active1 & 0x8100000004000020L) != 0L || (active2 & 0x800000000180L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x10000L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0xc0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 19;
             return 89;
          }
+         if ((active1 & 0x80L) != 0L || (active5 & 0x10L) != 0L || (active7 & 0x4000L) != 0L)
+            return 89;
          return -1;
       case 20:
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x180L) != 0L || (active6 & 0x7800L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0xc0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 20;
             return 89;
          }
@@ -5637,21 +5635,21 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
             return 89;
          return -1;
       case 21:
-         if ((active2 & 0x80L) != 0L || (active9 & 0x40000000000L) != 0L || (active10 & 0x6000L) != 0L)
-            return 89;
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x7800L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0x80000000000L) != 0L || (active10 & 0x20000000009000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 21;
             return 89;
          }
+         if ((active2 & 0x80L) != 0L || (active9 & 0x40000000000L) != 0L || (active10 & 0x6000L) != 0L)
+            return 89;
          return -1;
       case 22:
          if ((active6 & 0x2000L) != 0L)
             return 89;
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x5800L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0x80000000000L) != 0L || (active10 & 0x20000000009000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 22;
             return 89;
          }
@@ -5659,7 +5657,7 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
       case 23:
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x5800L) != 0L || (active8 & 0x116c0000000L) != 0L || (active10 & 0x20000000001000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 23;
             return 89;
          }
@@ -5669,7 +5667,7 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
       case 24:
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x1800L) != 0L || (active8 & 0x116c0000000L) != 0L || (active10 & 0x20000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 24;
             return 89;
          }
@@ -5677,29 +5675,29 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
             return 89;
          return -1;
       case 25:
+         if ((active6 & 0x1800L) != 0L || (active8 & 0x100c0000000L) != 0L || (active10 & 0x20000000000000L) != 0L)
+            return 89;
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active8 & 0x1600000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 25;
             return 89;
          }
-         if ((active6 & 0x1800L) != 0L || (active8 & 0x100c0000000L) != 0L || (active10 & 0x20000000000000L) != 0L)
-            return 89;
          return -1;
       case 26:
+         if ((active2 & 0x100L) != 0L || (active8 & 0x600000000L) != 0L)
+            return 89;
          if ((active1 & 0x8100000000000000L) != 0L || (active8 & 0x1000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 26;
             return 89;
          }
-         if ((active2 & 0x100L) != 0L || (active8 & 0x600000000L) != 0L)
-            return 89;
          return -1;
       case 27:
          if ((active1 & 0x8100000000000000L) != 0L || (active8 & 0x1000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 27;
             return 89;
          }
@@ -5707,7 +5705,7 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
       case 28:
          if ((active1 & 0x8100000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 28;
             return 89;
          }
@@ -5717,7 +5715,7 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
       case 29:
          if ((active1 & 0x8100000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 29;
             return 89;
          }
@@ -5725,7 +5723,7 @@ private final int jjStopStringLiteralDfa_0(int pos, long active0, long active1,
       case 30:
          if ((active1 & 0x8000000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 30;
             return 89;
          }
@@ -5754,56 +5752,56 @@ private final int jjMoveStringLiteralDfa0_0()
    {
       case 33:
          jjmatchedKind = 1;
-         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x100000000000L, 0x0L);
+         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x80000000000L, 0x0L);
       case 34:
-         return jjStopAtPos(0, 758);
+         return jjStopAtPos(0, 757);
       case 36:
-         return jjStartNfaWithStates_0(0, 761, 89);
+         return jjStartNfaWithStates_0(0, 760, 89);
       case 37:
-         return jjStopAtPos(0, 753);
+         return jjStopAtPos(0, 752);
       case 39:
-         return jjStartNfaWithStates_0(0, 757, 55);
+         return jjStartNfaWithStates_0(0, 756, 55);
       case 40:
-         return jjStopAtPos(0, 727);
+         return jjStopAtPos(0, 726);
       case 41:
-         return jjStopAtPos(0, 728);
+         return jjStopAtPos(0, 727);
       case 42:
-         jjmatchedKind = 751;
-         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x2L);
+         jjmatchedKind = 750;
+         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x1L);
       case 43:
-         return jjStopAtPos(0, 749);
+         return jjStopAtPos(0, 748);
       case 44:
-         return jjStopAtPos(0, 739);
+         return jjStopAtPos(0, 738);
       case 45:
-         return jjStartNfaWithStates_0(0, 750, 15);
+         return jjStartNfaWithStates_0(0, 749, 15);
       case 46:
-         jjmatchedKind = 738;
-         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x10000000000000L, 0x0L);
+         jjmatchedKind = 737;
+         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x8000000000000L, 0x0L);
       case 47:
-         jjmatchedKind = 752;
-         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x9L);
+         jjmatchedKind = 751;
+         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x8000000000000000L, 0x4L);
       case 58:
-         jjmatchedKind = 744;
-         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x400000000000000L, 0x0L);
+         jjmatchedKind = 743;
+         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x200000000000000L, 0x0L);
       case 59:
-         return jjStopAtPos(0, 737);
+         return jjStopAtPos(0, 736);
       case 60:
-         jjmatchedKind = 742;
-         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xa0000000000L, 0x0L);
+         jjmatchedKind = 741;
+         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x50000000000L, 0x0L);
       case 61:
-         jjmatchedKind = 740;
-         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x8000000000000L, 0x0L);
+         jjmatchedKind = 739;
+         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x4000000000000L, 0x0L);
       case 62:
-         jjmatchedKind = 741;
-         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x40000000000L, 0x0L);
+         jjmatchedKind = 740;
+         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x20000000000L, 0x0L);
       case 63:
-         return jjStopAtPos(0, 743);
+         return jjStopAtPos(0, 742);
       case 91:
-         return jjStartNfaWithStates_0(0, 735, 88);
+         return jjStartNfaWithStates_0(0, 734, 87);
       case 93:
-         return jjStopAtPos(0, 736);
+         return jjStopAtPos(0, 735);
       case 94:
-         return jjStopAtPos(0, 760);
+         return jjStopAtPos(0, 759);
       case 65:
       case 97:
          jjmatchedKind = 3;
@@ -5871,7 +5869,7 @@ private final int jjMoveStringLiteralDfa0_0()
          return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xfffffffff000000L, 0x2000000000000L, 0x0L, 0x0L);
       case 85:
       case 117:
-         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xf000000000000000L, 0xfffffL, 0x100L, 0x0L);
+         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xf000000000000000L, 0xfffffL, 0x0L, 0x0L);
       case 86:
       case 118:
          return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x20000007ff00000L, 0x0L, 0x0L);
@@ -5888,12 +5886,12 @@ private final int jjMoveStringLiteralDfa0_0()
       case 122:
          return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x400000000000L, 0x0L, 0x0L);
       case 123:
-         return jjStartNfaWithStates_0(0, 733, 87);
+         return jjStartNfaWithStates_0(0, 732, 86);
       case 124:
-         jjmatchedKind = 759;
-         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x4000000000000L, 0x0L);
+         jjmatchedKind = 758;
+         return jjMoveStringLiteralDfa1_0(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x2000000000000L, 0x0L);
       case 125:
-         return jjStopAtPos(0, 734);
+         return jjStopAtPos(0, 733);
       case 126:
          return jjStopAtPos(0, 2);
       default :
@@ -5910,37 +5908,37 @@ private final int jjMoveStringLiteralDfa1_0(long active0, long active1, long act
    switch(curChar)
    {
       case 42:
-         if ((active12 & 0x8L) != 0L)
+         if ((active12 & 0x4L) != 0L)
          {
-            jjmatchedKind = 771;
+            jjmatchedKind = 770;
             jjmatchedPos = 1;
          }
-         return jjMoveStringLiteralDfa2_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0x1L);
+         return jjMoveStringLiteralDfa2_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0x8000000000000000L, active12, 0L);
       case 46:
-         if ((active11 & 0x10000000000000L) != 0L)
-            return jjStopAtPos(1, 756);
+         if ((active11 & 0x8000000000000L) != 0L)
+            return jjStopAtPos(1, 755);
          break;
       case 47:
-         if ((active12 & 0x2L) != 0L)
-            return jjStopAtPos(1, 769);
+         if ((active12 & 0x1L) != 0L)
+            return jjStopAtPos(1, 768);
          break;
       case 58:
-         if ((active11 & 0x400000000000000L) != 0L)
-            return jjStopAtPos(1, 762);
+         if ((active11 & 0x200000000000000L) != 0L)
+            return jjStopAtPos(1, 761);
          break;
       case 61:
-         if ((active11 & 0x20000000000L) != 0L)
+         if ((active11 & 0x10000000000L) != 0L)
+            return jjStopAtPos(1, 744);
+         else if ((active11 & 0x20000000000L) != 0L)
             return jjStopAtPos(1, 745);
-         else if ((active11 & 0x40000000000L) != 0L)
-            return jjStopAtPos(1, 746);
-         else if ((active11 & 0x100000000000L) != 0L)
-            return jjStopAtPos(1, 748);
+         else if ((active11 & 0x80000000000L) != 0L)
+            return jjStopAtPos(1, 747);
          break;
       case 62:
-         if ((active11 & 0x80000000000L) != 0L)
-            return jjStopAtPos(1, 747);
-         else if ((active11 & 0x8000000000000L) != 0L)
-            return jjStopAtPos(1, 755);
+         if ((active11 & 0x40000000000L) != 0L)
+            return jjStopAtPos(1, 746);
+         else if ((active11 & 0x4000000000000L) != 0L)
+            return jjStopAtPos(1, 754);
          break;
       case 65:
       case 97:
@@ -6055,7 +6053,7 @@ private final int jjMoveStringLiteralDfa1_0(long active0, long active1, long act
          return jjMoveStringLiteralDfa2_0(active0, 0x700000000L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x8000006000000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0xffcL, active10, 0x100000000e0000L, active11, 0L, active12, 0L);
       case 85:
       case 117:
-         return jjMoveStringLiteralDfa2_0(active0, 0x800000000L, active1, 0xfff8000000000000L, active2, 0x7L, active3, 0x70000000L, active4, 0L, active5, 0x1f8000300000L, active6, 0x3000000000007L, active7, 0x400000000L, active8, 0L, active9, 0x4000000000ff000L, active10, 0L, active11, 0x180L, active12, 0L);
+         return jjMoveStringLiteralDfa2_0(active0, 0x800000000L, active1, 0xfff8000000000000L, active2, 0x7L, active3, 0x70000000L, active4, 0L, active5, 0x1f8000300000L, active6, 0x3000000000007L, active7, 0x400000000L, active8, 0L, active9, 0x4000000000ff000L, active10, 0L, active11, 0x80L, active12, 0L);
       case 86:
       case 118:
          return jjMoveStringLiteralDfa2_0(active0, 0x1000000000L, active1, 0L, active2, 0x2000000000000000L, active3, 0L, active4, 0L, active5, 0L, active6, 0x78L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
@@ -6068,8 +6066,8 @@ private final int jjMoveStringLiteralDfa1_0(long active0, long active1, long act
             return jjStartNfaWithStates_0(1, 50, 89);
          return jjMoveStringLiteralDfa2_0(active0, 0L, active1, 0L, active2, 0xe00000000008L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0x800000000f00000L, active10, 0L, active11, 0L, active12, 0L);
       case 124:
-         if ((active11 & 0x4000000000000L) != 0L)
-            return jjStopAtPos(1, 754);
+         if ((active11 & 0x2000000000000L) != 0L)
+            return jjStopAtPos(1, 753);
          break;
       default :
          break;
@@ -6082,23 +6080,23 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
       return jjStartNfa_0(0, old0, old1, old2, old3, old4, old5, old6, old7, old8, old9, old10, old11, old12); 
    try { curChar = input_stream.readChar(); }
    catch(java.io.IOException e) {
-      jjStopStringLiteralDfa_0(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, active12);
+      jjStopStringLiteralDfa_0(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, 0L);
       return 2;
    }
    switch(curChar)
    {
       case 43:
-         if ((active12 & 0x1L) != 0L)
-            return jjStopAtPos(2, 768);
+         if ((active11 & 0x8000000000000000L) != 0L)
+            return jjStopAtPos(2, 767);
          break;
       case 65:
       case 97:
          if ((active0 & 0x100L) != 0L)
             return jjStartNfaWithStates_0(2, 8, 89);
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0x9bffL, active2, 0x400000800L, active3, 0x18003000000L, active4, 0x30000000000L, active5, 0x1L, active6, 0x72000180000000L, active7, 0x203000000000L, active8, 0x120L, active9, 0x1ff800000003cL, active10, 0x320000000400L, active11, 0x4L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0x9bffL, active2, 0x400000800L, active3, 0x18003000000L, active4, 0x30000000000L, active5, 0x1L, active6, 0x72000180000000L, active7, 0x203000000000L, active8, 0x120L, active9, 0x1ff800000003cL, active10, 0x320000000400L, active11, 0x4L);
       case 66:
       case 98:
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0x8000000010000L, active2, 0L, active3, 0L, active4, 0x200200000000L, active5, 0L, active6, 0x1000000000000L, active7, 0L, active8, 0L, active9, 0x200000000703f000L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0x8000000010000L, active2, 0L, active3, 0L, active4, 0x200200000000L, active5, 0L, active6, 0x1000000000000L, active7, 0L, active8, 0L, active9, 0x200000000703f000L, active10, 0L, active11, 0L);
       case 67:
       case 99:
          if ((active0 & 0x4000000L) != 0L)
@@ -6108,7 +6106,7 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             jjmatchedKind = 140;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0xd01100000000e008L, active3, 0x3800000000000003L, active4, 0x3c000000000000L, active5, 0x400000000080L, active6, 0x80000000000000L, active7, 0x3c00000000000L, active8, 0L, active9, 0xc000000000040000L, active10, 0x4c4000000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0xd01100000000e008L, active3, 0x3800000000000003L, active4, 0x3c000000000000L, active5, 0x400000000080L, active6, 0x80000000000000L, active7, 0x3c00000000000L, active8, 0L, active9, 0xc000000000040000L, active10, 0x4c4000000000000L, active11, 0L);
       case 68:
       case 100:
          if ((active0 & 0x200L) != 0L)
@@ -6129,14 +6127,14 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             return jjStartNfaWithStates_0(2, 371, 89);
          else if ((active6 & 0x80L) != 0L)
             return jjStartNfaWithStates_0(2, 391, 89);
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0x1c0000000000000L, active3, 0x4000000000000000L, active4, 0L, active5, 0x7000000000018000L, active6, 0L, active7, 0L, active8, 0L, active9, 0xc0L, active10, 0x800000800000081L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0x1c0000000000000L, active3, 0x4000000000000000L, active4, 0L, active5, 0x7000000000018000L, active6, 0L, active7, 0L, active8, 0L, active9, 0xc0L, active10, 0x800000800000081L, active11, 0L);
       case 69:
       case 101:
          if ((active0 & 0x100000L) != 0L)
             return jjStartNfaWithStates_0(2, 20, 89);
          else if ((active5 & 0x40000000000000L) != 0L)
             return jjStartNfaWithStates_0(2, 374, 89);
-         return jjMoveStringLiteralDfa3_0(active0, 0x2000008000000L, active1, 0x2000000000400L, active2, 0x2002000000000000L, active3, 0x840000400000cL, active4, 0L, active5, 0x100000000000000L, active6, 0xfc000000078L, active7, 0x4000000000000L, active8, 0xe00L, active9, 0x2000030000000L, active10, 0x7c000f800L, active11, 0x80L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x2000008000000L, active1, 0x2000000000400L, active2, 0x2002000000000000L, active3, 0x840000400000cL, active4, 0L, active5, 0x100000000000000L, active6, 0xfc000000078L, active7, 0x4000000000000L, active8, 0xe00L, active9, 0x2000030000000L, active10, 0x7c000f800L, active11, 0x80L);
       case 70:
       case 102:
          if ((active6 & 0x100000000000000L) != 0L)
@@ -6144,28 +6142,28 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             jjmatchedKind = 440;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_0(active0, 0x2000000000L, active1, 0L, active2, 0x7f0000L, active3, 0L, active4, 0x40000000000L, active5, 0x4000000000000L, active6, 0x600000000000000L, active7, 0L, active8, 0L, active9, 0L, active10, 0x80000000e0000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x2000000000L, active1, 0L, active2, 0x7f0000L, active3, 0L, active4, 0x40000000000L, active5, 0x4000000000000L, active6, 0x600000000000000L, active7, 0L, active8, 0L, active9, 0L, active10, 0x80000000e0000L, active11, 0L);
       case 71:
       case 103:
          if ((active0 & 0x1000000000L) != 0L)
             return jjStartNfaWithStates_0(2, 36, 89);
          else if ((active4 & 0x400000000L) != 0L)
             return jjStartNfaWithStates_0(2, 290, 89);
-         return jjMoveStringLiteralDfa3_0(active0, 0x9c000000000L, active1, 0L, active2, 0x800000L, active3, 0L, active4, 0L, active5, 0L, active6, 0xf800000000000000L, active7, 0x20000fL, active8, 0L, active9, 0L, active10, 0x4000000000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x9c000000000L, active1, 0L, active2, 0x800000L, active3, 0L, active4, 0L, active5, 0L, active6, 0xf800000000000000L, active7, 0x20000fL, active8, 0L, active9, 0L, active10, 0x4000000000000000L, active11, 0L);
       case 72:
       case 104:
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x8000002008000000L, active6, 0L, active7, 0xc000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x8000002008000000L, active6, 0L, active7, 0xc000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 73:
       case 105:
          if ((active6 & 0x200000000L) != 0L)
             return jjStartNfaWithStates_0(2, 417, 89);
-         return jjMoveStringLiteralDfa3_0(active0, 0x6000000000000000L, active1, 0L, active2, 0L, active3, 0x8020000000000010L, active4, 0x100001L, active5, 0x10004000000000L, active6, 0x700000000000L, active7, 0x400000L, active8, 0x10L, active9, 0xfc000000000000L, active10, 0x20040000010006L, active11, 0x100L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x6000000000000000L, active1, 0L, active2, 0L, active3, 0x8020000000000010L, active4, 0x100001L, active5, 0x10004000000000L, active6, 0x700000000000L, active7, 0x400000L, active8, 0x10L, active9, 0xfc000000000000L, active10, 0x20040000010006L, active11, 0L);
       case 74:
       case 106:
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x200000000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x200000000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 75:
       case 107:
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0xc00000000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0x8L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0xc00000000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0x8L, active11, 0L);
       case 76:
       case 108:
          if ((active0 & 0x1000L) != 0L)
@@ -6180,12 +6178,12 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
          }
          else if ((active10 & 0x80000000000L) != 0L)
             return jjStartNfaWithStates_0(2, 683, 89);
-         return jjMoveStringLiteralDfa3_0(active0, 0x30000000006000L, active1, 0x1fe0000L, active2, 0x1000000L, active3, 0x800010041400L, active4, 0L, active5, 0x80078010100300L, active6, 0L, active7, 0x18000003800030L, active8, 0xffffffffffffe000L, active9, 0x1L, active10, 0xa200000000700000L, active11, 0x2L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x30000000006000L, active1, 0x1fe0000L, active2, 0x1000000L, active3, 0x800010041400L, active4, 0L, active5, 0x80078010100300L, active6, 0L, active7, 0x18000003800030L, active8, 0xffffffffffffe000L, active9, 0x1L, active10, 0xa200000000700000L, active11, 0x2L);
       case 77:
       case 109:
          if ((active9 & 0x80000L) != 0L)
             return jjStartNfaWithStates_0(2, 595, 89);
-         return jjMoveStringLiteralDfa3_0(active0, 0x400L, active1, 0x1000001e000000L, active2, 0x8000000000L, active3, 0xc0000000000000L, active4, 0x1000000000000L, active5, 0x180000e00002L, active6, 0L, active7, 0L, active8, 0x46L, active9, 0x400000fc8100000L, active10, 0x2800000000000L, active11, 0x20L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x400L, active1, 0x1000001e000000L, active2, 0x8000000000L, active3, 0xc0000000000000L, active4, 0x1000000000000L, active5, 0x180000e00002L, active6, 0L, active7, 0L, active8, 0x46L, active9, 0x400000fc8100000L, active10, 0x2800000000000L, active11, 0x20L);
       case 78:
       case 110:
          if ((active5 & 0x400L) != 0L)
@@ -6193,10 +6191,10 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             jjmatchedKind = 330;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_0(active0, 0x8000100000000000L, active1, 0xfffe0000000L, active2, 0xe00002000000L, active3, 0x10000320002000L, active4, 0x80800000002L, active5, 0x201063800L, active6, 0xc000000000000L, active7, 0x8020000400000000L, active8, 0L, active9, 0x1000000000L, active10, 0x401000000060L, active11, 0x8L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x8000100000000000L, active1, 0xfffe0000000L, active2, 0xe00002000000L, active3, 0x10000320002000L, active4, 0x80800000002L, active5, 0x201063800L, active6, 0xc000000000000L, active7, 0x8020000400000000L, active8, 0L, active9, 0x1000000000L, active10, 0x401000000060L, active11, 0x8L);
       case 79:
       case 111:
-         return jjMoveStringLiteralDfa3_0(active0, 0xc00100000000L, active1, 0x4000000006000L, active2, 0x200100000000000L, active3, 0x1e1408030000L, active4, 0x1fe70004L, active5, 0L, active6, 0x800000000000L, active7, 0xf0000000000L, active8, 0x1L, active9, 0L, active10, 0x10000000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0xc00100000000L, active1, 0x4000000006000L, active2, 0x200100000000000L, active3, 0x1e1408030000L, active4, 0x1fe70004L, active5, 0L, active6, 0x800000000000L, active7, 0xf0000000000L, active8, 0x1L, active9, 0L, active10, 0x10000000000000L, active11, 0L);
       case 80:
       case 112:
          if ((active3 & 0x20L) != 0L)
@@ -6208,10 +6206,10 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             return jjStartNfaWithStates_0(2, 240, 89);
          else if ((active4 & 0x100000000000000L) != 0L)
             return jjStartNfaWithStates_0(2, 312, 89);
-         return jjMoveStringLiteralDfa3_0(active0, 0x80000L, active1, 0L, active2, 0x8000004000000L, active3, 0x300000000000040L, active4, 0x8L, active5, 0L, active6, 0L, active7, 0x400000000000c0L, active8, 0L, active9, 0x800004000000000L, active10, 0x110L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x80000L, active1, 0L, active2, 0x8000004000000L, active3, 0x300000000000040L, active4, 0x8L, active5, 0L, active6, 0L, active7, 0x400000000000c0L, active8, 0L, active9, 0x800004000000000L, active10, 0x110L, active11, 0L);
       case 81:
       case 113:
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0x80000000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0x80000000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 82:
       case 114:
          if ((active3 & 0x80000L) != 0L)
@@ -6224,7 +6222,7 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             jjmatchedKind = 407;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_0(active0, 0x40020001e00000L, active1, 0xffe0300000000000L, active2, 0x800000018000007L, active3, 0x70c000L, active4, 0x1000000000L, active5, 0xc00080004L, active6, 0x43f01ff00L, active7, 0x700100000000000L, active8, 0L, active9, 0x702L, active10, 0x100001003f800000L, active11, 0x10L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x40020001e00000L, active1, 0xffe0300000000000L, active2, 0x800000018000007L, active3, 0x70c000L, active4, 0x1000000000L, active5, 0xc00080004L, active6, 0x43f01ff00L, active7, 0x700100000000000L, active8, 0L, active9, 0x702L, active10, 0x100001003f800000L, active11, 0x10L);
       case 83:
       case 115:
          if ((active0 & 0x10L) != 0L)
@@ -6232,7 +6230,7 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             jjmatchedKind = 4;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_0(active0, 0x780000030000060L, active1, 0L, active2, 0x40079e0000000L, active3, 0x40000000L, active4, 0x60000000f0L, active5, 0x20000038L, active6, 0x18001e0000L, active7, 0x1800000000001f00L, active8, 0L, active9, 0x1000000000e00000L, active10, 0x200L, active11, 0x1L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x780000030000060L, active1, 0L, active2, 0x40079e0000000L, active3, 0x40000000L, active4, 0x60000000f0L, active5, 0x20000038L, active6, 0x18001e0000L, active7, 0x1800000000001f00L, active8, 0L, active9, 0x1000000000e00000L, active10, 0x200L, active11, 0x1L);
       case 84:
       case 116:
          if ((active0 & 0x200000000000L) != 0L)
@@ -6258,13 +6256,13 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             jjmatchedKind = 509;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_0(active0, 0x1801040e00008880L, active1, 0L, active2, 0x2000001f0L, active3, 0x4000000b80L, active4, 0x3e00008000003e00L, active5, 0x601800006000040L, active6, 0x600006L, active7, 0x40000000000fe000L, active8, 0L, active9, 0L, active10, 0x10000e000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x1801040e00008880L, active1, 0L, active2, 0x2000001f0L, active3, 0x4000000b80L, active4, 0x3e00008000003e00L, active5, 0x601800006000040L, active6, 0x600006L, active7, 0x40000000000fe000L, active8, 0L, active9, 0L, active10, 0x10000e000000000L, active11, 0L);
       case 85:
       case 117:
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0x400000000000L, active2, 0x400020000000000L, active3, 0x6000000800000L, active4, 0L, active5, 0L, active6, 0L, active7, 0x3c000000L, active8, 0x80L, active9, 0x300000000000000L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0x400000000000L, active2, 0x400020000000000L, active3, 0x6000000800000L, active4, 0L, active5, 0L, active6, 0L, active7, 0x3c000000L, active8, 0x80L, active9, 0x300000000000000L, active10, 0L, active11, 0L);
       case 86:
       case 118:
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0x1800000000000L, active2, 0L, active3, 0x200000000000L, active4, 0x100000084000L, active5, 0L, active6, 0x40000000L, active7, 0x800100000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0x1800000000000L, active2, 0L, active3, 0x200000000000L, active4, 0x100000084000L, active5, 0L, active6, 0x40000000L, active7, 0x800100000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 87:
       case 119:
          if ((active2 & 0x40000000000L) != 0L)
@@ -6276,7 +6274,7 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             jjmatchedKind = 478;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_0(active0, 0x10000L, active1, 0L, active2, 0L, active3, 0L, active4, 0x40000000000000L, active5, 0L, active6, 0x2000000000L, active7, 0x380000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x10000L, active1, 0L, active2, 0L, active3, 0L, active4, 0x40000000000000L, active5, 0L, active6, 0x2000000000L, active7, 0x380000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 88:
       case 120:
          if ((active4 & 0x4000000000000000L) != 0L)
@@ -6284,7 +6282,7 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             jjmatchedKind = 318;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0x8000000000000000L, active5, 0x80000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0x8000000000000000L, active5, 0x80000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 89:
       case 121:
          if ((active0 & 0x40000L) != 0L)
@@ -6301,19 +6299,19 @@ private final int jjMoveStringLiteralDfa2_0(long old0, long active0, long old1,
             jjmatchedKind = 286;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_0(active0, 0x40000000L, active1, 0L, active2, 0x400L, active3, 0L, active4, 0x180000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0x800L, active10, 0L, active11, 0x40L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0x40000000L, active1, 0L, active2, 0x400L, active3, 0L, active4, 0x180000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0x800L, active10, 0L, active11, 0x40L);
       case 90:
       case 122:
-         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0x8L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_0(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0x8L, active9, 0L, active10, 0L, active11, 0L);
       default :
          break;
    }
-   return jjStartNfa_0(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, active12);
+   return jjStartNfa_0(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, 0L);
 }
-private final int jjMoveStringLiteralDfa3_0(long old0, long active0, long old1, long active1, long old2, long active2, long old3, long active3, long old4, long active4, long old5, long active5, long old6, long active6, long old7, long active7, long old8, long active8, long old9, long active9, long old10, long active10, long old11, long active11, long old12, long active12)
+private final int jjMoveStringLiteralDfa3_0(long old0, long active0, long old1, long active1, long old2, long active2, long old3, long active3, long old4, long active4, long old5, long active5, long old6, long active6, long old7, long active7, long old8, long active8, long old9, long active9, long old10, long active10, long old11, long active11)
 {
-   if (((active0 &= old0) | (active1 &= old1) | (active2 &= old2) | (active3 &= old3) | (active4 &= old4) | (active5 &= old5) | (active6 &= old6) | (active7 &= old7) | (active8 &= old8) | (active9 &= old9) | (active10 &= old10) | (active11 &= old11) | (active12 &= old12)) == 0L)
-      return jjStartNfa_0(1, old0, old1, old2, old3, old4, old5, old6, old7, old8, old9, old10, old11, old12); 
+   if (((active0 &= old0) | (active1 &= old1) | (active2 &= old2) | (active3 &= old3) | (active4 &= old4) | (active5 &= old5) | (active6 &= old6) | (active7 &= old7) | (active8 &= old8) | (active9 &= old9) | (active10 &= old10) | (active11 &= old11)) == 0L)
+      return jjStartNfa_0(1, old0, old1, old2, old3, old4, old5, old6, old7, old8, old9, old10, old11, 0L);
    try { curChar = input_stream.readChar(); }
    catch(java.io.IOException e) {
       jjStopStringLiteralDfa_0(2, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, 0L);
@@ -6377,8 +6375,6 @@ private final int jjMoveStringLiteralDfa3_0(long old0, long active0, long old1,
             jjmatchedKind = 436;
             jjmatchedPos = 3;
          }
-         else if ((active11 & 0x100L) != 0L)
-            return jjStartNfaWithStates_0(3, 712, 89);
          return jjMoveStringLiteralDfa4_0(active0, 0x40000000000000L, active1, 0xe0000000L, active2, 0L, active3, 0L, active4, 0x20000010000L, active5, 0L, active6, 0x20000000000000L, active7, 0L, active8, 0L, active9, 0xc0L, active10, 0x1000000000L, active11, 0L);
       case 69:
       case 101:
@@ -9638,14 +9634,14 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x3ff001000000000L & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (curChar == 36)
                      jjCheckNAdd(31);
                   break;
-               case 86:
+               case 88:
                   if ((0x7ff601000000000L & l) != 0L)
                      jjCheckNAddTwoStates(29, 30);
                   else if (curChar == 39)
@@ -9654,14 +9650,14 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x3ff001000000000L & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (curChar == 36)
                      jjCheckNAdd(31);
                   break;
-               case 88:
+               case 87:
                case 22:
                   if ((0xffffffffffffdbffL & l) != 0L)
                      jjCheckNAddStates(6, 8);
@@ -9669,8 +9665,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 84:
                   if (curChar == 47)
                   {
-                     if (kind > 772)
-                        kind = 772;
+                     if (kind > 771)
+                        kind = 771;
                      jjCheckNAddStates(9, 11);
                   }
                   else if (curChar == 42)
@@ -9687,8 +9683,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 59;
                   if ((0x3ff001000000000L & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (curChar == 36)
@@ -9699,15 +9695,15 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjCheckNAddStates(12, 14);
                   else if (curChar == 39)
                   {
-                     if (kind > 720)
-                        kind = 720;
+                     if (kind > 719)
+                        kind = 719;
                   }
                   if ((0xfc00f7faffffc9ffL & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 56;
                   if (curChar == 39)
                      jjstateSet[jjnewStateCnt++] = 52;
                   break;
-               case 87:
+               case 86:
                   if (curChar == 32)
                      jjCheckNAddTwoStates(78, 79);
                   if (curChar == 32)
@@ -9726,8 +9722,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 90:
                   if ((0x3ff000000000000L & l) != 0L)
                   {
-                     if (kind > 715)
-                        kind = 715;
+                     if (kind > 714)
+                        kind = 714;
                      jjCheckNAdd(49);
                   }
                   if ((0x3ff000000000000L & l) != 0L)
@@ -9742,8 +9738,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x3ff001000000000L & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (curChar == 36)
@@ -9760,21 +9756,21 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjCheckNAddTwoStates(48, 49);
                   else if (curChar == 7)
                   {
-                     if (kind > 786)
-                        kind = 786;
+                     if (kind > 785)
+                        kind = 785;
                   }
                   else if (curChar == 45)
                      jjstateSet[jjnewStateCnt++] = 15;
                   if ((0x3ff000000000000L & l) != 0L)
                   {
-                     if (kind > 713)
-                        kind = 713;
+                     if (kind > 712)
+                        kind = 712;
                      jjCheckNAddStates(24, 30);
                   }
                   else if (curChar == 36)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   break;
@@ -9791,8 +9787,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 3;
                   break;
                case 5:
-                  if (curChar == 39 && kind > 719)
-                     kind = 719;
+                  if (curChar == 39 && kind > 718)
+                     kind = 718;
                   break;
                case 7:
                   if ((0x3ff000000000000L & l) != 0L)
@@ -9816,30 +9812,30 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 11;
                   break;
                case 13:
-                  if (curChar == 39 && kind > 721)
-                     kind = 721;
+                  if (curChar == 39 && kind > 720)
+                     kind = 720;
                   break;
                case 15:
                   if (curChar != 45)
                      break;
-                  if (kind > 772)
-                     kind = 772;
+                  if (kind > 771)
+                     kind = 771;
                   jjCheckNAddStates(9, 11);
                   break;
                case 16:
                   if ((0xffffffffffffdbffL & l) == 0L)
                      break;
-                  if (kind > 772)
-                     kind = 772;
+                  if (kind > 771)
+                     kind = 771;
                   jjCheckNAddStates(9, 11);
                   break;
                case 17:
-                  if ((0x2400L & l) != 0L && kind > 772)
-                     kind = 772;
+                  if ((0x2400L & l) != 0L && kind > 771)
+                     kind = 771;
                   break;
                case 18:
-                  if (curChar == 10 && kind > 772)
-                     kind = 772;
+                  if (curChar == 10 && kind > 771)
+                     kind = 771;
                   break;
                case 19:
                   if (curChar == 13)
@@ -9852,15 +9848,15 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 26:
                   if (curChar != 36)
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 27:
                   if ((0x3ff001000000000L & l) == 0L)
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 28:
@@ -9878,8 +9874,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 31:
                   if (curChar != 36)
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjCheckNAddTwoStates(31, 32);
                   break;
                case 32:
@@ -9889,26 +9885,26 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 33:
                   if ((0x3ff001000000000L & l) == 0L)
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjCheckNAdd(33);
                   break;
                case 34:
-                  if (curChar == 7 && kind > 786)
-                     kind = 786;
+                  if (curChar == 7 && kind > 785)
+                     kind = 785;
                   break;
                case 35:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 713)
-                     kind = 713;
+                  if (kind > 712)
+                     kind = 712;
                   jjCheckNAddStates(24, 30);
                   break;
                case 36:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 713)
-                     kind = 713;
+                  if (kind > 712)
+                     kind = 712;
                   jjCheckNAdd(36);
                   break;
                case 37:
@@ -9922,8 +9918,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 40:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 714)
-                     kind = 714;
+                  if (kind > 713)
+                     kind = 713;
                   jjCheckNAdd(40);
                   break;
                case 41:
@@ -9937,22 +9933,22 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 43:
                   if (curChar != 46)
                      break;
-                  if (kind > 715)
-                     kind = 715;
+                  if (kind > 714)
+                     kind = 714;
                   jjCheckNAdd(44);
                   break;
                case 44:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 715)
-                     kind = 715;
+                  if (kind > 714)
+                     kind = 714;
                   jjCheckNAdd(44);
                   break;
                case 45:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 715)
-                     kind = 715;
+                  if (kind > 714)
+                     kind = 714;
                   jjCheckNAddStates(31, 33);
                   break;
                case 46:
@@ -9970,8 +9966,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 49:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
-                  if (kind > 715)
-                     kind = 715;
+                  if (kind > 714)
+                     kind = 714;
                   jjCheckNAdd(49);
                   break;
                case 50:
@@ -9991,12 +9987,12 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 52;
                   break;
                case 54:
-                  if (curChar == 39 && kind > 720)
-                     kind = 720;
+                  if (curChar == 39 && kind > 719)
+                     kind = 719;
                   break;
                case 56:
-                  if (curChar == 39 && kind > 726)
-                     kind = 726;
+                  if (curChar == 39 && kind > 725)
+                     kind = 725;
                   break;
                case 59:
                case 61:
@@ -10012,8 +10008,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 61;
                   break;
                case 63:
-                  if (curChar == 39 && kind > 722)
-                     kind = 722;
+                  if (curChar == 39 && kind > 721)
+                     kind = 721;
                   break;
                case 64:
                   if (curChar == 38)
@@ -10036,8 +10032,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 67;
                   break;
                case 69:
-                  if (curChar == 34 && kind > 783)
-                     kind = 783;
+                  if (curChar == 34 && kind > 782)
+                     kind = 782;
                   break;
                case 71:
                   if (curChar == 32)
@@ -10064,14 +10060,14 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 83;
                   break;
                case 83:
-                  if ((0xffff7fffffffffffL & l) != 0L && kind > 770)
-                     kind = 770;
+                  if ((0xffff7fffffffffffL & l) != 0L && kind > 769)
+                     kind = 769;
                   break;
                case 85:
                   if (curChar != 47)
                      break;
-                  if (kind > 772)
-                     kind = 772;
+                  if (kind > 771)
+                     kind = 771;
                   jjCheckNAddStates(9, 11);
                   break;
                default : break;
@@ -10092,24 +10088,24 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   break;
-               case 86:
+               case 88:
                   if ((0x7fffffe87fffffeL & l) != 0L)
                      jjCheckNAddTwoStates(29, 30);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                      jjCheckNAddStates(0, 2);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   break;
-               case 88:
+               case 87:
                   if ((0xffffffffdfffffffL & l) != 0L)
                      jjCheckNAddStates(6, 8);
                   else if (curChar == 93)
@@ -10122,8 +10118,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   break;
@@ -10132,20 +10128,20 @@ private final int jjMoveNfa_0(int startState, int curPos)
                   if ((0xffffff81ffffff81L & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 56;
                   break;
-               case 87:
+               case 86:
                   if ((0x4000000040L & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 80;
                   else if ((0x10000000100000L & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 77;
                   else if ((0x1000000010L & l) != 0L)
                   {
-                     if (kind > 729)
-                        kind = 729;
+                     if (kind > 728)
+                        kind = 728;
                   }
                   if ((0x10000000100000L & l) != 0L)
                   {
-                     if (kind > 730)
-                        kind = 730;
+                     if (kind > 729)
+                        kind = 729;
                   }
                   break;
                case 91:
@@ -10160,8 +10156,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   break;
@@ -10174,8 +10170,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjCheckNAddTwoStates(22, 24);
                   if ((0x7fffffe87fffffeL & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if ((0x20000000200000L & l) != 0L)
@@ -10210,8 +10206,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjCheckNAdd(9);
                   break;
                case 16:
-                  if (kind > 772)
-                     kind = 772;
+                  if (kind > 771)
+                     kind = 771;
                   jjAddStates(9, 11);
                   break;
                case 21:
@@ -10231,21 +10227,21 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjstateSet[jjnewStateCnt++] = 23;
                   break;
                case 25:
-                  if (curChar == 93 && kind > 776)
-                     kind = 776;
+                  if (curChar == 93 && kind > 775)
+                     kind = 775;
                   break;
                case 26:
                   if ((0x7fffffe87fffffeL & l) == 0L)
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 27:
                   if ((0x7fffffe87fffffeL & l) == 0L)
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 28:
@@ -10255,15 +10251,15 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 31:
                   if ((0x7fffffe87fffffeL & l) == 0L)
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjAddStates(54, 55);
                   break;
                case 33:
                   if ((0x7fffffe87fffffeL & l) == 0L)
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjstateSet[jjnewStateCnt++] = 33;
                   break;
                case 38:
@@ -10288,32 +10284,32 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjAddStates(44, 51);
                   break;
                case 72:
-                  if ((0x1000000010L & l) != 0L && kind > 729)
-                     kind = 729;
+                  if ((0x1000000010L & l) != 0L && kind > 728)
+                     kind = 728;
                   break;
                case 74:
-                  if ((0x10000000100000L & l) != 0L && kind > 730)
-                     kind = 730;
+                  if ((0x10000000100000L & l) != 0L && kind > 729)
+                     kind = 729;
                   break;
                case 76:
                   if ((0x10000000100000L & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 77;
                   break;
                case 77:
-                  if ((0x8000000080000L & l) != 0L && kind > 731)
-                     kind = 731;
+                  if ((0x8000000080000L & l) != 0L && kind > 730)
+                     kind = 730;
                   break;
                case 79:
                   if ((0x4000000040L & l) != 0L)
                      jjstateSet[jjnewStateCnt++] = 80;
                   break;
                case 80:
-                  if ((0x400000004000L & l) != 0L && kind > 732)
-                     kind = 732;
+                  if ((0x400000004000L & l) != 0L && kind > 731)
+                     kind = 731;
                   break;
                case 83:
-                  if (kind > 770)
-                     kind = 770;
+                  if (kind > 769)
+                     kind = 769;
                   break;
                default : break;
             }
@@ -10333,8 +10329,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 89:
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
@@ -10342,11 +10338,11 @@ private final int jjMoveNfa_0(int startState, int curPos)
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                      jjCheckNAddTwoStates(29, 30);
                   break;
-               case 86:
+               case 88:
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
@@ -10354,7 +10350,7 @@ private final int jjMoveNfa_0(int startState, int curPos)
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                      jjCheckNAddTwoStates(29, 30);
                   break;
-               case 88:
+               case 87:
                case 22:
                   if (jjCanMove_0(hiByte, i1, i2, l1, l2))
                      jjCheckNAddStates(6, 8);
@@ -10362,8 +10358,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 58:
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
@@ -10385,8 +10381,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 1:
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
@@ -10397,8 +10393,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 0:
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (jjCanMove_1(hiByte, i1, i2, l1, l2))
@@ -10415,22 +10411,22 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 16:
                   if (!jjCanMove_0(hiByte, i1, i2, l1, l2))
                      break;
-                  if (kind > 772)
-                     kind = 772;
+                  if (kind > 771)
+                     kind = 771;
                   jjAddStates(9, 11);
                   break;
                case 26:
                   if (!jjCanMove_1(hiByte, i1, i2, l1, l2))
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 27:
                   if (!jjCanMove_1(hiByte, i1, i2, l1, l2))
                      break;
-                  if (kind > 781)
-                     kind = 781;
+                  if (kind > 780)
+                     kind = 780;
                   jjCheckNAdd(27);
                   break;
                case 28:
@@ -10440,15 +10436,15 @@ private final int jjMoveNfa_0(int startState, int curPos)
                case 31:
                   if (!jjCanMove_1(hiByte, i1, i2, l1, l2))
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjAddStates(54, 55);
                   break;
                case 33:
                   if (!jjCanMove_1(hiByte, i1, i2, l1, l2))
                      break;
-                  if (kind > 782)
-                     kind = 782;
+                  if (kind > 781)
+                     kind = 781;
                   jjstateSet[jjnewStateCnt++] = 33;
                   break;
                case 51:
@@ -10464,8 +10460,8 @@ private final int jjMoveNfa_0(int startState, int curPos)
                      jjAddStates(41, 43);
                   break;
                case 83:
-                  if (jjCanMove_0(hiByte, i1, i2, l1, l2) && kind > 770)
-                     kind = 770;
+                  if (jjCanMove_0(hiByte, i1, i2, l1, l2) && kind > 769)
+                     kind = 769;
                   break;
                default : break;
             }
@@ -10509,7 +10505,7 @@ private final int jjMoveNfa_6(int startState, int curPos)
             {
                case 0:
                   if (curChar == 47)
-                     kind = 774;
+                     kind = 773;
                   break;
                case 1:
                   if (curChar == 42)
@@ -10563,60 +10559,60 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
    switch (pos)
    {
       case 0:
-         if ((active5 & 0x1fffffc00000L) != 0L || (active10 & 0x8000000000000000L) != 0L)
-         {
-            jjmatchedKind = 781;
-            return 86;
-         }
          if ((active10 & 0x80000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             return 1;
          }
-         if ((active11 & 0x20000000L) != 0L)
-            return 87;
-         if ((active9 & 0xf000000000000000L) != 0L || (active10 & 0xfffffL) != 0L || (active11 & 0x100L) != 0L)
-         {
-            jjmatchedKind = 781;
-            return 58;
-         }
-         if ((active11 & 0x20000000000000L) != 0L)
+         if ((active11 & 0x10000000L) != 0L)
+            return 86;
+         if ((active11 & 0x10000000000000L) != 0L)
             return 55;
-         if ((active11 & 0x1000000000000L) != 0L || (active12 & 0x9L) != 0L)
+         if ((active11 & 0x8000800000000000L) != 0L || (active12 & 0x4L) != 0L)
             return 84;
-         if ((active0 & 0x7ffe000000000L) != 0L || (active2 & 0xfffffffffffffff0L) != 0L || (active3 & 0xffffe0007fffffffL) != 0L || (active4 & 0x7ffffe1fffffffL) != 0L || (active5 & 0xffffe00000000000L) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0x7fffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xfffffffffffffffL) != 0L || (active10 & 0x7f27f7fffff00000L) != 0L || (active11 & 0x95L) != 0L)
+         if ((active5 & 0x1fffffc00000L) != 0L || (active10 & 0x8000000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
-            return 88;
+            jjmatchedKind = 780;
+            return 87;
          }
-         if ((active0 & 0xfff8001ffffffff8L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfL) != 0L || (active3 & 0x1fff80000000L) != 0L || (active4 & 0xff800001e0000000L) != 0L || (active5 & 0x3fffffL) != 0L || (active7 & 0x8000000000000000L) != 0L || (active10 & 0xd8000000000000L) != 0L || (active11 & 0x20000000000006aL) != 0L)
+         if ((active0 & 0xfff8001ffffffff8L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfL) != 0L || (active3 & 0x1fff80000000L) != 0L || (active4 & 0xff800001e0000000L) != 0L || (active5 & 0x3fffffL) != 0L || (active7 & 0x8000000000000000L) != 0L || (active10 & 0xd8000000000000L) != 0L || (active11 & 0x10000000000006aL) != 0L)
             return 88;
-         if ((active11 & 0x400000000000L) != 0L)
+         if ((active11 & 0x200000000000L) != 0L)
             return 15;
-         if ((active11 & 0x10000400000000L) != 0L)
+         if ((active0 & 0x7ffe000000000L) != 0L || (active2 & 0xfffffffffffffff0L) != 0L || (active3 & 0xffffe0007fffffffL) != 0L || (active4 & 0x7ffffe1fffffffL) != 0L || (active5 & 0xffffe00000000000L) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0x7fffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xfffffffffffffffL) != 0L || (active10 & 0x7f27f7fffff00000L) != 0L || (active11 & 0x95L) != 0L)
+         {
+            jjmatchedKind = 780;
+            return 88;
+         }
+         if ((active9 & 0xf000000000000000L) != 0L || (active10 & 0xfffffL) != 0L)
+         {
+            jjmatchedKind = 780;
+            return 58;
+         }
+         if ((active11 & 0x8000200000000L) != 0L)
             return 89;
          return -1;
       case 1:
-         if ((active12 & 0x9L) != 0L)
-            return 82;
-         if ((active0 & 0x40007fe000000L) != 0L || (active3 & 0xfc00006000000000L) != 0L || (active4 & 0x200000007ffffL) != 0L || (active5 & 0x78e6001f00000000L) != 0L || (active9 & 0x6000000000L) != 0L || (active10 & 0xa811000000000000L) != 0L || (active11 & 0x40L) != 0L)
-            return 88;
-         if ((active0 & 0xfff3fff801fffff0L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xffffffffffffffffL) != 0L || (active3 & 0x3ffff9f7fffffffL) != 0L || (active4 & 0xff7dffffdff80000L) != 0L || (active5 & 0x8719ffe0ffffffffL) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0xffffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xffffff9fffffffffL) != 0L || (active10 & 0x57eeffffffffffffL) != 0L || (active11 & 0x1bfL) != 0L)
+         if ((active0 & 0xfff3fff801fffff0L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xffffffffffffffffL) != 0L || (active3 & 0x3ffff9f7fffffffL) != 0L || (active4 & 0xff7dffffdff80000L) != 0L || (active5 & 0x8719ffe0ffffffffL) != 0L || (active6 & 0xffffffffffffffffL) != 0L || (active7 & 0xffffffffffffffffL) != 0L || (active8 & 0xffffffffffffffffL) != 0L || (active9 & 0xffffff9fffffffffL) != 0L || (active10 & 0x57eeffffffffffffL) != 0L || (active11 & 0xbfL) != 0L)
          {
             if (jjmatchedPos != 1)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 1;
             }
             return 88;
          }
+         if ((active11 & 0x8000000000000000L) != 0L || (active12 & 0x4L) != 0L)
+            return 82;
+         if ((active0 & 0x40007fe000000L) != 0L || (active3 & 0xfc00006000000000L) != 0L || (active4 & 0x200000007ffffL) != 0L || (active5 & 0x78e6001f00000000L) != 0L || (active9 & 0x6000000000L) != 0L || (active10 & 0xa811000000000000L) != 0L || (active11 & 0x40L) != 0L)
+            return 88;
          return -1;
       case 2:
-         if ((active0 & 0xfff3dfef79e98c80L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfe1ff2ffffff09ffL) != 0L || (active3 & 0xfbfeffd77f87ff9fL) != 0L || (active4 & 0x3e7dfffa1fff40ffL) != 0L || (active5 & 0xf795ffeebffe03ffL) != 0L || (active6 & 0xf8fffffdc07fff78L) != 0L || (active7 & 0x1ffffffc3fffffffL) != 0L || (active8 & 0xfffL) != 0L || (active9 & 0xffffffdffff7fffeL) != 0L || (active10 & 0xfffef7ffffffffffL) != 0L || (active11 & 0x1ffL) != 0L)
+         if ((active0 & 0xfff3dfef79e98c80L) != 0L || (active1 & 0xffffffffffffffffL) != 0L || (active2 & 0xfe1ff2ffffff09ffL) != 0L || (active3 & 0xfbfeffd77f87ff9fL) != 0L || (active4 & 0x3e7dfffa1fff40ffL) != 0L || (active5 & 0xf795ffeebffe03ffL) != 0L || (active6 & 0xf8fffffdc07fff78L) != 0L || (active7 & 0x1ffffffc3fffffffL) != 0L || (active8 & 0xfffL) != 0L || (active9 & 0xffffffdffff7fffeL) != 0L || (active10 & 0xfffef7ffffffffffL) != 0L || (active11 & 0xffL) != 0L)
          {
             if (jjmatchedPos != 2)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 2;
             }
             return 88;
@@ -10631,26 +10627,26 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 3)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 3;
             }
             return 88;
          }
-         if ((active0 & 0x6631400000000000L) != 0L || (active1 & 0x83000000023feL) != 0L || (active2 & 0x51001e00005f0L) != 0L || (active3 & 0x680401c00000cL) != 0L || (active4 & 0xc7601ff82000L) != 0L || (active5 & 0x190078280c80000L) != 0L || (active6 & 0x78080100300078L) != 0L || (active7 & 0x4014000200800000L) != 0L || (active8 & 0x59L) != 0L || (active9 & 0x9c0000ff0000002L) != 0L || (active10 & 0x100f1e3c002f800L) != 0L || (active11 & 0x106L) != 0L)
+         if ((active0 & 0x6631400000000000L) != 0L || (active1 & 0x83000000023feL) != 0L || (active2 & 0x51001e00005f0L) != 0L || (active3 & 0x680401c00000cL) != 0L || (active4 & 0xc7601ff82000L) != 0L || (active5 & 0x190078280c80000L) != 0L || (active6 & 0x78080100300078L) != 0L || (active7 & 0x4014000200800000L) != 0L || (active8 & 0x59L) != 0L || (active9 & 0x9c0000ff0000002L) != 0L || (active10 & 0x100f1e3c002f800L) != 0L || (active11 & 0x6L) != 0L)
             return 88;
          return -1;
       case 4:
+         if ((active2 & 0x40000000000000L) != 0L)
+            return 90;
          if ((active0 & 0xd9e29e2f780120e0L) != 0L || (active1 & 0xfff3affffffe9bfcL) != 0L || (active2 & 0xd592e2ffd3ffe9a7L) != 0L || (active3 & 0xfbd86017637413dbL) != 0L || (active4 & 0x8020aac99fc75ef1L) != 0L || (active5 & 0xc605fb2c2711bbfbL) != 0L || (active6 & 0xfe83d7dcbf4fff74L) != 0L || (active7 & 0x1febf0ddbf1ffeffL) != 0L || (active8 & 0xffffffffffffeea6L) != 0L || (active9 & 0xf6bdffdf88f7f7e1L) != 0L || (active10 & 0xf5de02da3f80f2fcL) != 0L || (active11 & 0x39L) != 0L)
          {
             if (jjmatchedPos != 4)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 4;
             }
             return 88;
          }
-         if ((active2 & 0x40000000000000L) != 0L)
-            return 90;
          if ((active0 & 0x1c001e8cc00L) != 0L || (active1 & 0x4400000014401L) != 0L || (active2 & 0x2a0800000c000008L) != 0L || (active3 & 0x241f800083ec00L) != 0L || (active4 & 0x3e5d10120000000eL) != 0L || (active5 & 0x3000044018a60004L) != 0L || (active6 & 0x24202040000002L) != 0L || (active7 & 0x80000f2000600100L) != 0L || (active8 & 0x100L) != 0L || (active9 & 0x200000700081cL) != 0L || (active10 & 0xa202404007d0503L) != 0L || (active11 & 0xc0L) != 0L)
             return 88;
          return -1;
@@ -10661,7 +10657,7 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 5)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 5;
             }
             return 88;
@@ -10672,17 +10668,17 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
       case 6:
          if ((active2 & 0x40000000000000L) != 0L)
             return 90;
+         if ((active0 & 0xd982840000000000L) != 0L || (active1 & 0xffe0080700420000L) != 0L || (active2 & 0x2e0001063c001L) != 0L || (active3 & 0x8800010100600249L) != 0L || (active4 & 0xc20228000024201L) != 0L || (active5 & 0x400100024001000L) != 0L || (active6 & 0x80021204af448020L) != 0L || (active7 & 0x180180043c0806a0L) != 0L || (active8 & 0x40002L) != 0L || (active9 & 0x103c001000000000L) != 0L || (active10 & 0x400402803c800038L) != 0L || (active11 & 0x30L) != 0L)
+            return 88;
          if ((active0 & 0x40038e79c02040L) != 0L || (active1 & 0x11a7f8f73c9bfcL) != 0L || (active2 & 0x8190007fc20c09a4L) != 0L || (active3 & 0x71c84c0623048102L) != 0L || (active4 & 0xb01880499fc41cd0L) != 0L || (active5 & 0x6000c12c0310a3b9L) != 0L || (active6 & 0x7e80c5d800097f50L) != 0L || (active7 & 0x5e20e898107c85fL) != 0L || (active8 & 0xfffffffffffbee20L) != 0L || (active9 & 0xe281ffcf8ed7b6d1L) != 0L || (active10 & 0xb7fa000a0340f000L) != 0L || (active11 & 0x9L) != 0L)
          {
             if (jjmatchedPos != 6)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 6;
             }
             return 88;
          }
-         if ((active0 & 0xd982840000000000L) != 0L || (active1 & 0xffe0080700420000L) != 0L || (active2 & 0x2e0001063c001L) != 0L || (active3 & 0x8800010100600249L) != 0L || (active4 & 0xc20228000024201L) != 0L || (active5 & 0x400100024001000L) != 0L || (active6 & 0x80021204af448020L) != 0L || (active7 & 0x180180043c0806a0L) != 0L || (active8 & 0x40002L) != 0L || (active9 & 0x103c001000000000L) != 0L || (active10 & 0x400402803c800038L) != 0L || (active11 & 0x30L) != 0L)
-            return 88;
          return -1;
       case 7:
          if ((active2 & 0x40000000000000L) != 0L)
@@ -10691,7 +10687,7 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 7)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 7;
             }
             return 88;
@@ -10700,37 +10696,37 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
             return 88;
          return -1;
       case 8:
-         if ((active0 & 0x20610400000L) != 0L || (active1 & 0x108000f03c03f8L) != 0L || (active2 & 0x8080000000000000L) != 0L || (active3 & 0x70c0400202040002L) != 0L || (active4 & 0x18000000040c01L) != 0L || (active5 & 0x2c00000000L) != 0L || (active6 & 0x1880818000017f00L) != 0L || (active7 & 0x60000880040000L) != 0L || (active8 & 0x300004000000L) != 0L || (active9 & 0x2000f00388130410L) != 0L || (active10 & 0x8410000002000000L) != 0L)
-            return 88;
          if ((active0 & 0x1040018869800000L) != 0L || (active1 & 0xffc121fe07001804L) != 0L || (active2 & 0x100c01782040985L) != 0L || (active3 & 0x100040001008000L) != 0L || (active4 & 0x300080409fc00090L) != 0L || (active5 & 0x4000c000010003b8L) != 0L || (active6 & 0x660040100e080040L) != 0L || (active7 & 0x15000e813803c041L) != 0L || (active8 & 0xfffacffffb13ac00L) != 0L || (active9 & 0xc0b90f4c06c0b2c1L) != 0L || (active10 & 0x23e800082000f000L) != 0L || (active11 & 0x8L) != 0L)
          {
             if (jjmatchedPos != 8)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 8;
             }
             return 88;
          }
+         if ((active0 & 0x20610400000L) != 0L || (active1 & 0x108000f03c03f8L) != 0L || (active2 & 0x8080000000000000L) != 0L || (active3 & 0x70c0400202040002L) != 0L || (active4 & 0x18000000040c01L) != 0L || (active5 & 0x2c00000000L) != 0L || (active6 & 0x1880818000017f00L) != 0L || (active7 & 0x60000880040000L) != 0L || (active8 & 0x300004000000L) != 0L || (active9 & 0x2000f00388130410L) != 0L || (active10 & 0x8410000002000000L) != 0L)
+            return 88;
          return -1;
       case 9:
-         if ((active0 & 0x468000000L) != 0L || (active1 & 0x100fe00000a00L) != 0L || (active2 & 0x1102040800L) != 0L || (active4 & 0x804098c00000L) != 0L || (active5 & 0x4000000801000200L) != 0L || (active6 & 0x2200400000000040L) != 0L || (active7 & 0x40100000041L) != 0L || (active8 & 0x2038000L) != 0L || (active9 & 0x80200002008240L) != 0L || (active10 & 0x280000020000000L) != 0L || (active11 & 0x8L) != 0L)
-            return 88;
          if ((active0 & 0x1040018801800000L) != 0L || (active1 & 0xffc02100c73811f4L) != 0L || (active2 & 0x100c00680000185L) != 0L || (active3 & 0x180040001008000L) != 0L || (active4 & 0x3010000007000890L) != 0L || (active5 & 0xc000000001b8L) != 0L || (active6 & 0x440000100e087e00L) != 0L || (active7 & 0x15000a803803c000L) != 0L || (active8 & 0xfffacffff9102c00L) != 0L || (active9 & 0xc0398f4f04c23081L) != 0L || (active10 & 0x216800080000f000L) != 0L)
          {
             if (jjmatchedPos != 9)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 9;
             }
             return 88;
          }
+         if ((active0 & 0x468000000L) != 0L || (active1 & 0x100fe00000a00L) != 0L || (active2 & 0x1102040800L) != 0L || (active4 & 0x804098c00000L) != 0L || (active5 & 0x4000000801000200L) != 0L || (active6 & 0x2200400000000040L) != 0L || (active7 & 0x40100000041L) != 0L || (active8 & 0x2038000L) != 0L || (active9 & 0x80200002008240L) != 0L || (active10 & 0x280000020000000L) != 0L || (active11 & 0x8L) != 0L)
+            return 88;
          return -1;
       case 10:
          if ((active0 & 0x1000010801800000L) != 0L || (active1 & 0xf7c02074863811f0L) != 0L || (active2 & 0x100c00200000181L) != 0L || (active3 & 0x100040000000000L) != 0L || (active4 & 0x3010000000800880L) != 0L || (active5 & 0xc00000000038L) != 0L || (active6 & 0x400000100e007e00L) != 0L || (active7 & 0x11000a003803c000L) != 0L || (active8 & 0xffca8ffff0002c00L) != 0L || (active9 & 0x8038804f00021000L) != 0L || (active10 & 0x2800080000f000L) != 0L)
          {
             if (jjmatchedPos != 10)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 10;
             }
             return 88;
@@ -10739,24 +10735,24 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
             return 88;
          return -1;
       case 11:
-         if ((active0 & 0x1000000000000000L) != 0L || (active1 & 0x6680000000001000L) != 0L || (active2 & 0x1L) != 0L || (active3 & 0x40000000000L) != 0L || (active4 & 0x1000000000000880L) != 0L || (active5 & 0x800000000020L) != 0L || (active6 & 0x8000000L) != 0L || (active7 & 0x1100080010000000L) != 0L || (active8 & 0x6082800000002800L) != 0L || (active9 & 0x10000100000000L) != 0L || (active10 & 0x8000800000000L) != 0L)
-            return 88;
          if ((active0 & 0x10801800000L) != 0L || (active1 & 0x91402074863801f0L) != 0L || (active2 & 0x100c00200000180L) != 0L || (active3 & 0x100000000000000L) != 0L || (active4 & 0x2010000004800000L) != 0L || (active5 & 0x400000000018L) != 0L || (active6 & 0x4000001006007e00L) != 0L || (active7 & 0x2002803c000L) != 0L || (active8 & 0x9f480ffff0000400L) != 0L || (active9 & 0x80288e4e00021000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
             if (jjmatchedPos != 11)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 11;
             }
             return 88;
          }
+         if ((active0 & 0x1000000000000000L) != 0L || (active1 & 0x6680000000001000L) != 0L || (active2 & 0x1L) != 0L || (active3 & 0x40000000000L) != 0L || (active4 & 0x1000000000000880L) != 0L || (active5 & 0x800000000020L) != 0L || (active6 & 0x8000000L) != 0L || (active7 & 0x1100080010000000L) != 0L || (active8 & 0x6082800000002800L) != 0L || (active9 & 0x10000100000000L) != 0L || (active10 & 0x8000800000000L) != 0L)
+            return 88;
          return -1;
       case 12:
          if ((active0 & 0x800000000L) != 0L || (active1 & 0x200000000000L) != 0L || (active2 & 0x100000200000000L) != 0L || (active4 & 0x800000L) != 0L || (active8 & 0x8408000000000400L) != 0L || (active9 & 0x8000000600000000L) != 0L)
             return 88;
          if ((active0 & 0x10001800000L) != 0L || (active1 & 0xd1400074863801f0L) != 0L || (active2 & 0xc00000000180L) != 0L || (active3 & 0x100000000000000L) != 0L || (active4 & 0x2010000004000000L) != 0L || (active5 & 0x400000000018L) != 0L || (active6 & 0x4000001006007e00L) != 0L || (active7 & 0x2002803c000L) != 0L || (active8 & 0x1b400ffff0000000L) != 0L || (active9 & 0x288e4800021000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 12;
             return 88;
          }
@@ -10764,7 +10760,7 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
       case 13:
          if ((active0 & 0x10001800000L) != 0L || (active1 & 0xc1400074862801f0L) != 0L || (active2 & 0xc00000000180L) != 0L || (active4 & 0x2000000000000000L) != 0L || (active5 & 0x400000000010L) != 0L || (active6 & 0x6007800L) != 0L || (active7 & 0x803c000L) != 0L || (active8 & 0x9400ffff0000000L) != 0L || (active9 & 0x88e4800021000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 13;
             return 88;
          }
@@ -10774,7 +10770,7 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
       case 14:
          if ((active0 & 0x1800000L) != 0L || (active1 & 0xc1000050862800f0L) != 0L || (active2 & 0xc00000000180L) != 0L || (active5 & 0x400000000010L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x34000L) != 0L || (active8 & 0x1400bfff0000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 14;
             return 88;
          }
@@ -10782,35 +10778,35 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
             return 88;
          return -1;
       case 15:
+         if ((active0 & 0x800000L) != 0L || (active1 & 0x86200010L) != 0L || (active2 & 0xc00000000000L) != 0L || (active8 & 0x800f0000000L) != 0L)
+            return 88;
          if ((active0 & 0x1000000L) != 0L || (active1 & 0xc1000050000800e0L) != 0L || (active2 & 0x180L) != 0L || (active5 & 0x400000000010L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x34000L) != 0L || (active8 & 0x14003ff00000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
             if (jjmatchedPos != 15)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 15;
             }
             return 88;
          }
-         if ((active0 & 0x800000L) != 0L || (active1 & 0x86200010L) != 0L || (active2 & 0xc00000000000L) != 0L || (active8 & 0x800f0000000L) != 0L)
-            return 88;
          return -1;
       case 16:
+         if ((active1 & 0x4000004000080000L) != 0L || (active5 & 0x400000000000L) != 0L || (active7 & 0x20000L) != 0L || (active8 & 0x38700000000L) != 0L)
+            return 88;
          if ((active0 & 0x1000000L) != 0L || (active1 & 0x81000010040000e0L) != 0L || (active2 & 0x800000000180L) != 0L || (active5 & 0x10L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x14000L) != 0L || (active8 & 0x1400078e0000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
             if (jjmatchedPos != 16)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 16;
             }
             return 88;
          }
-         if ((active1 & 0x4000004000080000L) != 0L || (active5 & 0x400000000000L) != 0L || (active7 & 0x20000L) != 0L || (active8 & 0x38700000000L) != 0L)
-            return 88;
          return -1;
       case 17:
          if ((active0 & 0x1000000L) != 0L || (active1 & 0x81000000040000a0L) != 0L || (active2 & 0x800000000180L) != 0L || (active5 & 0x10L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x14000L) != 0L || (active8 & 0x140015ee0000000L) != 0L || (active9 & 0xe0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 17;
             return 88;
          }
@@ -10822,7 +10818,7 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
          {
             if (jjmatchedPos != 18)
             {
-               jjmatchedKind = 781;
+               jjmatchedKind = 780;
                jjmatchedPos = 18;
             }
             return 88;
@@ -10831,19 +10827,19 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
             return 88;
          return -1;
       case 19:
-         if ((active1 & 0x80L) != 0L || (active5 & 0x10L) != 0L || (active7 & 0x4000L) != 0L)
-            return 88;
          if ((active0 & 0x1000000L) != 0L || (active1 & 0x8100000004000020L) != 0L || (active2 & 0x800000000180L) != 0L || (active6 & 0x7800L) != 0L || (active7 & 0x10000L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0xc0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 19;
             return 88;
          }
+         if ((active1 & 0x80L) != 0L || (active5 & 0x10L) != 0L || (active7 & 0x4000L) != 0L)
+            return 88;
          return -1;
       case 20:
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x180L) != 0L || (active6 & 0x7800L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0xc0000000000L) != 0L || (active10 & 0x2000000000f000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 20;
             return 88;
          }
@@ -10851,21 +10847,21 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
             return 88;
          return -1;
       case 21:
-         if ((active2 & 0x80L) != 0L || (active9 & 0x40000000000L) != 0L || (active10 & 0x6000L) != 0L)
-            return 88;
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x7800L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0x80000000000L) != 0L || (active10 & 0x20000000009000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 21;
             return 88;
          }
+         if ((active2 & 0x80L) != 0L || (active9 & 0x40000000000L) != 0L || (active10 & 0x6000L) != 0L)
+            return 88;
          return -1;
       case 22:
          if ((active6 & 0x2000L) != 0L)
             return 88;
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x5800L) != 0L || (active8 & 0x116e0000000L) != 0L || (active9 & 0x80000000000L) != 0L || (active10 & 0x20000000009000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 22;
             return 88;
          }
@@ -10873,7 +10869,7 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
       case 23:
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x5800L) != 0L || (active8 & 0x116c0000000L) != 0L || (active10 & 0x20000000001000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 23;
             return 88;
          }
@@ -10883,7 +10879,7 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
       case 24:
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active6 & 0x1800L) != 0L || (active8 & 0x116c0000000L) != 0L || (active10 & 0x20000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 24;
             return 88;
          }
@@ -10891,29 +10887,29 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
             return 88;
          return -1;
       case 25:
+         if ((active6 & 0x1800L) != 0L || (active8 & 0x100c0000000L) != 0L || (active10 & 0x20000000000000L) != 0L)
+            return 88;
          if ((active1 & 0x8100000000000000L) != 0L || (active2 & 0x100L) != 0L || (active8 & 0x1600000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 25;
             return 88;
          }
-         if ((active6 & 0x1800L) != 0L || (active8 & 0x100c0000000L) != 0L || (active10 & 0x20000000000000L) != 0L)
-            return 88;
          return -1;
       case 26:
+         if ((active2 & 0x100L) != 0L || (active8 & 0x600000000L) != 0L)
+            return 88;
          if ((active1 & 0x8100000000000000L) != 0L || (active8 & 0x1000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 26;
             return 88;
          }
-         if ((active2 & 0x100L) != 0L || (active8 & 0x600000000L) != 0L)
-            return 88;
          return -1;
       case 27:
          if ((active1 & 0x8100000000000000L) != 0L || (active8 & 0x1000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 27;
             return 88;
          }
@@ -10921,7 +10917,7 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
       case 28:
          if ((active1 & 0x8100000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 28;
             return 88;
          }
@@ -10931,7 +10927,7 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
       case 29:
          if ((active1 & 0x8100000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 29;
             return 88;
          }
@@ -10939,7 +10935,7 @@ private final int jjStopStringLiteralDfa_2(int pos, long active0, long active1,
       case 30:
          if ((active1 & 0x8000000000000000L) != 0L)
          {
-            jjmatchedKind = 781;
+            jjmatchedKind = 780;
             jjmatchedPos = 30;
             return 88;
          }
@@ -10968,56 +10964,56 @@ private final int jjMoveStringLiteralDfa0_2()
    {
       case 33:
          jjmatchedKind = 1;
-         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x100000000000L, 0x0L);
+         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x80000000000L, 0x0L);
       case 34:
-         return jjStopAtPos(0, 758);
+         return jjStopAtPos(0, 757);
       case 36:
-         return jjStartNfaWithStates_2(0, 761, 88);
+         return jjStartNfaWithStates_2(0, 760, 88);
       case 37:
-         return jjStopAtPos(0, 753);
+         return jjStopAtPos(0, 752);
       case 39:
-         return jjStartNfaWithStates_2(0, 757, 55);
+         return jjStartNfaWithStates_2(0, 756, 55);
       case 40:
-         return jjStopAtPos(0, 727);
+         return jjStopAtPos(0, 726);
       case 41:
-         return jjStopAtPos(0, 728);
+         return jjStopAtPos(0, 727);
       case 42:
-         jjmatchedKind = 751;
-         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x2L);
+         jjmatchedKind = 750;
+         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x1L);
       case 43:
-         return jjStopAtPos(0, 749);
+         return jjStopAtPos(0, 748);
       case 44:
-         return jjStopAtPos(0, 739);
+         return jjStopAtPos(0, 738);
       case 45:
-         return jjStartNfaWithStates_2(0, 750, 15);
+         return jjStartNfaWithStates_2(0, 749, 15);
       case 46:
-         jjmatchedKind = 738;
-         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x10000000000000L, 0x0L);
+         jjmatchedKind = 737;
+         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x8000000000000L, 0x0L);
       case 47:
-         jjmatchedKind = 752;
-         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x9L);
+         jjmatchedKind = 751;
+         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x8000000000000000L, 0x4L);
       case 58:
-         jjmatchedKind = 744;
-         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x400000000000000L, 0x0L);
+         jjmatchedKind = 743;
+         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x200000000000000L, 0x0L);
       case 59:
-         return jjStopAtPos(0, 737);
+         return jjStopAtPos(0, 736);
       case 60:
-         jjmatchedKind = 742;
-         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xa0000000000L, 0x0L);
+         jjmatchedKind = 741;
+         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x50000000000L, 0x0L);
       case 61:
-         jjmatchedKind = 740;
-         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x8000000000000L, 0x0L);
+         jjmatchedKind = 739;
+         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x4000000000000L, 0x0L);
       case 62:
-         jjmatchedKind = 741;
-         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x40000000000L, 0x0L);
+         jjmatchedKind = 740;
+         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x20000000000L, 0x0L);
       case 63:
-         return jjStopAtPos(0, 743);
+         return jjStopAtPos(0, 742);
       case 91:
-         return jjStopAtPos(0, 735);
+         return jjStopAtPos(0, 734);
       case 93:
-         return jjStopAtPos(0, 736);
+         return jjStopAtPos(0, 735);
       case 94:
-         return jjStopAtPos(0, 760);
+         return jjStopAtPos(0, 759);
       case 65:
       case 97:
          jjmatchedKind = 3;
@@ -11085,7 +11081,7 @@ private final int jjMoveStringLiteralDfa0_2()
          return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xfffffffff000000L, 0x2000000000000L, 0x0L, 0x0L);
       case 85:
       case 117:
-         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xf000000000000000L, 0xfffffL, 0x100L, 0x0L);
+         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0xf000000000000000L, 0xfffffL, 0x0L, 0x0L);
       case 86:
       case 118:
          return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x20000007ff00000L, 0x0L, 0x0L);
@@ -11102,12 +11098,12 @@ private final int jjMoveStringLiteralDfa0_2()
       case 122:
          return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x400000000000L, 0x0L, 0x0L);
       case 123:
-         return jjStartNfaWithStates_2(0, 733, 87);
+         return jjStartNfaWithStates_2(0, 732, 86);
       case 124:
-         jjmatchedKind = 759;
-         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x4000000000000L, 0x0L);
+         jjmatchedKind = 758;
+         return jjMoveStringLiteralDfa1_2(0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x0L, 0x2000000000000L, 0x0L);
       case 125:
-         return jjStopAtPos(0, 734);
+         return jjStopAtPos(0, 733);
       case 126:
          return jjStopAtPos(0, 2);
       default :
@@ -11124,37 +11120,37 @@ private final int jjMoveStringLiteralDfa1_2(long active0, long active1, long act
    switch(curChar)
    {
       case 42:
-         if ((active12 & 0x8L) != 0L)
+         if ((active12 & 0x4L) != 0L)
          {
-            jjmatchedKind = 771;
+            jjmatchedKind = 770;
             jjmatchedPos = 1;
          }
-         return jjMoveStringLiteralDfa2_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0x1L);
+         return jjMoveStringLiteralDfa2_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0x8000000000000000L, active12, 0L);
       case 46:
-         if ((active11 & 0x10000000000000L) != 0L)
-            return jjStopAtPos(1, 756);
+         if ((active11 & 0x8000000000000L) != 0L)
+            return jjStopAtPos(1, 755);
          break;
       case 47:
-         if ((active12 & 0x2L) != 0L)
-            return jjStopAtPos(1, 769);
+         if ((active12 & 0x1L) != 0L)
+            return jjStopAtPos(1, 768);
          break;
       case 58:
-         if ((active11 & 0x400000000000000L) != 0L)
-            return jjStopAtPos(1, 762);
+         if ((active11 & 0x200000000000000L) != 0L)
+            return jjStopAtPos(1, 761);
          break;
       case 61:
-         if ((active11 & 0x20000000000L) != 0L)
+         if ((active11 & 0x10000000000L) != 0L)
+            return jjStopAtPos(1, 744);
+         else if ((active11 & 0x20000000000L) != 0L)
             return jjStopAtPos(1, 745);
-         else if ((active11 & 0x40000000000L) != 0L)
-            return jjStopAtPos(1, 746);
-         else if ((active11 & 0x100000000000L) != 0L)
-            return jjStopAtPos(1, 748);
+         else if ((active11 & 0x80000000000L) != 0L)
+            return jjStopAtPos(1, 747);
          break;
       case 62:
-         if ((active11 & 0x80000000000L) != 0L)
-            return jjStopAtPos(1, 747);
-         else if ((active11 & 0x8000000000000L) != 0L)
-            return jjStopAtPos(1, 755);
+         if ((active11 & 0x40000000000L) != 0L)
+            return jjStopAtPos(1, 746);
+         else if ((active11 & 0x4000000000000L) != 0L)
+            return jjStopAtPos(1, 754);
          break;
       case 65:
       case 97:
@@ -11269,7 +11265,7 @@ private final int jjMoveStringLiteralDfa1_2(long active0, long active1, long act
          return jjMoveStringLiteralDfa2_2(active0, 0x700000000L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x8000006000000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0xffcL, active10, 0x100000000e0000L, active11, 0L, active12, 0L);
       case 85:
       case 117:
-         return jjMoveStringLiteralDfa2_2(active0, 0x800000000L, active1, 0xfff8000000000000L, active2, 0x7L, active3, 0x70000000L, active4, 0L, active5, 0x1f8000300000L, active6, 0x3000000000007L, active7, 0x400000000L, active8, 0L, active9, 0x4000000000ff000L, active10, 0L, active11, 0x180L, active12, 0L);
+         return jjMoveStringLiteralDfa2_2(active0, 0x800000000L, active1, 0xfff8000000000000L, active2, 0x7L, active3, 0x70000000L, active4, 0L, active5, 0x1f8000300000L, active6, 0x3000000000007L, active7, 0x400000000L, active8, 0L, active9, 0x4000000000ff000L, active10, 0L, active11, 0x80L, active12, 0L);
       case 86:
       case 118:
          return jjMoveStringLiteralDfa2_2(active0, 0x1000000000L, active1, 0L, active2, 0x2000000000000000L, active3, 0L, active4, 0L, active5, 0L, active6, 0x78L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
@@ -11282,8 +11278,8 @@ private final int jjMoveStringLiteralDfa1_2(long active0, long active1, long act
             return jjStartNfaWithStates_2(1, 50, 88);
          return jjMoveStringLiteralDfa2_2(active0, 0L, active1, 0L, active2, 0xe00000000008L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0x800000000f00000L, active10, 0L, active11, 0L, active12, 0L);
       case 124:
-         if ((active11 & 0x4000000000000L) != 0L)
-            return jjStopAtPos(1, 754);
+         if ((active11 & 0x2000000000000L) != 0L)
+            return jjStopAtPos(1, 753);
          break;
       default :
          break;
@@ -11296,23 +11292,23 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
       return jjStartNfa_2(0, old0, old1, old2, old3, old4, old5, old6, old7, old8, old9, old10, old11, old12); 
    try { curChar = input_stream.readChar(); }
    catch(java.io.IOException e) {
-      jjStopStringLiteralDfa_2(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, active12);
+      jjStopStringLiteralDfa_2(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, 0L);
       return 2;
    }
    switch(curChar)
    {
       case 43:
-         if ((active12 & 0x1L) != 0L)
-            return jjStopAtPos(2, 768);
+         if ((active11 & 0x8000000000000000L) != 0L)
+            return jjStopAtPos(2, 767);
          break;
       case 65:
       case 97:
          if ((active0 & 0x100L) != 0L)
             return jjStartNfaWithStates_2(2, 8, 88);
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0x9bffL, active2, 0x400000800L, active3, 0x18003000000L, active4, 0x30000000000L, active5, 0x1L, active6, 0x72000180000000L, active7, 0x203000000000L, active8, 0x120L, active9, 0x1ff800000003cL, active10, 0x320000000400L, active11, 0x4L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0x9bffL, active2, 0x400000800L, active3, 0x18003000000L, active4, 0x30000000000L, active5, 0x1L, active6, 0x72000180000000L, active7, 0x203000000000L, active8, 0x120L, active9, 0x1ff800000003cL, active10, 0x320000000400L, active11, 0x4L);
       case 66:
       case 98:
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0x8000000010000L, active2, 0L, active3, 0L, active4, 0x200200000000L, active5, 0L, active6, 0x1000000000000L, active7, 0L, active8, 0L, active9, 0x200000000703f000L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0x8000000010000L, active2, 0L, active3, 0L, active4, 0x200200000000L, active5, 0L, active6, 0x1000000000000L, active7, 0L, active8, 0L, active9, 0x200000000703f000L, active10, 0L, active11, 0L);
       case 67:
       case 99:
          if ((active0 & 0x4000000L) != 0L)
@@ -11322,7 +11318,7 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             jjmatchedKind = 140;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0xd01100000000e008L, active3, 0x3800000000000003L, active4, 0x3c000000000000L, active5, 0x400000000080L, active6, 0x80000000000000L, active7, 0x3c00000000000L, active8, 0L, active9, 0xc000000000040000L, active10, 0x4c4000000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0xd01100000000e008L, active3, 0x3800000000000003L, active4, 0x3c000000000000L, active5, 0x400000000080L, active6, 0x80000000000000L, active7, 0x3c00000000000L, active8, 0L, active9, 0xc000000000040000L, active10, 0x4c4000000000000L, active11, 0L);
       case 68:
       case 100:
          if ((active0 & 0x200L) != 0L)
@@ -11343,14 +11339,14 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             return jjStartNfaWithStates_2(2, 371, 88);
          else if ((active6 & 0x80L) != 0L)
             return jjStartNfaWithStates_2(2, 391, 88);
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0x1c0000000000000L, active3, 0x4000000000000000L, active4, 0L, active5, 0x7000000000018000L, active6, 0L, active7, 0L, active8, 0L, active9, 0xc0L, active10, 0x800000800000081L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0x1c0000000000000L, active3, 0x4000000000000000L, active4, 0L, active5, 0x7000000000018000L, active6, 0L, active7, 0L, active8, 0L, active9, 0xc0L, active10, 0x800000800000081L, active11, 0L);
       case 69:
       case 101:
          if ((active0 & 0x100000L) != 0L)
             return jjStartNfaWithStates_2(2, 20, 88);
          else if ((active5 & 0x40000000000000L) != 0L)
             return jjStartNfaWithStates_2(2, 374, 88);
-         return jjMoveStringLiteralDfa3_2(active0, 0x2000008000000L, active1, 0x2000000000400L, active2, 0x2002000000000000L, active3, 0x840000400000cL, active4, 0L, active5, 0x100000000000000L, active6, 0xfc000000078L, active7, 0x4000000000000L, active8, 0xe00L, active9, 0x2000030000000L, active10, 0x7c000f800L, active11, 0x80L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x2000008000000L, active1, 0x2000000000400L, active2, 0x2002000000000000L, active3, 0x840000400000cL, active4, 0L, active5, 0x100000000000000L, active6, 0xfc000000078L, active7, 0x4000000000000L, active8, 0xe00L, active9, 0x2000030000000L, active10, 0x7c000f800L, active11, 0x80L);
       case 70:
       case 102:
          if ((active6 & 0x100000000000000L) != 0L)
@@ -11358,28 +11354,28 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             jjmatchedKind = 440;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_2(active0, 0x2000000000L, active1, 0L, active2, 0x7f0000L, active3, 0L, active4, 0x40000000000L, active5, 0x4000000000000L, active6, 0x600000000000000L, active7, 0L, active8, 0L, active9, 0L, active10, 0x80000000e0000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x2000000000L, active1, 0L, active2, 0x7f0000L, active3, 0L, active4, 0x40000000000L, active5, 0x4000000000000L, active6, 0x600000000000000L, active7, 0L, active8, 0L, active9, 0L, active10, 0x80000000e0000L, active11, 0L);
       case 71:
       case 103:
          if ((active0 & 0x1000000000L) != 0L)
             return jjStartNfaWithStates_2(2, 36, 88);
          else if ((active4 & 0x400000000L) != 0L)
             return jjStartNfaWithStates_2(2, 290, 88);
-         return jjMoveStringLiteralDfa3_2(active0, 0x9c000000000L, active1, 0L, active2, 0x800000L, active3, 0L, active4, 0L, active5, 0L, active6, 0xf800000000000000L, active7, 0x20000fL, active8, 0L, active9, 0L, active10, 0x4000000000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x9c000000000L, active1, 0L, active2, 0x800000L, active3, 0L, active4, 0L, active5, 0L, active6, 0xf800000000000000L, active7, 0x20000fL, active8, 0L, active9, 0L, active10, 0x4000000000000000L, active11, 0L);
       case 72:
       case 104:
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x8000002008000000L, active6, 0L, active7, 0xc000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x8000002008000000L, active6, 0L, active7, 0xc000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 73:
       case 105:
          if ((active6 & 0x200000000L) != 0L)
             return jjStartNfaWithStates_2(2, 417, 88);
-         return jjMoveStringLiteralDfa3_2(active0, 0x6000000000000000L, active1, 0L, active2, 0L, active3, 0x8020000000000010L, active4, 0x100001L, active5, 0x10004000000000L, active6, 0x700000000000L, active7, 0x400000L, active8, 0x10L, active9, 0xfc000000000000L, active10, 0x20040000010006L, active11, 0x100L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x6000000000000000L, active1, 0L, active2, 0L, active3, 0x8020000000000010L, active4, 0x100001L, active5, 0x10004000000000L, active6, 0x700000000000L, active7, 0x400000L, active8, 0x10L, active9, 0xfc000000000000L, active10, 0x20040000010006L, active11, 0L);
       case 74:
       case 106:
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x200000000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0x200000000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 75:
       case 107:
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0xc00000000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0x8L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0xc00000000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0x8L, active11, 0L);
       case 76:
       case 108:
          if ((active0 & 0x1000L) != 0L)
@@ -11394,12 +11390,12 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
          }
          else if ((active10 & 0x80000000000L) != 0L)
             return jjStartNfaWithStates_2(2, 683, 88);
-         return jjMoveStringLiteralDfa3_2(active0, 0x30000000006000L, active1, 0x1fe0000L, active2, 0x1000000L, active3, 0x800010041400L, active4, 0L, active5, 0x80078010100300L, active6, 0L, active7, 0x18000003800030L, active8, 0xffffffffffffe000L, active9, 0x1L, active10, 0xa200000000700000L, active11, 0x2L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x30000000006000L, active1, 0x1fe0000L, active2, 0x1000000L, active3, 0x800010041400L, active4, 0L, active5, 0x80078010100300L, active6, 0L, active7, 0x18000003800030L, active8, 0xffffffffffffe000L, active9, 0x1L, active10, 0xa200000000700000L, active11, 0x2L);
       case 77:
       case 109:
          if ((active9 & 0x80000L) != 0L)
             return jjStartNfaWithStates_2(2, 595, 88);
-         return jjMoveStringLiteralDfa3_2(active0, 0x400L, active1, 0x1000001e000000L, active2, 0x8000000000L, active3, 0xc0000000000000L, active4, 0x1000000000000L, active5, 0x180000e00002L, active6, 0L, active7, 0L, active8, 0x46L, active9, 0x400000fc8100000L, active10, 0x2800000000000L, active11, 0x20L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x400L, active1, 0x1000001e000000L, active2, 0x8000000000L, active3, 0xc0000000000000L, active4, 0x1000000000000L, active5, 0x180000e00002L, active6, 0L, active7, 0L, active8, 0x46L, active9, 0x400000fc8100000L, active10, 0x2800000000000L, active11, 0x20L);
       case 78:
       case 110:
          if ((active5 & 0x400L) != 0L)
@@ -11407,10 +11403,10 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             jjmatchedKind = 330;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_2(active0, 0x8000100000000000L, active1, 0xfffe0000000L, active2, 0xe00002000000L, active3, 0x10000320002000L, active4, 0x80800000002L, active5, 0x201063800L, active6, 0xc000000000000L, active7, 0x8020000400000000L, active8, 0L, active9, 0x1000000000L, active10, 0x401000000060L, active11, 0x8L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x8000100000000000L, active1, 0xfffe0000000L, active2, 0xe00002000000L, active3, 0x10000320002000L, active4, 0x80800000002L, active5, 0x201063800L, active6, 0xc000000000000L, active7, 0x8020000400000000L, active8, 0L, active9, 0x1000000000L, active10, 0x401000000060L, active11, 0x8L);
       case 79:
       case 111:
-         return jjMoveStringLiteralDfa3_2(active0, 0xc00100000000L, active1, 0x4000000006000L, active2, 0x200100000000000L, active3, 0x1e1408030000L, active4, 0x1fe70004L, active5, 0L, active6, 0x800000000000L, active7, 0xf0000000000L, active8, 0x1L, active9, 0L, active10, 0x10000000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0xc00100000000L, active1, 0x4000000006000L, active2, 0x200100000000000L, active3, 0x1e1408030000L, active4, 0x1fe70004L, active5, 0L, active6, 0x800000000000L, active7, 0xf0000000000L, active8, 0x1L, active9, 0L, active10, 0x10000000000000L, active11, 0L);
       case 80:
       case 112:
          if ((active3 & 0x20L) != 0L)
@@ -11422,10 +11418,10 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             return jjStartNfaWithStates_2(2, 240, 88);
          else if ((active4 & 0x100000000000000L) != 0L)
             return jjStartNfaWithStates_2(2, 312, 88);
-         return jjMoveStringLiteralDfa3_2(active0, 0x80000L, active1, 0L, active2, 0x8000004000000L, active3, 0x300000000000040L, active4, 0x8L, active5, 0L, active6, 0L, active7, 0x400000000000c0L, active8, 0L, active9, 0x800004000000000L, active10, 0x110L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x80000L, active1, 0L, active2, 0x8000004000000L, active3, 0x300000000000040L, active4, 0x8L, active5, 0L, active6, 0L, active7, 0x400000000000c0L, active8, 0L, active9, 0x800004000000000L, active10, 0x110L, active11, 0L);
       case 81:
       case 113:
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0x80000000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0x80000000000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 82:
       case 114:
          if ((active3 & 0x80000L) != 0L)
@@ -11438,7 +11434,7 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             jjmatchedKind = 407;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_2(active0, 0x40020001e00000L, active1, 0xffe0300000000000L, active2, 0x800000018000007L, active3, 0x70c000L, active4, 0x1000000000L, active5, 0xc00080004L, active6, 0x43f01ff00L, active7, 0x700100000000000L, active8, 0L, active9, 0x702L, active10, 0x100001003f800000L, active11, 0x10L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x40020001e00000L, active1, 0xffe0300000000000L, active2, 0x800000018000007L, active3, 0x70c000L, active4, 0x1000000000L, active5, 0xc00080004L, active6, 0x43f01ff00L, active7, 0x700100000000000L, active8, 0L, active9, 0x702L, active10, 0x100001003f800000L, active11, 0x10L);
       case 83:
       case 115:
          if ((active0 & 0x10L) != 0L)
@@ -11446,7 +11442,7 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             jjmatchedKind = 4;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_2(active0, 0x780000030000060L, active1, 0L, active2, 0x40079e0000000L, active3, 0x40000000L, active4, 0x60000000f0L, active5, 0x20000038L, active6, 0x18001e0000L, active7, 0x1800000000001f00L, active8, 0L, active9, 0x1000000000e00000L, active10, 0x200L, active11, 0x1L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x780000030000060L, active1, 0L, active2, 0x40079e0000000L, active3, 0x40000000L, active4, 0x60000000f0L, active5, 0x20000038L, active6, 0x18001e0000L, active7, 0x1800000000001f00L, active8, 0L, active9, 0x1000000000e00000L, active10, 0x200L, active11, 0x1L);
       case 84:
       case 116:
          if ((active0 & 0x200000000000L) != 0L)
@@ -11472,13 +11468,13 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             jjmatchedKind = 509;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_2(active0, 0x1801040e00008880L, active1, 0L, active2, 0x2000001f0L, active3, 0x4000000b80L, active4, 0x3e00008000003e00L, active5, 0x601800006000040L, active6, 0x600006L, active7, 0x40000000000fe000L, active8, 0L, active9, 0L, active10, 0x10000e000000000L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x1801040e00008880L, active1, 0L, active2, 0x2000001f0L, active3, 0x4000000b80L, active4, 0x3e00008000003e00L, active5, 0x601800006000040L, active6, 0x600006L, active7, 0x40000000000fe000L, active8, 0L, active9, 0L, active10, 0x10000e000000000L, active11, 0L);
       case 85:
       case 117:
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0x400000000000L, active2, 0x400020000000000L, active3, 0x6000000800000L, active4, 0L, active5, 0L, active6, 0L, active7, 0x3c000000L, active8, 0x80L, active9, 0x300000000000000L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0x400000000000L, active2, 0x400020000000000L, active3, 0x6000000800000L, active4, 0L, active5, 0L, active6, 0L, active7, 0x3c000000L, active8, 0x80L, active9, 0x300000000000000L, active10, 0L, active11, 0L);
       case 86:
       case 118:
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0x1800000000000L, active2, 0L, active3, 0x200000000000L, active4, 0x100000084000L, active5, 0L, active6, 0x40000000L, active7, 0x800100000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0x1800000000000L, active2, 0L, active3, 0x200000000000L, active4, 0x100000084000L, active5, 0L, active6, 0x40000000L, active7, 0x800100000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 87:
       case 119:
          if ((active2 & 0x40000000000L) != 0L)
@@ -11490,7 +11486,7 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             jjmatchedKind = 478;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_2(active0, 0x10000L, active1, 0L, active2, 0L, active3, 0L, active4, 0x40000000000000L, active5, 0L, active6, 0x2000000000L, active7, 0x380000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x10000L, active1, 0L, active2, 0L, active3, 0L, active4, 0x40000000000000L, active5, 0L, active6, 0x2000000000L, active7, 0x380000000L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 88:
       case 120:
          if ((active4 & 0x4000000000000000L) != 0L)
@@ -11498,7 +11494,7 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             jjmatchedKind = 318;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0x8000000000000000L, active5, 0x80000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0x8000000000000000L, active5, 0x80000000L, active6, 0L, active7, 0L, active8, 0L, active9, 0L, active10, 0L, active11, 0L);
       case 89:
       case 121:
          if ((active0 & 0x40000L) != 0L)
@@ -11515,19 +11511,19 @@ private final int jjMoveStringLiteralDfa2_2(long old0, long active0, long old1,
             jjmatchedKind = 286;
             jjmatchedPos = 2;
          }
-         return jjMoveStringLiteralDfa3_2(active0, 0x40000000L, active1, 0L, active2, 0x400L, active3, 0L, active4, 0x180000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0x800L, active10, 0L, active11, 0x40L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0x40000000L, active1, 0L, active2, 0x400L, active3, 0L, active4, 0x180000000L, active5, 0L, active6, 0L, active7, 0L, active8, 0L, active9, 0x800L, active10, 0L, active11, 0x40L);
       case 90:
       case 122:
-         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0x8L, active9, 0L, active10, 0L, active11, 0L, active12, 0L);
+         return jjMoveStringLiteralDfa3_2(active0, 0L, active1, 0L, active2, 0L, active3, 0L, active4, 0L, active5, 0L, active6, 0L, active7, 0L, active8, 0x8L, active9, 0L, active10, 0L, active11, 0L);
       default :
          break;
    }
-   return jjStartNfa_2(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, active12);
+   return jjStartNfa_2(1, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, 0L);
 }
-private final int jjMoveStringLiteralDfa3_2(long old0, long active0, long old1, long active1, long old2, long active2, long old3, long active3, long old4, long active4, long old5, long active5, long old6, long active6, long old7, long active7, long old8, long active8, long old9, long active9, long old10, long active10, long old11, long active11, long old12, long active12)
+private final int jjMoveStringLiteralDfa3_2(long old0, long active0, long old1, long active1, long old2, long active2, long old3, long active3, long old4, long active4, long old5, long active5, long old6, long active6, long old7, long active7, long old8, long active8, long old9, long active9, long old10, long active10, long old11, long active11)
 {
-   if (((active0 &= old0) | (active1 &= old1) | (active2 &= old2) | (active3 &= old3) | (active4 &= old4) | (active5 &= old5) | (active6 &= old6) | (active7 &= old7) | (active8 &= old8) | (active9 &= old9) | (active10 &= old10) | (active11 &= old11) | (active12 &= old12)) == 0L)
-      return jjStartNfa_2(1, old0, old1, old2, old3, old4, old5, old6, old7, old8, old9, old10, old11, old12); 
+   if (((active0 &= old0) | (active1 &= old1) | (active2 &= old2) | (active3 &= old3) | (active4 &= old4) | (active5 &= old5) | (active6 &= old6) | (active7 &= old7) | (active8 &= old8) | (active9 &= old9) | (active10 &= old10) | (active11 &= old11)) == 0L)
+      return jjStartNfa_2(1, old0, old1, old2, old3, old4, old5, old6, old7, old8, old9, old10, old11, 0L);
    try { curChar = input_stream.readChar(); }
    catch(java.io.IOException e) {
       jjStopStringLiteralDfa_2(2, active0, active1, active2, active3, active4, active5, active6, active7, active8, active9, active10, active11, 0L);
@@ -11591,8 +11587,6 @@ private final int jjMoveStringLiteralDfa3_2(long old0, long active0, long old1,
             jjmatchedKind = 436;
             jjmatchedPos = 3;
          }
-         else if ((active11 & 0x100L) != 0L)
-            return jjStartNfaWithStates_2(3, 712, 88);
          return jjMoveStringLiteralDfa4_2(active0, 0x40000000000000L, active1, 0xe0000000L, active2, 0L, active3, 0L, active4, 0x20000010000L, active5, 0L, active6, 0x20000000000000L, active7, 0L, active8, 0L, active9, 0xc0L, active10, 0x1000000000L, active11, 0L);
       case 69:
       case 101:
@@ -14852,14 +14846,14 @@ private final int jjMoveNfa_2(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x3ff001000000000L & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (curChar == 36)
                      jjCheckNAdd(31);
                   break;
-               case 86:
+               case 87:
                   if ((0x7ff601000000000L & l) != 0L)
                      jjCheckNAddTwoStates(29, 30);
                   else if (curChar == 39)
@@ -14868,8 +14862,8 @@ private final int jjMoveNfa_2(int startState, int curPos)
                      jjCheckNAddStates(0, 2);
                   if ((0x3ff001000000000L & l) != 0L)
                   {
-                     if (kind > 781)
-                        kind = 781;
+                     if (kind > 780)
+                        kind = 780;
                      jjCheckNAdd(27);
                   }
                   if (curChar == 36)
@@ -14878,8 +14872,8 @@ private final int jjMoveNfa_2(int startState, int curPos)
                case 84:
                   if (curChar == 47)
                   {
-                     if (kind > 772)
-                        kind = 772;
+                     if (kind > 771)
+                        kind = 771;
                      jjCheckNAddStates(9, 11);
                   }
... 10746 lines suppressed ...