diff --git a/amoro-format-iceberg/src/main/java/org/apache/amoro/optimizing/IcebergRewriteExecutor.java b/amoro-format-iceberg/src/main/java/org/apache/amoro/optimizing/IcebergRewriteExecutor.java index fa355a8bfa..d172221720 100644 --- a/amoro-format-iceberg/src/main/java/org/apache/amoro/optimizing/IcebergRewriteExecutor.java +++ b/amoro-format-iceberg/src/main/java/org/apache/amoro/optimizing/IcebergRewriteExecutor.java @@ -24,7 +24,6 @@ import org.apache.amoro.table.MixedTable; import org.apache.amoro.utils.map.StructLikeCollections; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.StructLike; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.IdentityPartitionConverters; @@ -46,18 +45,6 @@ public IcebergRewriteExecutor( super(input, table, structLikeCollections); } - // TODO We can remove this override method after upgrading Iceberg version to 1.5+. - @Override - protected StructLike partition() { - StructLike partitionData = super.partition(); - if (partitionData != null && partitionData.size() == 0) { - // Cast empty partition data to NULL to avoid creating empty partition directory. - return null; - } else { - return partitionData; - } - } - @Override protected OptimizingDataReader dataReader() { return new GenericCombinedIcebergDataReader( diff --git a/amoro-format-iceberg/src/main/java/org/apache/amoro/table/BasicUnkeyedTable.java b/amoro-format-iceberg/src/main/java/org/apache/amoro/table/BasicUnkeyedTable.java index d5f6f878d4..6c5cceee63 100644 --- a/amoro-format-iceberg/src/main/java/org/apache/amoro/table/BasicUnkeyedTable.java +++ b/amoro-format-iceberg/src/main/java/org/apache/amoro/table/BasicUnkeyedTable.java @@ -65,6 +65,7 @@ import java.util.List; import java.util.Map; +import java.util.UUID; /** Basic implementation of {@link UnkeyedTable}, wrapping a {@link Table}. */ public class BasicUnkeyedTable implements UnkeyedTable, HasTableOperations { @@ -318,4 +319,9 @@ public StructLikeMap> partitionProperty() { public UpdatePartitionProperties updatePartitionProperties(Transaction transaction) { return new PartitionPropertiesUpdate(this, transaction); } + + @Override + public UUID uuid() { + return UUID.fromString(this.operations().current().uuid()); + } } diff --git a/amoro-format-iceberg/src/test/java/org/apache/iceberg/TestIcebergFindFiles.java b/amoro-format-iceberg/src/test/java/org/apache/iceberg/TestIcebergFindFiles.java index bd674c7d3b..e7fa4a37e2 100644 --- a/amoro-format-iceberg/src/test/java/org/apache/iceberg/TestIcebergFindFiles.java +++ b/amoro-format-iceberg/src/test/java/org/apache/iceberg/TestIcebergFindFiles.java @@ -25,24 +25,44 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; +import org.junit.After; import org.junit.Assert; import org.junit.Assume; +import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.util.Arrays; +import java.util.List; import java.util.Set; @RunWith(Parameterized.class) -public class TestIcebergFindFiles extends TableTestBase { +public class TestIcebergFindFiles extends TestBase { + @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; + public static List parameters() { + return Arrays.asList(1, 2); } + @Rule public TemporaryFolder tempDir = new TemporaryFolder(); + public TestIcebergFindFiles(int formatVersion) { - super(formatVersion); + this.formatVersion = formatVersion; + } + + @Before + public void setUp() throws Exception { + this.tableDir = tempDir.newFolder(); + this.tableDir.delete(); + super.setupTable(); + } + + @After + public void cleanUp() throws Exception { + super.cleanupTables(); } @Test diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-format/pom.xml b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-format/pom.xml index 8be96c28d8..4e85049c91 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-format/pom.xml +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-format/pom.xml @@ -31,7 +31,7 @@ https://amoro.apache.org - 1.17.1 + 1.18.1 diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-iceberg-bridge/pom.xml b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-iceberg-bridge/pom.xml index 51d0dc00db..d894cbb283 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-iceberg-bridge/pom.xml +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-iceberg-bridge/pom.xml @@ -31,10 +31,9 @@ https://amoro.apache.org - 3.2.3 3.21.0 - 1.17.2 - 1.17.1 + 1.18.3 + 1.18.1 @@ -58,7 +57,7 @@ org.apache.iceberg - iceberg-flink-1.17 + iceberg-flink-1.18 ${iceberg.version} provided @@ -99,7 +98,7 @@ org.apache.flink flink-connector-kafka - ${flink.version} + 3.2.0-1.18 provided @@ -184,7 +183,7 @@ org.apache.iceberg - iceberg-flink-1.17 + iceberg-flink-1.18 ${iceberg.version} tests test @@ -275,9 +274,9 @@ - com.fasterxml.jackson.core - jackson-databind - ${jackson.vesion} + org.apache.flink + flink-shaded-jackson + 2.15.3-18.0 provided diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-iceberg-bridge/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-iceberg-bridge/src/main/java/org/apache/iceberg/flink/source/ScanContext.java index c188536472..9a0f7d7700 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-iceberg-bridge/src/main/java/org/apache/iceberg/flink/source/ScanContext.java +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common-iceberg-bridge/src/main/java/org/apache/iceberg/flink/source/ScanContext.java @@ -32,8 +32,10 @@ import java.io.Serializable; import java.time.Duration; +import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; /** * Copy from Iceberg {@link ScanContext}. only change line 115 and expand the modifier. Context @@ -173,9 +175,12 @@ public class ScanContext implements Serializable { protected final List filters; protected final long limit; protected final boolean includeColumnStats; + protected final Collection includeStatsForColumns; protected final Integer planParallelism; protected final int maxPlanningSnapshotCount; protected final int maxAllowedPlanningFailures; + protected final String watermarkColumn; + protected final TimeUnit watermarkColumnTimeUnit; protected ScanContext( boolean caseSensitive, @@ -195,10 +200,13 @@ protected ScanContext( List filters, long limit, boolean includeColumnStats, + Collection includeStatsForColumns, boolean exposeLocality, Integer planParallelism, int maxPlanningSnapshotCount, int maxAllowedPlanningFailures, + String watermarkColumn, + TimeUnit watermarkColumnTimeUnit, String branch, String tag, String startTag, @@ -225,15 +233,18 @@ protected ScanContext( this.filters = filters; this.limit = limit; this.includeColumnStats = includeColumnStats; + this.includeStatsForColumns = includeStatsForColumns; this.exposeLocality = exposeLocality; this.planParallelism = planParallelism; this.maxPlanningSnapshotCount = maxPlanningSnapshotCount; this.maxAllowedPlanningFailures = maxAllowedPlanningFailures; + this.watermarkColumn = watermarkColumn; + this.watermarkColumnTimeUnit = watermarkColumnTimeUnit; validate(); } - private void validate() { + void validate() { if (isStreaming) { if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) { Preconditions.checkArgument( @@ -251,15 +262,13 @@ private void validate() { startSnapshotId == null, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); } - Preconditions.checkArgument( - branch == null, - String.format( - "Cannot scan table using ref %s configured for streaming reader yet", branch)); Preconditions.checkArgument( tag == null, String.format("Cannot scan table using ref %s configured for streaming reader", tag)); + Preconditions.checkArgument(endTag == null, "Cannot set end-tag option for streaming reader"); } + Preconditions.checkArgument( !(startTag != null && startSnapshotId() != null), "START_SNAPSHOT_ID and START_TAG cannot both be set."); @@ -273,106 +282,118 @@ private void validate() { "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); } - boolean caseSensitive() { + public boolean caseSensitive() { return caseSensitive; } - Long snapshotId() { + public Long snapshotId() { return snapshotId; } - String branch() { + public String branch() { return branch; } - String tag() { + public String tag() { return tag; } - String startTag() { + public String startTag() { return startTag; } - String endTag() { + public String endTag() { return endTag; } - StreamingStartingStrategy streamingStartingStrategy() { + public StreamingStartingStrategy streamingStartingStrategy() { return startingStrategy; } - Long startSnapshotTimestamp() { + public Long startSnapshotTimestamp() { return startSnapshotTimestamp; } - Long startSnapshotId() { + public Long startSnapshotId() { return startSnapshotId; } - Long endSnapshotId() { + public Long endSnapshotId() { return endSnapshotId; } - Long asOfTimestamp() { + public Long asOfTimestamp() { return asOfTimestamp; } - Long splitSize() { + public Long splitSize() { return splitSize; } - Integer splitLookback() { + public Integer splitLookback() { return splitLookback; } - Long splitOpenFileCost() { + public Long splitOpenFileCost() { return splitOpenFileCost; } - boolean isStreaming() { + public boolean isStreaming() { return isStreaming; } - Duration monitorInterval() { + public Duration monitorInterval() { return monitorInterval; } - String nameMapping() { + public String nameMapping() { return nameMapping; } - Schema project() { + public Schema project() { return schema; } - List filters() { + public List filters() { return filters; } - long limit() { + public long limit() { return limit; } - boolean includeColumnStats() { + public boolean includeColumnStats() { return includeColumnStats; } - boolean exposeLocality() { + public Collection includeStatsForColumns() { + return includeStatsForColumns; + } + + public boolean exposeLocality() { return exposeLocality; } - Integer planParallelism() { + public Integer planParallelism() { return planParallelism; } - int maxPlanningSnapshotCount() { + public int maxPlanningSnapshotCount() { return maxPlanningSnapshotCount; } - int maxAllowedPlanningFailures() { + public int maxAllowedPlanningFailures() { return maxAllowedPlanningFailures; } + public String watermarkColumn() { + return watermarkColumn; + } + + public TimeUnit watermarkColumnTimeUnit() { + return watermarkColumnTimeUnit; + } + ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSnapshotId) { return ScanContext.builder() .caseSensitive(caseSensitive) @@ -394,10 +415,13 @@ ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSnapshotI .filters(filters) .limit(limit) .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) .exposeLocality(exposeLocality) .planParallelism(planParallelism) .maxPlanningSnapshotCount(maxPlanningSnapshotCount) .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .watermarkColumn(watermarkColumn) + .watermarkColumnTimeUnit(watermarkColumnTimeUnit) .build(); } @@ -406,7 +430,7 @@ ScanContext copyWithSnapshotId(long newSnapshotId) { .caseSensitive(caseSensitive) .useSnapshotId(newSnapshotId) .useBranch(branch) - .useTag(null) + .useTag(tag) .startSnapshotId(null) .endSnapshotId(null) .startTag(null) @@ -422,18 +446,21 @@ ScanContext copyWithSnapshotId(long newSnapshotId) { .filters(filters) .limit(limit) .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) .exposeLocality(exposeLocality) .planParallelism(planParallelism) .maxPlanningSnapshotCount(maxPlanningSnapshotCount) .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .watermarkColumn(watermarkColumn) + .watermarkColumnTimeUnit(watermarkColumnTimeUnit) .build(); } - static Builder builder() { + public static Builder builder() { return new Builder(); } - static class Builder { + public static class Builder { private boolean caseSensitive = FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue(); private Long snapshotId = FlinkReadOptions.SNAPSHOT_ID.defaultValue(); private String branch = FlinkReadOptions.BRANCH.defaultValue(); @@ -458,6 +485,7 @@ static class Builder { private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue(); private boolean includeColumnStats = FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue(); + private Collection includeStatsForColumns = null; private boolean exposeLocality; private Integer planParallelism = FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); @@ -465,135 +493,153 @@ static class Builder { FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue(); private int maxAllowedPlanningFailures = FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue(); + private String watermarkColumn = FlinkReadOptions.WATERMARK_COLUMN_OPTION.defaultValue(); + private TimeUnit watermarkColumnTimeUnit = + FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION.defaultValue(); private Builder() {} - Builder caseSensitive(boolean newCaseSensitive) { + public Builder caseSensitive(boolean newCaseSensitive) { this.caseSensitive = newCaseSensitive; return this; } - Builder useSnapshotId(Long newSnapshotId) { + public Builder useSnapshotId(Long newSnapshotId) { this.snapshotId = newSnapshotId; return this; } - Builder useTag(String newTag) { + public Builder useTag(String newTag) { this.tag = newTag; return this; } - Builder useBranch(String newBranch) { + public Builder useBranch(String newBranch) { this.branch = newBranch; return this; } - Builder startingStrategy(StreamingStartingStrategy newStartingStrategy) { + public Builder startingStrategy(StreamingStartingStrategy newStartingStrategy) { this.startingStrategy = newStartingStrategy; return this; } - Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { + public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { this.startSnapshotTimestamp = newStartSnapshotTimestamp; return this; } - Builder startSnapshotId(Long newStartSnapshotId) { + public Builder startSnapshotId(Long newStartSnapshotId) { this.startSnapshotId = newStartSnapshotId; return this; } - Builder endSnapshotId(Long newEndSnapshotId) { + public Builder endSnapshotId(Long newEndSnapshotId) { this.endSnapshotId = newEndSnapshotId; return this; } - Builder startTag(String newStartTag) { + public Builder startTag(String newStartTag) { this.startTag = newStartTag; return this; } - Builder endTag(String newEndTag) { + public Builder endTag(String newEndTag) { this.endTag = newEndTag; return this; } - Builder asOfTimestamp(Long newAsOfTimestamp) { + public Builder asOfTimestamp(Long newAsOfTimestamp) { this.asOfTimestamp = newAsOfTimestamp; return this; } - Builder splitSize(Long newSplitSize) { + public Builder splitSize(Long newSplitSize) { this.splitSize = newSplitSize; return this; } - Builder splitLookback(Integer newSplitLookback) { + public Builder splitLookback(Integer newSplitLookback) { this.splitLookback = newSplitLookback; return this; } - Builder splitOpenFileCost(Long newSplitOpenFileCost) { + public Builder splitOpenFileCost(Long newSplitOpenFileCost) { this.splitOpenFileCost = newSplitOpenFileCost; return this; } - Builder streaming(boolean streaming) { + public Builder streaming(boolean streaming) { this.isStreaming = streaming; return this; } - Builder monitorInterval(Duration newMonitorInterval) { + public Builder monitorInterval(Duration newMonitorInterval) { this.monitorInterval = newMonitorInterval; return this; } - Builder nameMapping(String newNameMapping) { + public Builder nameMapping(String newNameMapping) { this.nameMapping = newNameMapping; return this; } - Builder project(Schema newProjectedSchema) { + public Builder project(Schema newProjectedSchema) { this.projectedSchema = newProjectedSchema; return this; } - Builder filters(List newFilters) { + public Builder filters(List newFilters) { this.filters = newFilters; return this; } - Builder limit(long newLimit) { + public Builder limit(long newLimit) { this.limit = newLimit; return this; } - Builder includeColumnStats(boolean newIncludeColumnStats) { + public Builder includeColumnStats(boolean newIncludeColumnStats) { this.includeColumnStats = newIncludeColumnStats; return this; } - Builder exposeLocality(boolean newExposeLocality) { + public Builder includeColumnStats(Collection newIncludeStatsForColumns) { + this.includeStatsForColumns = newIncludeStatsForColumns; + return this; + } + + public Builder exposeLocality(boolean newExposeLocality) { this.exposeLocality = newExposeLocality; return this; } - Builder planParallelism(Integer parallelism) { + public Builder planParallelism(Integer parallelism) { this.planParallelism = parallelism; return this; } - Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { + public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { this.maxPlanningSnapshotCount = newMaxPlanningSnapshotCount; return this; } - Builder maxAllowedPlanningFailures(int newMaxAllowedPlanningFailures) { + public Builder maxAllowedPlanningFailures(int newMaxAllowedPlanningFailures) { this.maxAllowedPlanningFailures = newMaxAllowedPlanningFailures; return this; } - Builder resolveConfig( + public Builder watermarkColumn(String newWatermarkColumn) { + this.watermarkColumn = newWatermarkColumn; + return this; + } + + public Builder watermarkColumnTimeUnit(TimeUnit newWatermarkTimeUnit) { + this.watermarkColumnTimeUnit = newWatermarkTimeUnit; + return this; + } + + public Builder resolveConfig( Table table, Map readOptions, ReadableConfig readableConfig) { FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, readableConfig); @@ -618,7 +664,9 @@ Builder resolveConfig( .planParallelism(flinkReadConf.workerPoolSize()) .includeColumnStats(flinkReadConf.includeColumnStats()) .maxPlanningSnapshotCount(flinkReadConf.maxPlanningSnapshotCount()) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures); + .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .watermarkColumn(flinkReadConf.watermarkColumn()) + .watermarkColumnTimeUnit(flinkReadConf.watermarkColumnTimeUnit()); } public ScanContext build() { @@ -640,10 +688,13 @@ public ScanContext build() { filters, limit, includeColumnStats, + includeStatsForColumns, exposeLocality, planParallelism, maxPlanningSnapshotCount, maxAllowedPlanningFailures, + watermarkColumn, + watermarkColumnTimeUnit, branch, tag, startTag, diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/pom.xml b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/pom.xml index 41a51b9d89..c226fa1fd2 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/pom.xml +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/pom.xml @@ -34,10 +34,9 @@ jar - 3.2.3 3.21.0 - 1.17.2 - 1.17.1 + 1.18.3 + 1.18.1 @@ -66,7 +65,7 @@ org.apache.iceberg - iceberg-flink-1.17 + iceberg-flink-1.18 ${iceberg.version} provided @@ -119,7 +118,7 @@ org.apache.flink flink-connector-kafka - ${flink.version} + 3.2.0-1.18 provided @@ -210,10 +209,11 @@ + - com.fasterxml.jackson.core - jackson-databind - ${jackson.vesion} + org.apache.flink + flink-shaded-jackson + 2.15.3-18.0 provided @@ -221,7 +221,7 @@ org.apache.iceberg - iceberg-flink-1.17 + iceberg-flink-1.18 ${iceberg.version} tests test diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/lookup/RocksDBCacheState.java b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/lookup/RocksDBCacheState.java index a82bff54db..2f5b93c352 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/lookup/RocksDBCacheState.java +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/lookup/RocksDBCacheState.java @@ -23,8 +23,8 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.shaded.guava30.com.google.common.cache.Cache; -import org.apache.flink.shaded.guava30.com.google.common.cache.CacheBuilder; +import org.apache.flink.shaded.guava31.com.google.common.cache.Cache; +import org.apache.flink.shaded.guava31.com.google.common.cache.CacheBuilder; import org.apache.flink.table.data.RowData; import org.apache.flink.types.RowKind; import org.apache.flink.util.FlinkRuntimeException; diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/lookup/RocksDBSetSpilledState.java b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/lookup/RocksDBSetSpilledState.java index 5215ed812c..3f8016bdfc 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/lookup/RocksDBSetSpilledState.java +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/lookup/RocksDBSetSpilledState.java @@ -22,7 +22,7 @@ import org.apache.amoro.utils.map.RocksDBBackend; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.shaded.guava30.com.google.common.cache.Cache; +import org.apache.flink.shaded.guava31.com.google.common.cache.Cache; import org.apache.flink.table.data.RowData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/read/internals/KafkaSource.java b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/read/internals/KafkaSource.java index 56317638a7..c6643e5ee3 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/read/internals/KafkaSource.java +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/read/internals/KafkaSource.java @@ -179,7 +179,7 @@ public SplitEnumerator restoreEnumera props, enumContext, boundedness, - checkpoint.assignedPartitions()); + checkpoint); } @Internal diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/read/source/MixedFormatScanContext.java b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/read/source/MixedFormatScanContext.java index 7fc850df07..88a522b7c8 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/read/source/MixedFormatScanContext.java +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/read/source/MixedFormatScanContext.java @@ -33,9 +33,11 @@ import java.io.Serializable; import java.time.Duration; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.TimeUnit; /** This is an mixed-format source scan context. */ public class MixedFormatScanContext extends ScanContext implements Serializable { @@ -64,10 +66,13 @@ protected MixedFormatScanContext(Builder builder) { builder.filters, builder.limit, builder.includeColumnStats, + builder.includeStatsForColumns, builder.exposeLocality, builder.planParallelism, builder.maxPlanningSnapshotCount, builder.maxAllowedPlanningFailures, + builder.watermarkColumn, + builder.watermarkColumnTimeUnit, builder.branch, builder.tag, builder.startTag, @@ -182,6 +187,11 @@ public static class Builder { private String endTag = FlinkReadOptions.END_TAG.defaultValue(); private String scanStartupMode; + private Collection includeStatsForColumns = null; + private String watermarkColumn = FlinkReadOptions.WATERMARK_COLUMN_OPTION.defaultValue(); + private TimeUnit watermarkColumnTimeUnit = + FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION.defaultValue(); + private boolean batchMode = false; private Builder() {} @@ -321,6 +331,21 @@ public Builder batchMode(boolean batchMode) { return this; } + public Builder includeColumnStats(Collection newIncludeStatsForColumns) { + this.includeStatsForColumns = newIncludeStatsForColumns; + return this; + } + + public Builder watermarkColumn(String newWatermarkColumn) { + this.watermarkColumn = newWatermarkColumn; + return this; + } + + public Builder watermarkColumnTimeUnit(TimeUnit newWatermarkTimeUnit) { + this.watermarkColumnTimeUnit = newWatermarkTimeUnit; + return this; + } + public Builder fromProperties(Map properties) { Configuration config = new Configuration(); properties.forEach(config::setString); @@ -344,8 +369,11 @@ public Builder fromProperties(Map properties) { .nameMapping(properties.get(DEFAULT_NAME_MAPPING)) .scanStartupMode(properties.get(MixedFormatValidator.SCAN_STARTUP_MODE.key())) .includeColumnStats(config.get(INCLUDE_COLUMN_STATS)) + .includeColumnStats(includeStatsForColumns) .maxPlanningSnapshotCount(config.get(MAX_PLANNING_SNAPSHOT_COUNT)) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures); + .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .watermarkColumn(watermarkColumn) + .watermarkColumnTimeUnit(watermarkColumnTimeUnit); } public MixedFormatScanContext build() { diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/table/FlinkSource.java b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/table/FlinkSource.java index 3e4080e8a8..9e1998c5fb 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/table/FlinkSource.java +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/main/java/org/apache/amoro/flink/table/FlinkSource.java @@ -240,7 +240,7 @@ public DataStream buildUnkeyedTableSource(String scanStartupMode) { .project(org.apache.amoro.flink.FlinkSchemaUtil.filterWatermark(projectedSchema)) .tableLoader(tableLoader) .filters(filters) - .properties(properties) + .setAll(properties) .flinkConf(flinkConf) .limit(limit); if (MixedFormatValidator.SCAN_STARTUP_MODE_LATEST.equalsIgnoreCase(scanStartupMode)) { diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/lookup/TestKVTable.java b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/lookup/TestKVTable.java index 1f830cb6d9..3dc0551cc6 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/lookup/TestKVTable.java +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/lookup/TestKVTable.java @@ -30,9 +30,9 @@ import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.flink.shaded.guava30.com.google.common.cache.Cache; -import org.apache.flink.shaded.guava30.com.google.common.cache.CacheBuilder; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava31.com.google.common.cache.Cache; +import org.apache.flink.shaded.guava31.com.google.common.cache.CacheBuilder; +import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/read/TestMixedFormatSource.java b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/read/TestMixedFormatSource.java index ecdf4c47e9..5ba76d4475 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/read/TestMixedFormatSource.java +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/read/TestMixedFormatSource.java @@ -234,6 +234,7 @@ public void testMixedFormatSource(FailoverType failoverType) throws Exception { @Test public void testDimTaskManagerFailover() throws Exception { + int restartAttempts = 10; List updated = updateRecords(); writeUpdate(updated); List records = generateRecords(2, 1); @@ -242,8 +243,8 @@ public void testDimTaskManagerFailover() throws Exception { MixedFormatSource mixedFormatSource = initMixedFormatDimSource(true); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // enable checkpoint - env.enableCheckpointing(1000); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(10, 0)); + env.enableCheckpointing(3000); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(restartAttempts, 0)); DataStream input = env.fromSource( @@ -262,8 +263,8 @@ public void testDimTaskManagerFailover() throws Exception { WatermarkAwareFailWrapper::continueProcessing, miniClusterResource.getMiniCluster()); - while (WatermarkAwareFailWrapper.watermarkCounter.get() != PARALLELISM) { - Thread.sleep(1000); + while (WatermarkAwareFailWrapper.watermarkCounter.get() != restartAttempts) { + Thread.sleep(2000); LOG.info("wait for watermark after failover"); } Assert.assertEquals(Long.MAX_VALUE, WatermarkAwareFailWrapper.getWatermarkAfterFailover()); diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/read/hybrid/reader/TestRowDataReaderFunction.java b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/read/hybrid/reader/TestRowDataReaderFunction.java index 4bad7e47fe..a86e2afda8 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/read/hybrid/reader/TestRowDataReaderFunction.java +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/read/hybrid/reader/TestRowDataReaderFunction.java @@ -31,7 +31,7 @@ import org.apache.amoro.scan.MixedFileScanTask; import org.apache.amoro.table.KeyedTable; import org.apache.flink.configuration.Configuration; -import org.apache.flink.shaded.guava30.com.google.common.collect.Maps; +import org.apache.flink.shaded.guava31.com.google.common.collect.Maps; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; diff --git a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/table/TestJoin.java b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/table/TestJoin.java index da5347a5e5..8d2d4bed08 100644 --- a/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/table/TestJoin.java +++ b/amoro-format-mixed/amoro-mixed-flink/amoro-mixed-flink-common/src/test/java/org/apache/amoro/flink/table/TestJoin.java @@ -32,7 +32,7 @@ import org.apache.amoro.table.TableIdentifier; import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.testutils.CommonTestUtils; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.TableSchema; diff --git a/amoro-format-mixed/amoro-mixed-flink/pom.xml b/amoro-format-mixed/amoro-mixed-flink/pom.xml index 32871a988b..808615e1ee 100644 --- a/amoro-format-mixed/amoro-mixed-flink/pom.xml +++ b/amoro-format-mixed/amoro-mixed-flink/pom.xml @@ -37,19 +37,13 @@ amoro-mixed-flink-common amoro-mixed-flink-common-format amoro-mixed-flink-common-iceberg-bridge - v1.15/amoro-mixed-flink-1.15 - v1.15/amoro-mixed-flink-runtime-1.15 - v1.16/amoro-mixed-flink-1.16 - v1.16/amoro-mixed-flink-runtime-1.16 - v1.17/amoro-mixed-flink-1.17 - v1.17/amoro-mixed-flink-runtime-1.17 + v1.18/amoro-mixed-flink-1.18 + v1.18/amoro-mixed-flink-runtime-1.18 - 1.17.1 - 2.4.1 - 2.9.0 - 2.10.2 + 1.18.1 + 2.11.0 diff --git a/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-1.15/pom.xml b/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-1.15/pom.xml deleted file mode 100644 index 0f9202e688..0000000000 --- a/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-1.15/pom.xml +++ /dev/null @@ -1,108 +0,0 @@ - - - - 4.0.0 - - org.apache.amoro - amoro-mixed-flink - 0.8-SNAPSHOT - ../../pom.xml - - - amoro-format-mixed-flink-1.15 - Amoro Project Mixed Format Flink 1.15 - https://amoro.apache.org - - jar - - - 2.8.1 - 3.21.0 - 1.17.2 - 1.15.3 - - - - - org.apache.amoro - amoro-mixed-flink-common - ${project.parent.version} - - - - org.apache.iceberg - iceberg-flink-1.15 - ${iceberg.version} - - - org.slf4j - slf4j-api - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-avro - - - - - - org.apache.paimon - paimon-flink-1.15 - ${paimon.version} - - - - org.apache.flink - flink-table-api-java-bridge - ${flink.version} - provided - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-amoro - package - - shade - - - - - org.apache.amoro:amoro-format-mixed-flink-common - - - - - - - - - diff --git a/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-1.15/src/main/java/org/apache/amoro/flink/lookup/MixedFormatRowDataLookupFunction.java b/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-1.15/src/main/java/org/apache/amoro/flink/lookup/MixedFormatRowDataLookupFunction.java deleted file mode 100644 index 1a508d9307..0000000000 --- a/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-1.15/src/main/java/org/apache/amoro/flink/lookup/MixedFormatRowDataLookupFunction.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.flink.lookup; - -import org.apache.amoro.flink.read.hybrid.reader.DataIteratorReaderFunction; -import org.apache.amoro.flink.table.MixedFormatTableLoader; -import org.apache.amoro.hive.io.reader.AbstractAdaptHiveKeyedDataReader; -import org.apache.amoro.table.MixedTable; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.functions.FunctionContext; -import org.apache.flink.table.functions.TableFunction; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Expression; - -import java.io.IOException; -import java.util.List; -import java.util.function.Predicate; - -/** A lookup function for {@link RowData} type. */ -public class MixedFormatRowDataLookupFunction extends TableFunction { - private static final long serialVersionUID = -7694050999266540499L; - private final BasicLookupFunction basicLookupFunction; - - public MixedFormatRowDataLookupFunction( - TableFactory tableFactory, - MixedTable mixedTable, - List joinKeys, - Schema projectSchema, - List filters, - MixedFormatTableLoader tableLoader, - Configuration config, - Predicate predicate, - AbstractAdaptHiveKeyedDataReader flinkMORDataReader, - DataIteratorReaderFunction readerFunction) { - this.basicLookupFunction = - new BasicLookupFunction<>( - tableFactory, - mixedTable, - joinKeys, - projectSchema, - filters, - tableLoader, - config, - predicate, - flinkMORDataReader, - readerFunction); - } - - @Override - public void open(FunctionContext context) throws IOException { - basicLookupFunction.open(context); - } - - public void eval(Object... rowKey) throws IOException { - List results = basicLookupFunction.lookup(GenericRowData.of(rowKey)); - results.forEach(this::collect); - } - - @Override - public void close() throws Exception { - basicLookupFunction.close(); - } -} diff --git a/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-1.15/src/main/java/org/apache/amoro/flink/table/MixedFormatDynamicSource.java b/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-1.15/src/main/java/org/apache/amoro/flink/table/MixedFormatDynamicSource.java deleted file mode 100644 index 4574c2121e..0000000000 --- a/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-1.15/src/main/java/org/apache/amoro/flink/table/MixedFormatDynamicSource.java +++ /dev/null @@ -1,384 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.flink.table; - -import org.apache.amoro.flink.lookup.KVTableFactory; -import org.apache.amoro.flink.lookup.MixedFormatRowDataLookupFunction; -import org.apache.amoro.flink.lookup.filter.RowDataPredicate; -import org.apache.amoro.flink.lookup.filter.RowDataPredicateExpressionVisitor; -import org.apache.amoro.flink.read.hybrid.reader.DataIteratorReaderFunction; -import org.apache.amoro.flink.read.hybrid.reader.RowDataReaderFunction; -import org.apache.amoro.flink.read.source.FlinkKeyedMORDataReader; -import org.apache.amoro.flink.util.FilterUtil; -import org.apache.amoro.flink.util.IcebergAndFlinkFilters; -import org.apache.amoro.hive.io.reader.AbstractAdaptHiveKeyedDataReader; -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.utils.SchemaUtil; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.source.DataStreamScanProvider; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.LookupTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.connector.source.TableFunctionProvider; -import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.expressions.CallExpression; -import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; -import org.apache.flink.table.functions.FunctionIdentifier; -import org.apache.flink.table.functions.TableFunction; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.data.RowDataUtil; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.function.BiFunction; -import java.util.stream.Collectors; - -/** Flink table api that generates source operators. */ -public class MixedFormatDynamicSource - implements ScanTableSource, - SupportsFilterPushDown, - SupportsProjectionPushDown, - SupportsLimitPushDown, - SupportsWatermarkPushDown, - LookupTableSource { - - private static final Logger LOG = LoggerFactory.getLogger(MixedFormatDynamicSource.class); - - protected final String tableName; - - protected final ScanTableSource mixedFormatDynamicSource; - protected final MixedTable mixedTable; - protected final Map properties; - - protected int[] projectFields; - protected List filters; - protected ResolvedExpression flinkExpression; - protected final MixedFormatTableLoader tableLoader; - - @Nullable protected WatermarkStrategy watermarkStrategy; - - /** - * @param tableName tableName - * @param mixedFormatDynamicSource underlying source - * @param mixedTable mixedTable - * @param properties With all mixed-format table properties and sql options - * @param tableLoader - */ - public MixedFormatDynamicSource( - String tableName, - ScanTableSource mixedFormatDynamicSource, - MixedTable mixedTable, - Map properties, - MixedFormatTableLoader tableLoader) { - this.tableName = tableName; - this.mixedFormatDynamicSource = mixedFormatDynamicSource; - this.mixedTable = mixedTable; - this.properties = properties; - this.tableLoader = tableLoader; - } - - public MixedFormatDynamicSource( - String tableName, - ScanTableSource mixedFormatDynamicSource, - MixedTable mixedTable, - Map properties, - MixedFormatTableLoader tableLoader, - int[] projectFields, - List filters, - ResolvedExpression flinkExpression) { - this.tableName = tableName; - this.mixedFormatDynamicSource = mixedFormatDynamicSource; - this.mixedTable = mixedTable; - this.properties = properties; - this.tableLoader = tableLoader; - this.projectFields = projectFields; - this.filters = filters; - this.flinkExpression = flinkExpression; - } - - @Override - public ChangelogMode getChangelogMode() { - return mixedFormatDynamicSource.getChangelogMode(); - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { - ScanRuntimeProvider origin = mixedFormatDynamicSource.getScanRuntimeProvider(scanContext); - Preconditions.checkArgument( - origin instanceof DataStreamScanProvider, - "file or log ScanRuntimeProvider should be DataStreamScanProvider, but provided is " - + origin.getClass()); - return origin; - } - - @Override - public DynamicTableSource copy() { - return new MixedFormatDynamicSource( - tableName, - mixedFormatDynamicSource, - mixedTable, - properties, - tableLoader, - projectFields, - filters, - flinkExpression); - } - - @Override - public String asSummaryString() { - return "Mixed-format Dynamic Source"; - } - - @Override - public Result applyFilters(List filters) { - IcebergAndFlinkFilters icebergAndFlinkFilters = - FilterUtil.convertFlinkExpressToIceberg(filters); - this.filters = icebergAndFlinkFilters.expressions(); - - if (filters.size() == 1) { - flinkExpression = filters.get(0); - } else if (filters.size() >= 2) { - flinkExpression = and(filters.get(0), filters.get(1)); - for (int i = 2; i < filters.size(); i++) { - flinkExpression = and(flinkExpression, filters.subList(i, i + 1).get(0)); - } - } - - if (mixedFormatDynamicSource instanceof SupportsFilterPushDown) { - return ((SupportsFilterPushDown) mixedFormatDynamicSource).applyFilters(filters); - } else { - return Result.of(Collections.emptyList(), filters); - } - } - - @Override - public boolean supportsNestedProjection() { - if (mixedFormatDynamicSource instanceof SupportsProjectionPushDown) { - return ((SupportsProjectionPushDown) mixedFormatDynamicSource).supportsNestedProjection(); - } else { - return false; - } - } - - protected CallExpression and(ResolvedExpression left, ResolvedExpression right) { - return CallExpression.permanent( - FunctionIdentifier.of(BuiltInFunctionDefinitions.AND.getName()), - BuiltInFunctionDefinitions.AND, - Arrays.asList(left, right), - DataTypes.BOOLEAN()); - } - - @Override - public void applyProjection(int[][] projectedFields, DataType producedDataType) { - projectFields = new int[projectedFields.length]; - for (int i = 0; i < projectedFields.length; i++) { - Preconditions.checkArgument( - projectedFields[i].length == 1, "Don't support nested projection now."); - projectFields[i] = projectedFields[i][0]; - } - - if (mixedFormatDynamicSource instanceof SupportsProjectionPushDown) { - ((SupportsProjectionPushDown) mixedFormatDynamicSource) - .applyProjection(projectedFields, producedDataType); - } - } - - @Override - public void applyLimit(long newLimit) { - if (mixedFormatDynamicSource instanceof SupportsLimitPushDown) { - ((SupportsLimitPushDown) mixedFormatDynamicSource).applyLimit(newLimit); - } - } - - @Override - public void applyWatermark(WatermarkStrategy watermarkStrategy) { - if (mixedFormatDynamicSource instanceof SupportsWatermarkPushDown) { - ((SupportsWatermarkPushDown) mixedFormatDynamicSource).applyWatermark(watermarkStrategy); - } - } - - @Override - public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { - int[] joinKeys = new int[context.getKeys().length]; - for (int i = 0; i < context.getKeys().length; i++) { - Preconditions.checkArgument( - context.getKeys()[i].length == 1, - "Mixed-format lookup join doesn't support the row field as a joining key."); - joinKeys[i] = context.getKeys()[i][0]; - } - - return TableFunctionProvider.of(getLookupFunction(joinKeys)); - } - - protected TableFunction getLookupFunction(int[] joinKeys) { - Schema projectedSchema = getProjectedSchema(); - - List joinKeyNames = getJoinKeyNames(joinKeys, projectedSchema); - - Configuration config = new Configuration(); - properties.forEach(config::setString); - - Optional rowDataPredicate = - generatePredicate(projectedSchema, flinkExpression); - - AbstractAdaptHiveKeyedDataReader flinkArcticMORDataReader = - generateMORReader(mixedTable, projectedSchema); - DataIteratorReaderFunction readerFunction = - generateReaderFunction(mixedTable, projectedSchema); - - return new MixedFormatRowDataLookupFunction( - KVTableFactory.INSTANCE, - mixedTable, - joinKeyNames, - projectedSchema, - filters, - tableLoader, - config, - rowDataPredicate.orElse(null), - flinkArcticMORDataReader, - readerFunction); - } - - protected DataIteratorReaderFunction generateReaderFunction( - MixedTable mixedTable, Schema projectedSchema) { - return new RowDataReaderFunction( - new Configuration(), - mixedTable.schema(), - projectedSchema, - mixedTable.asKeyedTable().primaryKeySpec(), - null, - true, - mixedTable.io(), - true); - } - - protected AbstractAdaptHiveKeyedDataReader generateMORReader( - MixedTable mixedTable, Schema projectedSchema) { - BiFunction convertConstant = new ConvertTask(); - - return new FlinkKeyedMORDataReader( - mixedTable.io(), - mixedTable.schema(), - projectedSchema, - mixedTable.asKeyedTable().primaryKeySpec(), - null, - true, - convertConstant, - true); - } - - static class ConvertTask implements BiFunction, Serializable { - private static final long serialVersionUID = 4607513893568225789L; - - @Override - public Object apply(Type t, Object u) { - return RowDataUtil.convertConstant(t, u); - } - } - - protected List getJoinKeyNames(int[] joinKeys, Schema projectedSchema) { - return Arrays.stream(joinKeys) - .mapToObj(index -> projectedSchema.columns().get(index).name()) - .collect(Collectors.toList()); - } - - protected Schema getProjectedSchema() { - Schema mixedFormatTableSchema = mixedTable.schema(); - Schema projectedSchema; - if (projectFields == null) { - LOG.info("The projected fields is null."); - projectedSchema = mixedTable.schema(); - } else { - if (mixedTable.isUnkeyedTable()) { - throw new UnsupportedOperationException("Unkeyed table doesn't support lookup join."); - } - List primaryKeys = mixedTable.asKeyedTable().primaryKeySpec().fieldNames(); - List projectFieldList = - Arrays.stream(projectFields).boxed().collect(Collectors.toList()); - List columns = mixedFormatTableSchema.columns(); - for (int i = 0; i < mixedFormatTableSchema.columns().size(); i++) { - if (primaryKeys.contains(columns.get(i).name()) && !projectFieldList.contains(i)) { - projectFieldList.add(i); - LOG.info( - "Add identifier field {} to projected schema, due to this field is mismatched.", - columns.get(i).name()); - } - } - - List projectedFieldNames = - projectFieldList.stream() - .map(index -> columns.get(index).name()) - .collect(Collectors.toList()); - projectedSchema = SchemaUtil.selectInOrder(mixedFormatTableSchema, projectedFieldNames); - LOG.info("The projected schema {}.\n table schema {}.", projectedSchema, mixedTable.schema()); - } - return projectedSchema; - } - - protected Optional generatePredicate( - final Schema projectedSchema, final ResolvedExpression flinkExpression) { - if (flinkExpression == null) { - return Optional.empty(); - } - - final Map fieldIndexMap = new HashMap<>(); - final Map fieldDataTypeMap = new HashMap<>(); - List fields = projectedSchema.asStruct().fields(); - for (int i = 0; i < fields.size(); i++) { - Types.NestedField field = fields.get(i); - fieldIndexMap.put(field.name(), i); - fieldDataTypeMap.put( - field.name(), - TypeConversions.fromLogicalToDataType(FlinkSchemaUtil.convert(field.type()))); - } - - RowDataPredicateExpressionVisitor visitor = - generateExpressionVisitor(fieldIndexMap, fieldDataTypeMap); - return flinkExpression.accept(visitor); - } - - protected RowDataPredicateExpressionVisitor generateExpressionVisitor( - Map fieldIndexMap, Map fieldDataTypeMap) { - return new RowDataPredicateExpressionVisitor(fieldIndexMap, fieldDataTypeMap); - } -} diff --git a/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-runtime-1.15/pom.xml b/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-runtime-1.15/pom.xml deleted file mode 100644 index e218e978aa..0000000000 --- a/amoro-format-mixed/amoro-mixed-flink/v1.15/amoro-mixed-flink-runtime-1.15/pom.xml +++ /dev/null @@ -1,252 +0,0 @@ - - - - 4.0.0 - - org.apache.amoro - amoro-mixed-flink - 0.8-SNAPSHOT - ../../pom.xml - - - amoro-format-mixed-flink-runtime-1.15 - Amoro Project Mixed Format Flink 1.15 Runtime - https://amoro.apache.org - - - 1.15.3 - - - - - org.apache.amoro - amoro-format-mixed-flink-1.15 - ${project.parent.version} - - - - org.apache.flink - flink-connector-kafka - ${flink.version} - - - com.github.luben - zstd-jni - - - - - - org.apache.kafka - kafka-clients - ${kafka.version} - - - com.github.luben - zstd-jni - - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-amoro - package - - shade - - - false - - - org.apache.amoro:* - org.apache.iceberg:* - com.fasterxml.jackson.core:* - org.apache.parquet:* - org.apache.commons:* - commons-lang:* - com.github.ben-manes.caffeine:* - org.apache.avro:* - org.apache.orc:* - io.airlift:* - commons-collections:* - cglib:* - com.google.guava:* - asm:* - org.apache.httpcomponents.client5:* - org.apache.httpcomponents.core5:* - org.apache.flink:flink-connector-kafka - org.apache.kafka:* - com.github.luben:* - - - - - - org.apache.iceberg - org.apache.amoro.shade.org.apache.iceberg - - org.apache.iceberg.mr.hive.* - - - - - org.apache.parquet - org.apache.amoro.shade.org.apache.parquet - - - - org.apache.commons - org.apache.amoro.shade.org.apache.commons - - - - - org.apache.avro - org.apache.amoro.shade.org.apache.avro - - - - org.apache.orc - org.apache.amoro.shade.org.apache.orc - - - - org.apache.hc - org.apache.amoro.shade.org.apache.hc - - - - org.apache.jute - org.apache.amoro.shade.org.apache.jute - - - - org.apache.kafka - org.apache.amoro.shade.org.apache.kafka - - - - shaded.parquet - org.apache.amoro.shade.shaded.parquet - - - - com.fasterxml - org.apache.amoro.shade.com.fasterxml - - - - com.github.benmanes.caffeine - - org.apache.amoro.shade.com.github.benmanes.caffeine - - - - - org.threeten.extra - org.apache.amoro.shade.org.threeten.extra - - - - net.sf.cglib - org.apache.amoro.shade.net.sf.cglib - - - - com.google - org.apache.amoro.shade.com.google - - - - org.objectweb.asm - org.apache.amoro.shade.org.objectweb.asm - - - - com.facebook.fb303 - org.apache.amoro.shade.com.facebook.fb303 - - - - io.airlift - org.apache.amoro.shade.io.airlift - - - - - org.apache.flink.connector.kafka - - org.apache.amoro.shade.org.apache.flink.connector.kafka - - - - org.apache.flink.streaming.connectors.kafka - - org.apache.amoro.shade.org.apache.flink.streaming.connectors.kafka - - - - - org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema - - - org.apache.amoro.shade.org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema - - - - - org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema - - - org.apache.amoro.shade.org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema - - - - - org.apache.flink.streaming.util.serialization.KeyedSerializationSchema - - - org.apache.amoro.shade.org.apache.flink.streaming.util.serialization.KeyedSerializationSchema - - - - - org.apache.flink.streaming.util.serialization.TypeInformationKeyValueSerializationSchema - - - org.apache.amoro.shade.org.apache.flink.streaming.util.serialization.TypeInformationKeyValueSerializationSchema - - - - - - - - - - diff --git a/amoro-format-mixed/amoro-mixed-flink/v1.17/amoro-mixed-flink-1.17/pom.xml b/amoro-format-mixed/amoro-mixed-flink/v1.17/amoro-mixed-flink-1.17/pom.xml deleted file mode 100644 index 5fd713cca7..0000000000 --- a/amoro-format-mixed/amoro-mixed-flink/v1.17/amoro-mixed-flink-1.17/pom.xml +++ /dev/null @@ -1,101 +0,0 @@ - - - - 4.0.0 - - org.apache.amoro - amoro-mixed-flink - 0.8-SNAPSHOT - ../../pom.xml - - - amoro-format-mixed-flink-1.17 - Amoro Project Mixed Format Flink 1.17 - https://amoro.apache.org - - jar - - - 3.2.3 - 3.21.0 - 1.17.2 - 1.17.1 - - - - - org.apache.amoro - amoro-mixed-flink-common - ${project.parent.version} - - - - org.apache.iceberg - iceberg-flink-1.17 - ${iceberg.version} - - - org.slf4j - slf4j-api - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-avro - - - - - - org.apache.paimon - paimon-flink-1.17 - ${paimon.version} - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-amoro - package - - shade - - - - - org.apache.amoro:amoro-format-mixed-flink-common - - - - - - - - - diff --git a/amoro-format-mixed/amoro-mixed-flink/v1.17/amoro-mixed-flink-runtime-1.17/pom.xml b/amoro-format-mixed/amoro-mixed-flink/v1.17/amoro-mixed-flink-runtime-1.17/pom.xml deleted file mode 100644 index 014d8c9d4c..0000000000 --- a/amoro-format-mixed/amoro-mixed-flink/v1.17/amoro-mixed-flink-runtime-1.17/pom.xml +++ /dev/null @@ -1,253 +0,0 @@ - - - - 4.0.0 - - org.apache.amoro - amoro-mixed-flink - 0.8-SNAPSHOT - ../../pom.xml - - - amoro-format-mixed-flink-runtime-1.17 - Amoro Project Mixed Format Flink 1.17 Runtime - https://amoro.apache.org - - - 1.17.1 - - - - - org.apache.amoro - amoro-format-mixed-flink-1.17 - ${project.parent.version} - - - - org.apache.flink - flink-connector-kafka - ${flink.version} - - - com.github.luben - zstd-jni - - - - - - org.apache.kafka - kafka-clients - ${kafka.version} - - - com.github.luben - zstd-jni - - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-amoro - package - - shade - - - false - - - org.apache.amoro:* - org.apache.iceberg:* - com.fasterxml.jackson.core:* - org.apache.parquet:* - org.apache.commons:* - commons-lang:* - com.github.ben-manes.caffeine:* - org.apache.avro:* - org.apache.orc:* - io.airlift:* - commons-collections:* - cglib:* - com.google.guava:* - asm:* - org.apache.httpcomponents.client5:* - org.apache.httpcomponents.core5:* - org.apache.flink:flink-connector-kafka - org.apache.kafka:* - com.github.luben:* - com.github.luben:* - - - - - - org.apache.iceberg - org.apache.amoro.shade.org.apache.iceberg - - org.apache.iceberg.mr.hive.* - - - - - org.apache.parquet - org.apache.amoro.shade.org.apache.parquet - - - - org.apache.commons - org.apache.amoro.shade.org.apache.commons - - - - - org.apache.avro - org.apache.amoro.shade.org.apache.avro - - - - org.apache.orc - org.apache.amoro.shade.org.apache.orc - - - - org.apache.hc - org.apache.amoro.shade.org.apache.hc - - - - org.apache.jute - org.apache.amoro.shade.org.apache.jute - - - - org.apache.kafka - org.apache.amoro.shade.org.apache.kafka - - - - shaded.parquet - org.apache.amoro.shade.shaded.parquet - - - - com.fasterxml - org.apache.amoro.shade.com.fasterxml - - - - com.github.benmanes.caffeine - - org.apache.amoro.shade.com.github.benmanes.caffeine - - - - - org.threeten.extra - org.apache.amoro.shade.org.threeten.extra - - - - net.sf.cglib - org.apache.amoro.shade.net.sf.cglib - - - - com.google - org.apache.amoro.shade.com.google - - - - org.objectweb.asm - org.apache.amoro.shade.org.objectweb.asm - - - - com.facebook.fb303 - org.apache.amoro.shade.com.facebook.fb303 - - - - io.airlift - org.apache.amoro.shade.io.airlift - - - - - org.apache.flink.connector.kafka - - org.apache.amoro.shade.org.apache.flink.connector.kafka - - - - org.apache.flink.streaming.connectors.kafka - - org.apache.amoro.shade.org.apache.flink.streaming.connectors.kafka - - - - - org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema - - - org.apache.amoro.shade.org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema - - - - - org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema - - - org.apache.amoro.shade.org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema - - - - - org.apache.flink.streaming.util.serialization.KeyedSerializationSchema - - - org.apache.amoro.shade.org.apache.flink.streaming.util.serialization.KeyedSerializationSchema - - - - - org.apache.flink.streaming.util.serialization.TypeInformationKeyValueSerializationSchema - - - org.apache.amoro.shade.org.apache.flink.streaming.util.serialization.TypeInformationKeyValueSerializationSchema - - - - - - - - - - diff --git a/amoro-format-mixed/amoro-mixed-flink/v1.16/amoro-mixed-flink-1.16/pom.xml b/amoro-format-mixed/amoro-mixed-flink/v1.18/amoro-mixed-flink-1.18/pom.xml similarity index 90% rename from amoro-format-mixed/amoro-mixed-flink/v1.16/amoro-mixed-flink-1.16/pom.xml rename to amoro-format-mixed/amoro-mixed-flink/v1.18/amoro-mixed-flink-1.18/pom.xml index cee7422677..d9af12ac48 100644 --- a/amoro-format-mixed/amoro-mixed-flink/v1.16/amoro-mixed-flink-1.16/pom.xml +++ b/amoro-format-mixed/amoro-mixed-flink/v1.18/amoro-mixed-flink-1.18/pom.xml @@ -27,17 +27,16 @@ ../../pom.xml - amoro-format-mixed-flink-1.16 - Amoro Project Mixed Format Flink 1.16 + amoro-format-mixed-flink-1.18 + Amoro Project Mixed Format Flink 1.18 https://amoro.apache.org jar - 3.2.3 3.21.0 - 1.17.2 - 1.16.2 + 1.18.3 + 1.18.1 @@ -49,7 +48,7 @@ org.apache.iceberg - iceberg-flink-1.16 + iceberg-flink-1.18 ${iceberg.version} @@ -69,7 +68,7 @@ org.apache.paimon - paimon-flink-1.16 + paimon-flink-1.18 ${paimon.version} @@ -98,4 +97,4 @@ - + \ No newline at end of file diff --git a/amoro-format-mixed/amoro-mixed-flink/v1.16/amoro-mixed-flink-runtime-1.16/pom.xml b/amoro-format-mixed/amoro-mixed-flink/v1.18/amoro-mixed-flink-runtime-1.18/pom.xml similarity index 93% rename from amoro-format-mixed/amoro-mixed-flink/v1.16/amoro-mixed-flink-runtime-1.16/pom.xml rename to amoro-format-mixed/amoro-mixed-flink/v1.18/amoro-mixed-flink-runtime-1.18/pom.xml index 1bfcfb1386..cc52b33324 100644 --- a/amoro-format-mixed/amoro-mixed-flink/v1.16/amoro-mixed-flink-runtime-1.16/pom.xml +++ b/amoro-format-mixed/amoro-mixed-flink/v1.18/amoro-mixed-flink-runtime-1.18/pom.xml @@ -27,37 +27,25 @@ ../../pom.xml - amoro-format-mixed-flink-runtime-1.16 - Amoro Project Mixed Format Flink 1.16 Runtime + amoro-format-mixed-flink-runtime-1.18 + Amoro Project Mixed Format Flink 1.18 Runtime https://amoro.apache.org - 1.16.2 + 1.18.1 org.apache.amoro - amoro-format-mixed-flink-1.16 + amoro-format-mixed-flink-1.18 ${project.parent.version} org.apache.flink flink-connector-kafka - ${flink.version} - - - com.github.luben - zstd-jni - - - - - - org.apache.kafka - kafka-clients - ${kafka.version} + 3.2.0-1.18 com.github.luben @@ -85,7 +73,7 @@ org.apache.amoro:* org.apache.iceberg:* - com.fasterxml.jackson.core:* + org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core:* org.apache.parquet:* org.apache.commons:* commons-lang:* @@ -102,6 +90,7 @@ org.apache.flink:flink-connector-kafka org.apache.kafka:* com.github.luben:* + com.github.luben:* @@ -249,4 +238,4 @@ - + \ No newline at end of file diff --git a/amoro-format-mixed/amoro-mixed-spark/amoro-mixed-spark-3-common/pom.xml b/amoro-format-mixed/amoro-mixed-spark/amoro-mixed-spark-3-common/pom.xml index dd4fa09950..fa53c61222 100644 --- a/amoro-format-mixed/amoro-mixed-spark/amoro-mixed-spark-3-common/pom.xml +++ b/amoro-format-mixed/amoro-mixed-spark/amoro-mixed-spark-3-common/pom.xml @@ -144,7 +144,7 @@ org.apache.iceberg - iceberg-spark-3.2_2.12 + iceberg-spark-3.3_2.12 ${iceberg.version} provided diff --git a/amoro-format-mixed/amoro-mixed-spark/pom.xml b/amoro-format-mixed/amoro-mixed-spark/pom.xml index 306eda8e8e..10be997fc7 100644 --- a/amoro-format-mixed/amoro-mixed-spark/pom.xml +++ b/amoro-format-mixed/amoro-mixed-spark/pom.xml @@ -33,8 +33,6 @@ amoro-mixed-spark-3-common - v3.2/amoro-mixed-spark-3.2 - v3.2/amoro-mixed-spark-runtime-3.2 v3.3/amoro-mixed-spark-3.3 v3.3/amoro-mixed-spark-runtime-3.3 diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/pom.xml b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/pom.xml deleted file mode 100644 index cc4e7a8d3d..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/pom.xml +++ /dev/null @@ -1,473 +0,0 @@ - - - - - - amoro-mixed-spark - org.apache.amoro - 0.8-SNAPSHOT - ../../pom.xml - - 4.0.0 - - amoro-format-mixed-spark-3.2 - jar - Amoro Project Mixed Format Spark 3.2 - https://amoro.apache.org - - - 2.3.9 - 3.2.4 - 2.12.15 - 2.11.0 - - - - - org.scala-lang.modules - scala-collection-compat_${scala.binary.version} - ${scala.collection.compat} - - - - org.apache.spark - spark-sql_2.12 - ${spark.version} - provided - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-hadoop - - - org.apache.parquet - parquet-hadoop - - - org.slf4j - slf4j-api - - - org.apache.avro - avro - - - org.apache.arrow - arrow-memory-core - - - org.apache.arrow - arrow-memory-netty - - - org.apache.arrow - arrow-vector - - - org.apache.orc - orc-core - - - org.apache.orc - orc-mapreduce - - - - - - org.scala-lang - scala-library - ${scala.version} - - - - org.scala-lang - scala-compiler - ${scala.version} - - - - org.apache.spark - spark-core_2.12 - ${spark.version} - provided - - - com.google.guava - guava - - - org.apache.hadoop - hadoop-client-api - - - org.slf4j - slf4j-api - - - org.apache.avro - avro - - - org.apache.arrow - arrow-memory-core - - - org.apache.arrow - arrow-memory-netty - - - org.apache.arrow - arrow-vector - - - log4j - log4j - - - - - - org.apache.spark - spark-hive_2.12 - ${spark.version} - provided - - - org.apache.hive - hive-metastore - - - commons-logging - commons-logging - - - com.google.guava - guava - - - org.slf4j - slf4j-api - - - org.apache.avro - avro - - - org.apache.arrow - arrow-memory-core - - - org.apache.arrow - arrow-memory-netty - - - org.apache.arrow - arrow-vector - - - org.apache.hive - * - - - - - - - org.apache.amoro - amoro-common - ${project.version} - - - org.apache.hadoop - hadoop-hdfs - - - commons-logging - commons-logging - - - javax.servlet - servlet-api - - - com.google.guava - guava - - - - - - org.apache.amoro - amoro-mixed-hive - ${project.version} - - - com.google.guava - guava - - - org.apache.hive - * - - - - - - org.apache.amoro - amoro-format-mixed-spark-3-common - ${project.version} - - - - org.apache.iceberg - iceberg-spark-3.2_2.12 - ${iceberg.version} - - - org.apache.parquet - parquet-column - - - - - - org.apache.iceberg - iceberg-spark-extensions-3.2_2.12 - ${iceberg.version} - - - org.apache.parquet - parquet-column - - - - - - - org.apache.paimon - paimon-spark-3.2 - ${paimon.version} - test - - - org.apache.paimon - paimon-hive-connector-3.1 - ${paimon.version} - test - - - - org.junit.platform - junit-platform-launcher - test - - - org.junit.platform - junit-platform-suite-engine - test - - - - org.apache.iceberg - iceberg-api - ${iceberg.version} - test-jar - test - - - - com.google.guava - guava - ${guava.version} - test - - - - org.apache.amoro - amoro-common - ${project.version} - test - test-jar - - - - org.apache.amoro - amoro-format-paimon - ${project.version} - test - - - - org.apache.hive - hive-metastore - ${hive.version} - test - - - jdk.tools - jdk.tools - - - com.google.guava - guava - - - com.fasterxml.jackson.core - * - - - log4j - log4j - - - - - - org.apache.amoro - amoro-mixed-hive - ${project.version} - test-jar - test - - - org.apache.hive - * - - - - - - org.apache.amoro - amoro-format-mixed-spark-3-common - ${project.version} - test-jar - test - - - org.apache.amoro - amoro-format-iceberg - ${project.version} - test-jar - test - - - - org.apache.hive - hive-exec - ${hive.version} - test - core - - - com.google.guava - guava - - - org.pentaho - * - - - com.fasterxml.jackson.core - * - - - org.codehaus.janino - * - - - org.apache.orc - * - - - - - - - src/main/java - - - src/main/resources - - - - - org.antlr - antlr4-maven-plugin - 4.8 - - true - true - ${basedir}/src/main/gen-antlr - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.2.0 - - - add-source - generate-sources - - add-source - - - - src/main/gen-antlr - - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - net.alchim31.maven - scala-maven-plugin - - ${scala.version} - - - - org.jacoco - jacoco-maven-plugin - - - - **/org/apache/amoro/spark/sql/parser/**.class - **/org/apache/spark/sql/amoro/parser/**.class - **/org/apache/iceberg/spark/data/**.class - **/org/apache/amoro/spark/MultiDelegateSessionCatalog* - **/org/apache/amoro/spark/reader/SparkParquetReaders* - - - - - - - diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/antlr4/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtend.g4 b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/antlr4/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtend.g4 deleted file mode 100644 index b8cbdeb137..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/antlr4/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtend.g4 +++ /dev/null @@ -1,1576 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - * This file is an adaptation of Presto's presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 grammar. - */ - -grammar MixedFormatSqlExtend; - -@parser::members { - /** - * When false, INTERSECT is given the greater precedence over the other set - * operations (UNION, EXCEPT and MINUS) as per the SQL standard. - */ - public boolean legacy_setops_precedence_enabled = false; - - /** - * When false, a literal with an exponent would be converted into - * double type rather than decimal type. - */ - public boolean legacy_exponent_literal_as_decimal_enabled = false; - - /** - * When true, the behavior of keywords follows ANSI SQL standard. - */ - public boolean SQL_standard_keyword_behavior = false; -} - -@lexer::members { - /** - * When true, parser should throw ParseExcetion for unclosed bracketed comment. - */ - public boolean has_unclosed_bracketed_comment = false; - - /** - * Verify whether current token is a valid decimal token (which contains dot). - * Returns true if the character that follows the token is not a digit or letter or underscore. - * - * For example: - * For char stream "2.3", "2." is not a valid decimal token, because it is followed by digit '3'. - * For char stream "2.3_", "2.3" is not a valid decimal token, because it is followed by '_'. - * For char stream "2.3W", "2.3" is not a valid decimal token, because it is followed by 'W'. - * For char stream "12.0D 34.E2+0.12 " 12.0D is a valid decimal token because it is followed - * by a space. 34.E2 is a valid decimal token because it is followed by symbol '+' - * which is not a digit or letter or underscore. - */ - public boolean isValidDecimal() { - int nextChar = _input.LA(1); - if (nextChar >= 'A' && nextChar <= 'Z' || nextChar >= '0' && nextChar <= '9' || - nextChar == '_') { - return false; - } else { - return true; - } - } - - /** - * This method will be called when we see '/*' and try to match it as a bracketed comment. - * If the next character is '+', it should be parsed as hint later, and we cannot match - * it as a bracketed comment. - * - * Returns true if the next character is '+'. - */ - public boolean isHint() { - int nextChar = _input.LA(1); - if (nextChar == '+') { - return true; - } else { - return false; - } - } - - /** - * This method will be called when the character stream ends and try to find out the - * unclosed bracketed comment. - * If the method be called, it means the end of the entire character stream match, - * and we set the flag and fail later. - */ - public void markUnclosedComment() { - has_unclosed_bracketed_comment = true; - } -} - - -extendStatement - : statement ';'* EOF - ; - -statement - : createTableHeader colListAndPk tableProvider? - createTableClauses - (AS? query)? #createTableWithPk - | EXPLAIN (LOGICAL | FORMATTED | EXTENDED | CODEGEN | COST)? - statement #explain - ; - -createTableHeader - : CREATE TEMPORARY? EXTERNAL? TABLE (IF NOT EXISTS)? multipartIdentifier - ; - -colListAndPk - :'(' colTypeList (',' primarySpec )? ')' #colListWithPk - | primarySpec #colListOnlyPk - ; - -primarySpec - : PRIMARY KEY identifierList - ; - -bucketSpec - : CLUSTERED BY identifierList - (SORTED BY orderedIdentifierList)? - INTO INTEGER_VALUE BUCKETS - ; - -skewSpec - : SKEWED BY identifierList - ON (constantList | nestedConstantList) - (STORED AS DIRECTORIES)? - ; - -locationSpec - : LOCATION STRING - ; - -commentSpec - : COMMENT STRING - ; - -query - : ctes? queryTerm queryOrganization - ; - - -ctes - : WITH namedQuery (',' namedQuery)* - ; - -namedQuery - : name=errorCapturingIdentifier (columnAliases=identifierList)? AS? '(' query ')' - ; - -tableProvider - : USING multipartIdentifier - ; - -createTableClauses - :((OPTIONS options=tablePropertyList) | - (PARTITIONED BY partitioning=partitionFieldList) | - skewSpec | - bucketSpec | - rowFormat | - createFileFormat | - locationSpec | - commentSpec | - (TBLPROPERTIES tableProps=tablePropertyList))* - ; - -tablePropertyList - : '(' tableProperty (',' tableProperty)* ')' - ; - -tableProperty - : key=tablePropertyKey (EQ? value=tablePropertyValue)? - ; - -tablePropertyKey - : identifier ('.' identifier)* - | STRING - ; - -tablePropertyValue - : INTEGER_VALUE - | DECIMAL_VALUE - | booleanValue - | STRING - ; - -constantList - : '(' constant (',' constant)* ')' - ; - -nestedConstantList - : '(' constantList (',' constantList)* ')' - ; - -createFileFormat - : STORED AS fileFormat - | STORED BY storageHandler - ; - -fileFormat - : INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING #tableFileFormat - | identifier #genericFileFormat - ; - -storageHandler - : STRING (WITH SERDEPROPERTIES tablePropertyList)? - ; - - - -queryOrganization - : (ORDER BY order+=sortItem (',' order+=sortItem)*)? - (CLUSTER BY clusterBy+=expression (',' clusterBy+=expression)*)? - (DISTRIBUTE BY distributeBy+=expression (',' distributeBy+=expression)*)? - (SORT BY sort+=sortItem (',' sort+=sortItem)*)? - windowClause? - (LIMIT (ALL | limit=expression))? - ; - - - -queryTerm - : queryPrimary #queryTermDefault - | left=queryTerm {legacy_setops_precedence_enabled}? - operator=(INTERSECT | UNION | EXCEPT | SETMINUS) setQuantifier? right=queryTerm #setOperation - | left=queryTerm {!legacy_setops_precedence_enabled}? - operator=INTERSECT setQuantifier? right=queryTerm #setOperation - | left=queryTerm {!legacy_setops_precedence_enabled}? - operator=(UNION | EXCEPT | SETMINUS) setQuantifier? right=queryTerm #setOperation - ; - -queryPrimary - : querySpecification #queryPrimaryDefault - | fromStatement #fromStmt - | TABLE multipartIdentifier #table - | inlineTable #inlineTableDefault1 - | '(' query ')' #subquery - ; - -sortItem - : expression ordering=(ASC | DESC)? (NULLS nullOrder=(LAST | FIRST))? - ; - -fromStatement - : fromClause fromStatementBody+ - ; - -fromStatementBody - : transformClause - whereClause? - queryOrganization - | selectClause - lateralView* - whereClause? - aggregationClause? - havingClause? - windowClause? - queryOrganization - ; - -querySpecification - : transformClause - fromClause? - lateralView* - whereClause? - aggregationClause? - havingClause? - windowClause? #transformQuerySpecification - | selectClause - fromClause? - lateralView* - whereClause? - aggregationClause? - havingClause? - windowClause? #regularQuerySpecification - ; - -transformClause - : (SELECT kind=TRANSFORM '(' setQuantifier? expressionSeq ')' - | kind=MAP setQuantifier? expressionSeq - | kind=REDUCE setQuantifier? expressionSeq) - inRowFormat=rowFormat? - (RECORDWRITER recordWriter=STRING)? - USING script=STRING - (AS (identifierSeq | colTypeList | ('(' (identifierSeq | colTypeList) ')')))? - outRowFormat=rowFormat? - (RECORDREADER recordReader=STRING)? - ; - -selectClause - : SELECT (hints+=hint)* setQuantifier? namedExpressionSeq - ; - - - -whereClause - : WHERE booleanExpression - ; - -havingClause - : HAVING booleanExpression - ; - -hint - : '/*+' hintStatements+=hintStatement (','? hintStatements+=hintStatement)* '*/' - ; - -hintStatement - : hintName=identifier - | hintName=identifier '(' parameters+=primaryExpression (',' parameters+=primaryExpression)* ')' - ; - -fromClause - : FROM relation (',' relation)* lateralView* pivotClause? - ; - -aggregationClause - : GROUP BY groupingExpressionsWithGroupingAnalytics+=groupByClause - (',' groupingExpressionsWithGroupingAnalytics+=groupByClause)* - | GROUP BY groupingExpressions+=expression (',' groupingExpressions+=expression)* ( - WITH kind=ROLLUP - | WITH kind=CUBE - | kind=GROUPING SETS '(' groupingSet (',' groupingSet)* ')')? - ; - -groupByClause - : groupingAnalytics - | expression - ; - -groupingAnalytics - : (ROLLUP | CUBE) '(' groupingSet (',' groupingSet)* ')' - | GROUPING SETS '(' groupingElement (',' groupingElement)* ')' - ; - -groupingElement - : groupingAnalytics - | groupingSet - ; - -groupingSet - : '(' (expression (',' expression)*)? ')' - | expression - ; - -pivotClause - : PIVOT '(' aggregates=namedExpressionSeq FOR pivotColumn IN '(' pivotValues+=pivotValue (',' pivotValues+=pivotValue)* ')' ')' - ; - -pivotColumn - : identifiers+=identifier - | '(' identifiers+=identifier (',' identifiers+=identifier)* ')' - ; - -pivotValue - : expression (AS? identifier)? - ; - -lateralView - : LATERAL VIEW (OUTER)? qualifiedName '(' (expression (',' expression)*)? ')' tblName=identifier (AS? colName+=identifier (',' colName+=identifier)*)? - ; - -setQuantifier - : DISTINCT - | ALL - ; - -relation - : LATERAL? relationPrimary joinRelation* - ; - -joinRelation - : (joinType) JOIN LATERAL? right=relationPrimary joinCriteria? - | NATURAL joinType JOIN LATERAL? right=relationPrimary - ; - -joinType - : INNER? - | CROSS - | LEFT OUTER? - | LEFT? SEMI - | RIGHT OUTER? - | FULL OUTER? - | LEFT? ANTI - ; - -joinCriteria - : ON booleanExpression - | USING identifierList - ; - -sample - : TABLESAMPLE '(' sampleMethod? ')' - ; - -sampleMethod - : negativeSign=MINUS? percentage=(INTEGER_VALUE | DECIMAL_VALUE) PERCENTLIT #sampleByPercentile - | expression ROWS #sampleByRows - | sampleType=BUCKET numerator=INTEGER_VALUE OUT OF denominator=INTEGER_VALUE - (ON (identifier | qualifiedName '(' ')'))? #sampleByBucket - | bytes=expression #sampleByBytes - ; - -identifierList - : '(' identifierSeq ')' - ; - -identifierSeq - : ident+=errorCapturingIdentifier (',' ident+=errorCapturingIdentifier)* - ; - -orderedIdentifierList - : '(' orderedIdentifier (',' orderedIdentifier)* ')' - ; - -orderedIdentifier - : ident=errorCapturingIdentifier ordering=(ASC | DESC)? - ; - - - - -relationPrimary - : multipartIdentifier sample? tableAlias #tableName - | '(' query ')' sample? tableAlias #aliasedQuery - | '(' relation ')' sample? tableAlias #aliasedRelation - | inlineTable #inlineTableDefault2 - | functionTable #tableValuedFunction - ; - -inlineTable - : VALUES expression (',' expression)* tableAlias - ; - -functionTable - : funcName=functionName '(' (expression (',' expression)*)? ')' tableAlias - ; - -tableAlias - : (AS? strictIdentifier identifierList?)? - ; - -rowFormat - : ROW FORMAT SERDE name=STRING (WITH SERDEPROPERTIES props=tablePropertyList)? #rowFormatSerde - | ROW FORMAT DELIMITED - (FIELDS TERMINATED BY fieldsTerminatedBy=STRING (ESCAPED BY escapedBy=STRING)?)? - (COLLECTION ITEMS TERMINATED BY collectionItemsTerminatedBy=STRING)? - (MAP KEYS TERMINATED BY keysTerminatedBy=STRING)? - (LINES TERMINATED BY linesSeparatedBy=STRING)? - (NULL DEFINED AS nullDefinedAs=STRING)? #rowFormatDelimited - ; - -multipartIdentifier - : parts+=errorCapturingIdentifier ('.' parts+=errorCapturingIdentifier)* - ; - -namedExpression - : expression (AS? (name=errorCapturingIdentifier | identifierList))? - ; - -namedExpressionSeq - : namedExpression (',' namedExpression)* - ; - -partitionFieldList - : '(' fields+=partitionField (',' fields+=partitionField)* ')' - ; - -partitionField - : transform #partitionTransform - | colType #partitionColumn - ; - -transform - : qualifiedName #identityTransform - | transformName=identifier - '(' argument+=transformArgument (',' argument+=transformArgument)* ')' #applyTransform - ; - -transformArgument - : qualifiedName - | constant - ; - -expression - : booleanExpression - ; - -expressionSeq - : expression (',' expression)* - ; - -booleanExpression - : NOT booleanExpression #logicalNot - | EXISTS '(' query ')' #exists - | valueExpression predicate? #predicated - | left=booleanExpression operator=AND right=booleanExpression #logicalBinary - | left=booleanExpression operator=OR right=booleanExpression #logicalBinary - ; - -predicate - : NOT? kind=BETWEEN lower=valueExpression AND upper=valueExpression - | NOT? kind=IN '(' expression (',' expression)* ')' - | NOT? kind=IN '(' query ')' - | NOT? kind=RLIKE pattern=valueExpression - | NOT? kind=LIKE quantifier=(ANY | SOME | ALL) ('('')' | '(' expression (',' expression)* ')') - | NOT? kind=LIKE pattern=valueExpression (ESCAPE escapeChar=STRING)? - | IS NOT? kind=NULL - | IS NOT? kind=(TRUE | FALSE | UNKNOWN) - | IS NOT? kind=DISTINCT FROM right=valueExpression - ; - -valueExpression - : primaryExpression #valueExpressionDefault - | operator=(MINUS | PLUS | TILDE) valueExpression #arithmeticUnary - | left=valueExpression operator=(ASTERISK | SLASH | PERCENT | DIV) right=valueExpression #arithmeticBinary - | left=valueExpression operator=(PLUS | MINUS | CONCAT_PIPE) right=valueExpression #arithmeticBinary - | left=valueExpression operator=AMPERSAND right=valueExpression #arithmeticBinary - | left=valueExpression operator=HAT right=valueExpression #arithmeticBinary - | left=valueExpression operator=PIPE right=valueExpression #arithmeticBinary - | left=valueExpression comparisonOperator right=valueExpression #comparison - ; - -primaryExpression - : name=(CURRENT_DATE | CURRENT_TIMESTAMP | CURRENT_USER) #currentLike - | CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase - | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase - | name=(CAST | TRY_CAST) '(' expression AS dataType ')' #cast - | STRUCT '(' (argument+=namedExpression (',' argument+=namedExpression)*)? ')' #struct - | FIRST '(' expression (IGNORE NULLS)? ')' #first - | LAST '(' expression (IGNORE NULLS)? ')' #last - | POSITION '(' substr=valueExpression IN str=valueExpression ')' #position - | constant #constantDefault - | ASTERISK #star - | qualifiedName '.' ASTERISK #star - | '(' namedExpression (',' namedExpression)+ ')' #rowConstructor - | '(' query ')' #subqueryExpression - | functionName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' - (FILTER '(' WHERE where=booleanExpression ')')? - (nullsOption=(IGNORE | RESPECT) NULLS)? ( OVER windowSpec)? #functionCall - | identifier '->' expression #lambda - | '(' identifier (',' identifier)+ ')' '->' expression #lambda - | value=primaryExpression '[' index=valueExpression ']' #subscript - | identifier #columnReference - | base=primaryExpression '.' fieldName=identifier #dereference - | '(' expression ')' #parenthesizedExpression - | EXTRACT '(' field=identifier FROM source=valueExpression ')' #extract - | (SUBSTR | SUBSTRING) '(' str=valueExpression (FROM | ',') pos=valueExpression - ((FOR | ',') len=valueExpression)? ')' #substring - | TRIM '(' trimOption=(BOTH | LEADING | TRAILING)? (trimStr=valueExpression)? - FROM srcStr=valueExpression ')' #trim - | OVERLAY '(' input=valueExpression PLACING replace=valueExpression - FROM position=valueExpression (FOR length=valueExpression)? ')' #overlay - ; - -constant - : NULL #nullLiteral - | interval #intervalLiteral - | identifier STRING #typeConstructor - | number #numericLiteral - | booleanValue #booleanLiteral - | STRING+ #stringLiteral - ; - -comparisonOperator - : EQ | NEQ | NEQJ | LT | LTE | GT | GTE | NSEQ - ; - -booleanValue - : TRUE | FALSE - ; - -interval - : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? - ; - -errorCapturingMultiUnitsInterval - : body=multiUnitsInterval unitToUnitInterval? - ; - -multiUnitsInterval - : (intervalValue unit+=identifier)+ - ; - -errorCapturingUnitToUnitInterval - : body=unitToUnitInterval (error1=multiUnitsInterval | error2=unitToUnitInterval)? - ; - -unitToUnitInterval - : value=intervalValue from=identifier TO to=identifier - ; - -intervalValue - : (PLUS | MINUS)? (INTEGER_VALUE | DECIMAL_VALUE | STRING) - ; - -colPosition - : position=FIRST | position=AFTER afterCol=errorCapturingIdentifier - ; - -dataType - : complex=ARRAY '<' dataType '>' #complexDataType - | complex=MAP '<' dataType ',' dataType '>' #complexDataType - | complex=STRUCT ('<' complexColTypeList? '>' | NEQ) #complexDataType - | INTERVAL from=(YEAR | MONTH) (TO to=MONTH)? #yearMonthIntervalDataType - | INTERVAL from=(DAY | HOUR | MINUTE | SECOND) - (TO to=(HOUR | MINUTE | SECOND))? #dayTimeIntervalDataType - | identifier ('(' INTEGER_VALUE (',' INTEGER_VALUE)* ')')? #primitiveDataType - ; - - -colTypeList - : colType (',' colType)* - ; - -colType - : colName=errorCapturingIdentifier dataType (NOT NULL)? commentSpec? - ; - -complexColTypeList - : complexColType (',' complexColType)* - ; - -complexColType - : identifier ':'? dataType (NOT NULL)? commentSpec? - ; - -whenClause - : WHEN condition=expression THEN result=expression - ; - -windowClause - : WINDOW namedWindow (',' namedWindow)* - ; - -namedWindow - : name=errorCapturingIdentifier AS windowSpec - ; - -windowSpec - : name=errorCapturingIdentifier #windowRef - | '('name=errorCapturingIdentifier')' #windowRef - | '(' - ( CLUSTER BY partition+=expression (',' partition+=expression)* - | ((PARTITION | DISTRIBUTE) BY partition+=expression (',' partition+=expression)*)? - ((ORDER | SORT) BY sortItem (',' sortItem)*)?) - windowFrame? - ')' #windowDef - ; - -windowFrame - : frameType=RANGE start=frameBound - | frameType=ROWS start=frameBound - | frameType=RANGE BETWEEN start=frameBound AND end=frameBound - | frameType=ROWS BETWEEN start=frameBound AND end=frameBound - ; - -frameBound - : UNBOUNDED boundType=(PRECEDING | FOLLOWING) - | boundType=CURRENT ROW - | expression boundType=(PRECEDING | FOLLOWING) - ; - - -functionName - : qualifiedName - | FILTER - | LEFT - | RIGHT - ; - -qualifiedName - : identifier ('.' identifier)* - ; - -// this rule is used for explicitly capturing wrong identifiers such as test-table, which should actually be `test-table` -// replace identifier with errorCapturingIdentifier where the immediate follow symbol is not an expression, otherwise -// valid expressions such as "a-b" can be recognized as an identifier -errorCapturingIdentifier - : identifier errorCapturingIdentifierExtra - ; - -// extra left-factoring grammar -errorCapturingIdentifierExtra - : (MINUS identifier)+ #errorIdent - | #realIdent - ; - -identifier - : strictIdentifier - | {!SQL_standard_keyword_behavior}? strictNonReserved - ; - -strictIdentifier - : IDENTIFIER #unquotedIdentifier - | quotedIdentifier #quotedIdentifierAlternative - | {SQL_standard_keyword_behavior}? ansiNonReserved #unquotedIdentifier - | {!SQL_standard_keyword_behavior}? nonReserved #unquotedIdentifier - ; - -quotedIdentifier - : BACKQUOTED_IDENTIFIER - ; - -number - : {!legacy_exponent_literal_as_decimal_enabled}? MINUS? EXPONENT_VALUE #exponentLiteral - | {!legacy_exponent_literal_as_decimal_enabled}? MINUS? DECIMAL_VALUE #decimalLiteral - | {legacy_exponent_literal_as_decimal_enabled}? MINUS? (EXPONENT_VALUE | DECIMAL_VALUE) #legacyDecimalLiteral - | MINUS? INTEGER_VALUE #integerLiteral - | MINUS? BIGINT_LITERAL #bigIntLiteral - | MINUS? SMALLINT_LITERAL #smallIntLiteral - | MINUS? TINYINT_LITERAL #tinyIntLiteral - | MINUS? DOUBLE_LITERAL #doubleLiteral - | MINUS? FLOAT_LITERAL #floatLiteral - | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral - ; - - -// When `SQL_standard_keyword_behavior=true`, there are 2 kinds of keywords in Spark SQL. -// - Reserved keywords: -// Keywords that are reserved and can't be used as identifiers for table, view, column, -// function, alias, etc. -// - Non-reserved keywords: -// Keywords that have a special meaning only in particular contexts and can be used as -// identifiers in other contexts. For example, `EXPLAIN SELECT ...` is a command, but EXPLAIN -// can be used as identifiers in other places. -// You can find the full keywords list by searching "Start of the keywords list" in this file. -// The non-reserved keywords are listed below. Keywords not in this list are reserved keywords. -ansiNonReserved -//--ANSI-NON-RESERVED-START - : ADD - | AFTER - | ALTER - | ANALYZE - | ANTI - | ARCHIVE - | ARRAY - | ASC - | AT - | BETWEEN - | BUCKET - | BUCKETS - | BY - | CACHE - | CASCADE - | CHANGE - | CLEAR - | CLUSTER - | CLUSTERED - | CODEGEN - | COLLECTION - | COLUMNS - | COMMENT - | COMMIT - | COMPACT - | COMPACTIONS - | COMPUTE - | CONCATENATE - | COST - | CUBE - | CURRENT - | DATA - | DATABASE - | DATABASES - | DAY - | DBPROPERTIES - | DEFINED - | DELETE - | DELIMITED - | DESC - | DESCRIBE - | DFS - | DIRECTORIES - | DIRECTORY - | DISTRIBUTE - | DIV - | DROP - | ESCAPED - | EXCHANGE - | EXISTS - | EXPLAIN - | EXPORT - | EXTENDED - | EXTERNAL - | EXTRACT - | FIELDS - | FILEFORMAT - | FIRST - | FOLLOWING - | FORMAT - | FORMATTED - | FUNCTION - | FUNCTIONS - | GLOBAL - | GROUPING - | HOUR - | IF - | IGNORE - | IMPORT - | INDEX - | INDEXES - | INPATH - | INPUTFORMAT - | INSERT - | INTERVAL - | ITEMS - | KEYS - | LAST - | LAZY - | LIKE - | LIMIT - | LINES - | LIST - | LOAD - | LOCAL - | LOCATION - | LOCK - | LOCKS - | LOGICAL - | MACRO - | MAP - | MATCHED - | MERGE - | MINUTE - | MONTH - | MSCK - | NAMESPACE - | NAMESPACES - | NO - | NULLS - | OF - | OPTION - | OPTIONS - | OUT - | OUTPUTFORMAT - | OVER - | OVERLAY - | OVERWRITE - | PARTITION - | PARTITIONED - | PARTITIONS - | PERCENTLIT - | PIVOT - | PLACING - | POSITION - | PRECEDING - | PRINCIPALS - | PROPERTIES - | PURGE - | QUERY - | RANGE - | RECORDREADER - | RECORDWRITER - | RECOVER - | REDUCE - | REFRESH - | RENAME - | REPAIR - | REPLACE - | RESET - | RESPECT - | RESTRICT - | REVOKE - | RLIKE - | ROLE - | ROLES - | ROLLBACK - | ROLLUP - | ROW - | ROWS - | SCHEMA - | SECOND - | SEMI - | SEPARATED - | SERDE - | SERDEPROPERTIES - | SET - | SETMINUS - | SETS - | SHOW - | SKEWED - | SORT - | SORTED - | START - | STATISTICS - | STORED - | STRATIFY - | STRUCT - | SUBSTR - | SUBSTRING - | SYNC - | TABLES - | TABLESAMPLE - | TBLPROPERTIES - | TEMPORARY - | TERMINATED - | TOUCH - | TRANSACTION - | TRANSACTIONS - | TRANSFORM - | TRIM - | TRUE - | TRUNCATE - | TRY_CAST - | TYPE - | UNARCHIVE - | UNBOUNDED - | UNCACHE - | UNLOCK - | UNSET - | UPDATE - | USE - | VALUES - | VIEW - | VIEWS - | WINDOW - | YEAR - | ZONE -//--ANSI-NON-RESERVED-END - ; - -// When `SQL_standard_keyword_behavior=false`, there are 2 kinds of keywords in Spark SQL. -// - Non-reserved keywords: -// Same definition as the one when `SQL_standard_keyword_behavior=true`. -// - Strict-non-reserved keywords: -// A strict version of non-reserved keywords, which can not be used as table alias. -// You can find the full keywords list by searching "Start of the keywords list" in this file. -// The strict-non-reserved keywords are listed in `strictNonReserved`. -// The non-reserved keywords are listed in `nonReserved`. -// These 2 together contain all the keywords. -strictNonReserved - : ANTI - | CROSS - | EXCEPT - | FULL - | INNER - | INTERSECT - | JOIN - | LATERAL - | LEFT - | NATURAL - | ON - | RIGHT - | SEMI - | SETMINUS - | UNION - | USING - ; - -nonReserved -//--DEFAULT-NON-RESERVED-START - : ADD - | AFTER - | ALL - | ALTER - | ANALYZE - | AND - | ANY - | ARCHIVE - | ARRAY - | AS - | ASC - | AT - | AUTHORIZATION - | BETWEEN - | BOTH - | BUCKET - | BUCKETS - | BY - | CACHE - | CASCADE - | CASE - | CAST - | CHANGE - | CHECK - | CLEAR - | CLUSTER - | CLUSTERED - | CODEGEN - | COLLATE - | COLLECTION - | COLUMN - | COLUMNS - | COMMENT - | COMMIT - | COMPACT - | COMPACTIONS - | COMPUTE - | CONCATENATE - | CONSTRAINT - | COST - | CREATE - | CUBE - | CURRENT - | CURRENT_DATE - | CURRENT_TIME - | CURRENT_TIMESTAMP - | CURRENT_USER - | DATA - | DATABASE - | DATABASES - | DAY - | DBPROPERTIES - | DEFINED - | DELETE - | DELIMITED - | DESC - | DESCRIBE - | DFS - | DIRECTORIES - | DIRECTORY - | DISTINCT - | DISTRIBUTE - | DIV - | DROP - | ELSE - | END - | ESCAPE - | ESCAPED - | EXCHANGE - | EXISTS - | EXPLAIN - | EXPORT - | EXTENDED - | EXTERNAL - | EXTRACT - | FALSE - | FETCH - | FILTER - | FIELDS - | FILEFORMAT - | FIRST - | FOLLOWING - | FOR - | FOREIGN - | FORMAT - | FORMATTED - | FROM - | FUNCTION - | FUNCTIONS - | GLOBAL - | GRANT - | GROUP - | GROUPING - | HAVING - | HOUR - | IF - | IGNORE - | IMPORT - | IN - | INDEX - | INDEXES - | INPATH - | INPUTFORMAT - | INSERT - | INTERVAL - | INTO - | IS - | ITEMS - | KEYS - | LAST - | LAZY - | LEADING - | LIKE - | LIMIT - | LINES - | LIST - | LOAD - | LOCAL - | LOCATION - | LOCK - | LOCKS - | LOGICAL - | MACRO - | MAP - | MATCHED - | MERGE - | MINUTE - | MONTH - | MSCK - | NAMESPACE - | NAMESPACES - | NO - | NOT - | NULL - | NULLS - | OF - | ONLY - | OPTION - | OPTIONS - | OR - | ORDER - | OUT - | OUTER - | OUTPUTFORMAT - | OVER - | OVERLAPS - | OVERLAY - | OVERWRITE - | PARTITION - | PARTITIONED - | PARTITIONS - | PERCENTLIT - | PIVOT - | PLACING - | POSITION - | PRECEDING - | PRIMARY - | PRINCIPALS - | PROPERTIES - | PURGE - | QUERY - | RANGE - | RECORDREADER - | RECORDWRITER - | RECOVER - | REDUCE - | REFERENCES - | REFRESH - | RENAME - | REPAIR - | REPLACE - | RESET - | RESPECT - | RESTRICT - | REVOKE - | RLIKE - | ROLE - | ROLES - | ROLLBACK - | ROLLUP - | ROW - | ROWS - | SCHEMA - | SECOND - | SELECT - | SEPARATED - | SERDE - | SERDEPROPERTIES - | SESSION_USER - | SET - | SETS - | SHOW - | SKEWED - | SOME - | SORT - | SORTED - | START - | STATISTICS - | STORED - | STRATIFY - | STRUCT - | SUBSTR - | SUBSTRING - | SYNC - | TABLE - | TABLES - | TABLESAMPLE - | TBLPROPERTIES - | TEMPORARY - | TERMINATED - | THEN - | TIME - | TO - | TOUCH - | TRAILING - | TRANSACTION - | TRANSACTIONS - | TRANSFORM - | TRIM - | TRUE - | TRUNCATE - | TRY_CAST - | TYPE - | UNARCHIVE - | UNBOUNDED - | UNCACHE - | UNIQUE - | UNKNOWN - | UNLOCK - | UNSET - | UPDATE - | USE - | USER - | VALUES - | VIEW - | VIEWS - | WHEN - | WHERE - | WINDOW - | WITH - | YEAR - | ZONE -//--DEFAULT-NON-RESERVED-END - ; - -// NOTE: If you add a new token in the list below, you should update the list of keywords -// and reserved tag in `docs/sql-ref-ansi-compliance.md#sql-keywords`. - -//============================ -// Start of the keywords list -//============================ -//--SPARK-KEYWORD-LIST-START -ADD: 'ADD'; -AFTER: 'AFTER'; -ALL: 'ALL'; -ALTER: 'ALTER'; -ANALYZE: 'ANALYZE'; -AND: 'AND'; -ANTI: 'ANTI'; -ANY: 'ANY'; -ARCHIVE: 'ARCHIVE'; -ARRAY: 'ARRAY'; -AS: 'AS'; -ASC: 'ASC'; -AT: 'AT'; -AUTHORIZATION: 'AUTHORIZATION'; -BETWEEN: 'BETWEEN'; -BOTH: 'BOTH'; -BUCKET: 'BUCKET'; -BUCKETS: 'BUCKETS'; -BY: 'BY'; -CACHE: 'CACHE'; -CASCADE: 'CASCADE'; -CASE: 'CASE'; -CAST: 'CAST'; -CHANGE: 'CHANGE'; -CHECK: 'CHECK'; -CLEAR: 'CLEAR'; -CLUSTER: 'CLUSTER'; -CLUSTERED: 'CLUSTERED'; -CODEGEN: 'CODEGEN'; -COLLATE: 'COLLATE'; -COLLECTION: 'COLLECTION'; -COLUMN: 'COLUMN'; -COLUMNS: 'COLUMNS'; -COMMENT: 'COMMENT'; -COMMIT: 'COMMIT'; -COMPACT: 'COMPACT'; -COMPACTIONS: 'COMPACTIONS'; -COMPUTE: 'COMPUTE'; -CONCATENATE: 'CONCATENATE'; -CONSTRAINT: 'CONSTRAINT'; -COST: 'COST'; -CREATE: 'CREATE'; -CROSS: 'CROSS'; -CUBE: 'CUBE'; -CURRENT: 'CURRENT'; -CURRENT_DATE: 'CURRENT_DATE'; -CURRENT_TIME: 'CURRENT_TIME'; -CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP'; -CURRENT_USER: 'CURRENT_USER'; -DAY: 'DAY'; -DATA: 'DATA'; -DATABASE: 'DATABASE'; -DATABASES: 'DATABASES' | 'SCHEMAS'; -DBPROPERTIES: 'DBPROPERTIES'; -DEFINED: 'DEFINED'; -DELETE: 'DELETE'; -DELIMITED: 'DELIMITED'; -DESC: 'DESC'; -DESCRIBE: 'DESCRIBE'; -DFS: 'DFS'; -DIRECTORIES: 'DIRECTORIES'; -DIRECTORY: 'DIRECTORY'; -DISTINCT: 'DISTINCT'; -DISTRIBUTE: 'DISTRIBUTE'; -DIV: 'DIV'; -DROP: 'DROP'; -ELSE: 'ELSE'; -END: 'END'; -ESCAPE: 'ESCAPE'; -ESCAPED: 'ESCAPED'; -EXCEPT: 'EXCEPT'; -EXCHANGE: 'EXCHANGE'; -EXISTS: 'EXISTS'; -EXPLAIN: 'EXPLAIN'; -EXPORT: 'EXPORT'; -EXTENDED: 'EXTENDED'; -EXTERNAL: 'EXTERNAL'; -EXTRACT: 'EXTRACT'; -FALSE: 'FALSE'; -FETCH: 'FETCH'; -FIELDS: 'FIELDS'; -FILTER: 'FILTER'; -FILEFORMAT: 'FILEFORMAT'; -FIRST: 'FIRST'; -FOLLOWING: 'FOLLOWING'; -FOR: 'FOR'; -FOREIGN: 'FOREIGN'; -FORMAT: 'FORMAT'; -FORMATTED: 'FORMATTED'; -FROM: 'FROM'; -FULL: 'FULL'; -FUNCTION: 'FUNCTION'; -FUNCTIONS: 'FUNCTIONS'; -GLOBAL: 'GLOBAL'; -GRANT: 'GRANT'; -GROUP: 'GROUP'; -GROUPING: 'GROUPING'; -HAVING: 'HAVING'; -HOUR: 'HOUR'; -IF: 'IF'; -IGNORE: 'IGNORE'; -IMPORT: 'IMPORT'; -IN: 'IN'; -INDEX: 'INDEX'; -INDEXES: 'INDEXES'; -INNER: 'INNER'; -INPATH: 'INPATH'; -INPUTFORMAT: 'INPUTFORMAT'; -INSERT: 'INSERT'; -INTERSECT: 'INTERSECT'; -INTERVAL: 'INTERVAL'; -INTO: 'INTO'; -IS: 'IS'; -ITEMS: 'ITEMS'; -JOIN: 'JOIN'; -KEYS: 'KEYS'; -LAST: 'LAST'; -LATERAL: 'LATERAL'; -LAZY: 'LAZY'; -LEADING: 'LEADING'; -LEFT: 'LEFT'; -LIKE: 'LIKE'; -LIMIT: 'LIMIT'; -LINES: 'LINES'; -LIST: 'LIST'; -LOAD: 'LOAD'; -LOCAL: 'LOCAL'; -LOCATION: 'LOCATION'; -LOCK: 'LOCK'; -LOCKS: 'LOCKS'; -LOGICAL: 'LOGICAL'; -MACRO: 'MACRO'; -MAP: 'MAP'; -MATCHED: 'MATCHED'; -MERGE: 'MERGE'; -MINUTE: 'MINUTE'; -MONTH: 'MONTH'; -MSCK: 'MSCK'; -NAMESPACE: 'NAMESPACE'; -NAMESPACES: 'NAMESPACES'; -NATURAL: 'NATURAL'; -NO: 'NO'; -NOT: 'NOT' | '!'; -NULL: 'NULL'; -NULLS: 'NULLS'; -OF: 'OF'; -ON: 'ON'; -ONLY: 'ONLY'; -OPTION: 'OPTION'; -OPTIONS: 'OPTIONS'; -OR: 'OR'; -ORDER: 'ORDER'; -OUT: 'OUT'; -OUTER: 'OUTER'; -OUTPUTFORMAT: 'OUTPUTFORMAT'; -OVER: 'OVER'; -OVERLAPS: 'OVERLAPS'; -OVERLAY: 'OVERLAY'; -OVERWRITE: 'OVERWRITE'; -PARTITION: 'PARTITION'; -PARTITIONED: 'PARTITIONED'; -PARTITIONS: 'PARTITIONS'; -PERCENTLIT: 'PERCENT'; -PIVOT: 'PIVOT'; -PLACING: 'PLACING'; -POSITION: 'POSITION'; -PRECEDING: 'PRECEDING'; -PRIMARY: 'PRIMARY'; -PRINCIPALS: 'PRINCIPALS'; -PROPERTIES: 'PROPERTIES'; -PURGE: 'PURGE'; -QUERY: 'QUERY'; -RANGE: 'RANGE'; -RECORDREADER: 'RECORDREADER'; -RECORDWRITER: 'RECORDWRITER'; -RECOVER: 'RECOVER'; -REDUCE: 'REDUCE'; -REFERENCES: 'REFERENCES'; -REFRESH: 'REFRESH'; -RENAME: 'RENAME'; -REPAIR: 'REPAIR'; -REPLACE: 'REPLACE'; -RESET: 'RESET'; -RESPECT: 'RESPECT'; -RESTRICT: 'RESTRICT'; -REVOKE: 'REVOKE'; -RIGHT: 'RIGHT'; -RLIKE: 'RLIKE' | 'REGEXP'; -ROLE: 'ROLE'; -ROLES: 'ROLES'; -ROLLBACK: 'ROLLBACK'; -ROLLUP: 'ROLLUP'; -ROW: 'ROW'; -ROWS: 'ROWS'; -SECOND: 'SECOND'; -SCHEMA: 'SCHEMA'; -SELECT: 'SELECT'; -SEMI: 'SEMI'; -SEPARATED: 'SEPARATED'; -SERDE: 'SERDE'; -SERDEPROPERTIES: 'SERDEPROPERTIES'; -SESSION_USER: 'SESSION_USER'; -SET: 'SET'; -SETMINUS: 'MINUS'; -SETS: 'SETS'; -SHOW: 'SHOW'; -SKEWED: 'SKEWED'; -SOME: 'SOME'; -SORT: 'SORT'; -SORTED: 'SORTED'; -START: 'START'; -STATISTICS: 'STATISTICS'; -STORED: 'STORED'; -STRATIFY: 'STRATIFY'; -STRUCT: 'STRUCT'; -SUBSTR: 'SUBSTR'; -SUBSTRING: 'SUBSTRING'; -SYNC: 'SYNC'; -TABLE: 'TABLE'; -TABLES: 'TABLES'; -TABLESAMPLE: 'TABLESAMPLE'; -TBLPROPERTIES: 'TBLPROPERTIES'; -TEMPORARY: 'TEMPORARY' | 'TEMP'; -TERMINATED: 'TERMINATED'; -THEN: 'THEN'; -TIME: 'TIME'; -TO: 'TO'; -TOUCH: 'TOUCH'; -TRAILING: 'TRAILING'; -TRANSACTION: 'TRANSACTION'; -TRANSACTIONS: 'TRANSACTIONS'; -TRANSFORM: 'TRANSFORM'; -TRIM: 'TRIM'; -TRUE: 'TRUE'; -TRUNCATE: 'TRUNCATE'; -TRY_CAST: 'TRY_CAST'; -TYPE: 'TYPE'; -UNARCHIVE: 'UNARCHIVE'; -UNBOUNDED: 'UNBOUNDED'; -UNCACHE: 'UNCACHE'; -UNION: 'UNION'; -UNIQUE: 'UNIQUE'; -UNKNOWN: 'UNKNOWN'; -UNLOCK: 'UNLOCK'; -UNSET: 'UNSET'; -UPDATE: 'UPDATE'; -USE: 'USE'; -USER: 'USER'; -USING: 'USING'; -VALUES: 'VALUES'; -VIEW: 'VIEW'; -VIEWS: 'VIEWS'; -WHEN: 'WHEN'; -WHERE: 'WHERE'; -WINDOW: 'WINDOW'; -WITH: 'WITH'; -YEAR: 'YEAR'; -ZONE: 'ZONE'; -KEY: 'KEY'; -//--SPARK-KEYWORD-LIST-END -//============================ -// End of the keywords list -//============================ - -EQ : '=' | '=='; -NSEQ: '<=>'; -NEQ : '<>'; -NEQJ: '!='; -LT : '<'; -LTE : '<=' | '!>'; -GT : '>'; -GTE : '>=' | '!<'; - -PLUS: '+'; -MINUS: '-'; -ASTERISK: '*'; -SLASH: '/'; -PERCENT: '%'; -TILDE: '~'; -AMPERSAND: '&'; -PIPE: '|'; -CONCAT_PIPE: '||'; -HAT: '^'; - -STRING - : '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' - | '"' ( ~('"'|'\\') | ('\\' .) )* '"' - ; - -BIGINT_LITERAL - : DIGIT+ 'L' - ; - -SMALLINT_LITERAL - : DIGIT+ 'S' - ; - -TINYINT_LITERAL - : DIGIT+ 'Y' - ; - -INTEGER_VALUE - : DIGIT+ - ; - -EXPONENT_VALUE - : DIGIT+ EXPONENT - | DECIMAL_DIGITS EXPONENT {isValidDecimal()}? - ; - -DECIMAL_VALUE - : DECIMAL_DIGITS {isValidDecimal()}? - ; - -FLOAT_LITERAL - : DIGIT+ EXPONENT? 'F' - | DECIMAL_DIGITS EXPONENT? 'F' {isValidDecimal()}? - ; - -DOUBLE_LITERAL - : DIGIT+ EXPONENT? 'D' - | DECIMAL_DIGITS EXPONENT? 'D' {isValidDecimal()}? - ; - -BIGDECIMAL_LITERAL - : DIGIT+ EXPONENT? 'BD' - | DECIMAL_DIGITS EXPONENT? 'BD' {isValidDecimal()}? - ; - -IDENTIFIER - : (LETTER | DIGIT | '_')+ - ; - -BACKQUOTED_IDENTIFIER - : '`' ( ~'`' | '``' )* '`' - ; - -fragment DECIMAL_DIGITS - : DIGIT+ '.' DIGIT* - | '.' DIGIT+ - ; - -fragment EXPONENT - : 'E' [+-]? DIGIT+ - ; - -fragment DIGIT - : [0-9] - ; - -fragment LETTER - : [A-Z] - ; - -SIMPLE_COMMENT - : '--' ('\\\n' | ~[\r\n])* '\r'? '\n'? -> channel(HIDDEN) - ; - -BRACKETED_COMMENT - : '/*' {!isHint()}? ( BRACKETED_COMMENT | . )*? ('*/' | {markUnclosedComment();} EOF) -> channel(HIDDEN) - ; - -WS - : [ \r\n\t]+ -> channel(HIDDEN) - ; - -// Catch-all for anything we can't recognize. -// We use this to be able to ignore and recover all the text -// when splitting statements with DelimiterLexer -UNRECOGNIZED - : . - ; diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/.gitignore b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/.gitignore deleted file mode 100644 index 0a3714d3c6..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -*.tokens -*.interp \ No newline at end of file diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendBaseListener.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendBaseListener.java deleted file mode 100644 index 8a782f9e69..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendBaseListener.java +++ /dev/null @@ -1,2115 +0,0 @@ -// Generated from org/apache/amoro/spark/sql/parser/MixedFormatSqlExtend.g4 by ANTLR 4.8 -package org.apache.amoro.spark.sql.parser; - -import org.antlr.v4.runtime.ParserRuleContext; -import org.antlr.v4.runtime.tree.ErrorNode; -import org.antlr.v4.runtime.tree.TerminalNode; - -/** - * This class provides an empty implementation of {@link MixedFormatSqlExtendListener}, - * which can be extended to create a listener which only needs to handle a subset - * of the available methods. - */ -public class MixedFormatSqlExtendBaseListener implements MixedFormatSqlExtendListener { - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterExtendStatement(MixedFormatSqlExtendParser.ExtendStatementContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitExtendStatement(MixedFormatSqlExtendParser.ExtendStatementContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterCreateTableWithPk(MixedFormatSqlExtendParser.CreateTableWithPkContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitCreateTableWithPk(MixedFormatSqlExtendParser.CreateTableWithPkContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterExplain(MixedFormatSqlExtendParser.ExplainContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitExplain(MixedFormatSqlExtendParser.ExplainContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterCreateTableHeader(MixedFormatSqlExtendParser.CreateTableHeaderContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitCreateTableHeader(MixedFormatSqlExtendParser.CreateTableHeaderContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterColListWithPk(MixedFormatSqlExtendParser.ColListWithPkContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitColListWithPk(MixedFormatSqlExtendParser.ColListWithPkContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterColListOnlyPk(MixedFormatSqlExtendParser.ColListOnlyPkContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitColListOnlyPk(MixedFormatSqlExtendParser.ColListOnlyPkContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPrimarySpec(MixedFormatSqlExtendParser.PrimarySpecContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPrimarySpec(MixedFormatSqlExtendParser.PrimarySpecContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterBucketSpec(MixedFormatSqlExtendParser.BucketSpecContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitBucketSpec(MixedFormatSqlExtendParser.BucketSpecContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSkewSpec(MixedFormatSqlExtendParser.SkewSpecContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSkewSpec(MixedFormatSqlExtendParser.SkewSpecContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterLocationSpec(MixedFormatSqlExtendParser.LocationSpecContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitLocationSpec(MixedFormatSqlExtendParser.LocationSpecContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterCommentSpec(MixedFormatSqlExtendParser.CommentSpecContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitCommentSpec(MixedFormatSqlExtendParser.CommentSpecContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterQuery(MixedFormatSqlExtendParser.QueryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitQuery(MixedFormatSqlExtendParser.QueryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterCtes(MixedFormatSqlExtendParser.CtesContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitCtes(MixedFormatSqlExtendParser.CtesContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterNamedQuery(MixedFormatSqlExtendParser.NamedQueryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitNamedQuery(MixedFormatSqlExtendParser.NamedQueryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTableProvider(MixedFormatSqlExtendParser.TableProviderContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTableProvider(MixedFormatSqlExtendParser.TableProviderContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterCreateTableClauses(MixedFormatSqlExtendParser.CreateTableClausesContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitCreateTableClauses(MixedFormatSqlExtendParser.CreateTableClausesContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTablePropertyList(MixedFormatSqlExtendParser.TablePropertyListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTablePropertyList(MixedFormatSqlExtendParser.TablePropertyListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTableProperty(MixedFormatSqlExtendParser.TablePropertyContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTableProperty(MixedFormatSqlExtendParser.TablePropertyContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTablePropertyKey(MixedFormatSqlExtendParser.TablePropertyKeyContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTablePropertyKey(MixedFormatSqlExtendParser.TablePropertyKeyContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTablePropertyValue(MixedFormatSqlExtendParser.TablePropertyValueContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTablePropertyValue(MixedFormatSqlExtendParser.TablePropertyValueContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterConstantList(MixedFormatSqlExtendParser.ConstantListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitConstantList(MixedFormatSqlExtendParser.ConstantListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterNestedConstantList(MixedFormatSqlExtendParser.NestedConstantListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitNestedConstantList(MixedFormatSqlExtendParser.NestedConstantListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterCreateFileFormat(MixedFormatSqlExtendParser.CreateFileFormatContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitCreateFileFormat(MixedFormatSqlExtendParser.CreateFileFormatContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTableFileFormat(MixedFormatSqlExtendParser.TableFileFormatContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTableFileFormat(MixedFormatSqlExtendParser.TableFileFormatContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterGenericFileFormat(MixedFormatSqlExtendParser.GenericFileFormatContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitGenericFileFormat(MixedFormatSqlExtendParser.GenericFileFormatContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterStorageHandler(MixedFormatSqlExtendParser.StorageHandlerContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitStorageHandler(MixedFormatSqlExtendParser.StorageHandlerContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterQueryOrganization(MixedFormatSqlExtendParser.QueryOrganizationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitQueryOrganization(MixedFormatSqlExtendParser.QueryOrganizationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterQueryTermDefault(MixedFormatSqlExtendParser.QueryTermDefaultContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitQueryTermDefault(MixedFormatSqlExtendParser.QueryTermDefaultContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSetOperation(MixedFormatSqlExtendParser.SetOperationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSetOperation(MixedFormatSqlExtendParser.SetOperationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterQueryPrimaryDefault(MixedFormatSqlExtendParser.QueryPrimaryDefaultContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitQueryPrimaryDefault(MixedFormatSqlExtendParser.QueryPrimaryDefaultContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterFromStmt(MixedFormatSqlExtendParser.FromStmtContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitFromStmt(MixedFormatSqlExtendParser.FromStmtContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTable(MixedFormatSqlExtendParser.TableContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTable(MixedFormatSqlExtendParser.TableContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterInlineTableDefault1(MixedFormatSqlExtendParser.InlineTableDefault1Context ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitInlineTableDefault1(MixedFormatSqlExtendParser.InlineTableDefault1Context ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSubquery(MixedFormatSqlExtendParser.SubqueryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSubquery(MixedFormatSqlExtendParser.SubqueryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSortItem(MixedFormatSqlExtendParser.SortItemContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSortItem(MixedFormatSqlExtendParser.SortItemContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterFromStatement(MixedFormatSqlExtendParser.FromStatementContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitFromStatement(MixedFormatSqlExtendParser.FromStatementContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterFromStatementBody(MixedFormatSqlExtendParser.FromStatementBodyContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitFromStatementBody(MixedFormatSqlExtendParser.FromStatementBodyContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTransformQuerySpecification(MixedFormatSqlExtendParser.TransformQuerySpecificationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTransformQuerySpecification(MixedFormatSqlExtendParser.TransformQuerySpecificationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterRegularQuerySpecification(MixedFormatSqlExtendParser.RegularQuerySpecificationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitRegularQuerySpecification(MixedFormatSqlExtendParser.RegularQuerySpecificationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTransformClause(MixedFormatSqlExtendParser.TransformClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTransformClause(MixedFormatSqlExtendParser.TransformClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSelectClause(MixedFormatSqlExtendParser.SelectClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSelectClause(MixedFormatSqlExtendParser.SelectClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterWhereClause(MixedFormatSqlExtendParser.WhereClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitWhereClause(MixedFormatSqlExtendParser.WhereClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterHavingClause(MixedFormatSqlExtendParser.HavingClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitHavingClause(MixedFormatSqlExtendParser.HavingClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterHint(MixedFormatSqlExtendParser.HintContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitHint(MixedFormatSqlExtendParser.HintContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterHintStatement(MixedFormatSqlExtendParser.HintStatementContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitHintStatement(MixedFormatSqlExtendParser.HintStatementContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterFromClause(MixedFormatSqlExtendParser.FromClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitFromClause(MixedFormatSqlExtendParser.FromClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterAggregationClause(MixedFormatSqlExtendParser.AggregationClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitAggregationClause(MixedFormatSqlExtendParser.AggregationClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterGroupByClause(MixedFormatSqlExtendParser.GroupByClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitGroupByClause(MixedFormatSqlExtendParser.GroupByClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterGroupingAnalytics(MixedFormatSqlExtendParser.GroupingAnalyticsContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitGroupingAnalytics(MixedFormatSqlExtendParser.GroupingAnalyticsContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterGroupingElement(MixedFormatSqlExtendParser.GroupingElementContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitGroupingElement(MixedFormatSqlExtendParser.GroupingElementContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterGroupingSet(MixedFormatSqlExtendParser.GroupingSetContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitGroupingSet(MixedFormatSqlExtendParser.GroupingSetContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPivotClause(MixedFormatSqlExtendParser.PivotClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPivotClause(MixedFormatSqlExtendParser.PivotClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPivotColumn(MixedFormatSqlExtendParser.PivotColumnContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPivotColumn(MixedFormatSqlExtendParser.PivotColumnContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPivotValue(MixedFormatSqlExtendParser.PivotValueContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPivotValue(MixedFormatSqlExtendParser.PivotValueContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterLateralView(MixedFormatSqlExtendParser.LateralViewContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitLateralView(MixedFormatSqlExtendParser.LateralViewContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSetQuantifier(MixedFormatSqlExtendParser.SetQuantifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSetQuantifier(MixedFormatSqlExtendParser.SetQuantifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterRelation(MixedFormatSqlExtendParser.RelationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitRelation(MixedFormatSqlExtendParser.RelationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterJoinRelation(MixedFormatSqlExtendParser.JoinRelationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitJoinRelation(MixedFormatSqlExtendParser.JoinRelationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterJoinType(MixedFormatSqlExtendParser.JoinTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitJoinType(MixedFormatSqlExtendParser.JoinTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterJoinCriteria(MixedFormatSqlExtendParser.JoinCriteriaContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitJoinCriteria(MixedFormatSqlExtendParser.JoinCriteriaContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSample(MixedFormatSqlExtendParser.SampleContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSample(MixedFormatSqlExtendParser.SampleContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSampleByPercentile(MixedFormatSqlExtendParser.SampleByPercentileContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSampleByPercentile(MixedFormatSqlExtendParser.SampleByPercentileContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSampleByRows(MixedFormatSqlExtendParser.SampleByRowsContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSampleByRows(MixedFormatSqlExtendParser.SampleByRowsContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSampleByBucket(MixedFormatSqlExtendParser.SampleByBucketContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSampleByBucket(MixedFormatSqlExtendParser.SampleByBucketContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSampleByBytes(MixedFormatSqlExtendParser.SampleByBytesContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSampleByBytes(MixedFormatSqlExtendParser.SampleByBytesContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterIdentifierList(MixedFormatSqlExtendParser.IdentifierListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitIdentifierList(MixedFormatSqlExtendParser.IdentifierListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterIdentifierSeq(MixedFormatSqlExtendParser.IdentifierSeqContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitIdentifierSeq(MixedFormatSqlExtendParser.IdentifierSeqContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterOrderedIdentifierList(MixedFormatSqlExtendParser.OrderedIdentifierListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitOrderedIdentifierList(MixedFormatSqlExtendParser.OrderedIdentifierListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterOrderedIdentifier(MixedFormatSqlExtendParser.OrderedIdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitOrderedIdentifier(MixedFormatSqlExtendParser.OrderedIdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTableName(MixedFormatSqlExtendParser.TableNameContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTableName(MixedFormatSqlExtendParser.TableNameContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterAliasedQuery(MixedFormatSqlExtendParser.AliasedQueryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitAliasedQuery(MixedFormatSqlExtendParser.AliasedQueryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterAliasedRelation(MixedFormatSqlExtendParser.AliasedRelationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitAliasedRelation(MixedFormatSqlExtendParser.AliasedRelationContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterInlineTableDefault2(MixedFormatSqlExtendParser.InlineTableDefault2Context ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitInlineTableDefault2(MixedFormatSqlExtendParser.InlineTableDefault2Context ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTableValuedFunction(MixedFormatSqlExtendParser.TableValuedFunctionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTableValuedFunction(MixedFormatSqlExtendParser.TableValuedFunctionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterInlineTable(MixedFormatSqlExtendParser.InlineTableContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitInlineTable(MixedFormatSqlExtendParser.InlineTableContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterFunctionTable(MixedFormatSqlExtendParser.FunctionTableContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitFunctionTable(MixedFormatSqlExtendParser.FunctionTableContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTableAlias(MixedFormatSqlExtendParser.TableAliasContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTableAlias(MixedFormatSqlExtendParser.TableAliasContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterRowFormatSerde(MixedFormatSqlExtendParser.RowFormatSerdeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitRowFormatSerde(MixedFormatSqlExtendParser.RowFormatSerdeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterRowFormatDelimited(MixedFormatSqlExtendParser.RowFormatDelimitedContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitRowFormatDelimited(MixedFormatSqlExtendParser.RowFormatDelimitedContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterMultipartIdentifier(MixedFormatSqlExtendParser.MultipartIdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitMultipartIdentifier(MixedFormatSqlExtendParser.MultipartIdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterNamedExpression(MixedFormatSqlExtendParser.NamedExpressionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitNamedExpression(MixedFormatSqlExtendParser.NamedExpressionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterNamedExpressionSeq(MixedFormatSqlExtendParser.NamedExpressionSeqContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitNamedExpressionSeq(MixedFormatSqlExtendParser.NamedExpressionSeqContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPartitionFieldList(MixedFormatSqlExtendParser.PartitionFieldListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPartitionFieldList(MixedFormatSqlExtendParser.PartitionFieldListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPartitionTransform(MixedFormatSqlExtendParser.PartitionTransformContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPartitionTransform(MixedFormatSqlExtendParser.PartitionTransformContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPartitionColumn(MixedFormatSqlExtendParser.PartitionColumnContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPartitionColumn(MixedFormatSqlExtendParser.PartitionColumnContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterIdentityTransform(MixedFormatSqlExtendParser.IdentityTransformContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitIdentityTransform(MixedFormatSqlExtendParser.IdentityTransformContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterApplyTransform(MixedFormatSqlExtendParser.ApplyTransformContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitApplyTransform(MixedFormatSqlExtendParser.ApplyTransformContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTransformArgument(MixedFormatSqlExtendParser.TransformArgumentContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTransformArgument(MixedFormatSqlExtendParser.TransformArgumentContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterExpression(MixedFormatSqlExtendParser.ExpressionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitExpression(MixedFormatSqlExtendParser.ExpressionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterExpressionSeq(MixedFormatSqlExtendParser.ExpressionSeqContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitExpressionSeq(MixedFormatSqlExtendParser.ExpressionSeqContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterLogicalNot(MixedFormatSqlExtendParser.LogicalNotContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitLogicalNot(MixedFormatSqlExtendParser.LogicalNotContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPredicated(MixedFormatSqlExtendParser.PredicatedContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPredicated(MixedFormatSqlExtendParser.PredicatedContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterExists(MixedFormatSqlExtendParser.ExistsContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitExists(MixedFormatSqlExtendParser.ExistsContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterLogicalBinary(MixedFormatSqlExtendParser.LogicalBinaryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitLogicalBinary(MixedFormatSqlExtendParser.LogicalBinaryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPredicate(MixedFormatSqlExtendParser.PredicateContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPredicate(MixedFormatSqlExtendParser.PredicateContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterValueExpressionDefault(MixedFormatSqlExtendParser.ValueExpressionDefaultContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitValueExpressionDefault(MixedFormatSqlExtendParser.ValueExpressionDefaultContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterComparison(MixedFormatSqlExtendParser.ComparisonContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitComparison(MixedFormatSqlExtendParser.ComparisonContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterArithmeticBinary(MixedFormatSqlExtendParser.ArithmeticBinaryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitArithmeticBinary(MixedFormatSqlExtendParser.ArithmeticBinaryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterArithmeticUnary(MixedFormatSqlExtendParser.ArithmeticUnaryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitArithmeticUnary(MixedFormatSqlExtendParser.ArithmeticUnaryContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterStruct(MixedFormatSqlExtendParser.StructContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitStruct(MixedFormatSqlExtendParser.StructContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterDereference(MixedFormatSqlExtendParser.DereferenceContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitDereference(MixedFormatSqlExtendParser.DereferenceContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSimpleCase(MixedFormatSqlExtendParser.SimpleCaseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSimpleCase(MixedFormatSqlExtendParser.SimpleCaseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterCurrentLike(MixedFormatSqlExtendParser.CurrentLikeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitCurrentLike(MixedFormatSqlExtendParser.CurrentLikeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterColumnReference(MixedFormatSqlExtendParser.ColumnReferenceContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitColumnReference(MixedFormatSqlExtendParser.ColumnReferenceContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterRowConstructor(MixedFormatSqlExtendParser.RowConstructorContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitRowConstructor(MixedFormatSqlExtendParser.RowConstructorContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterLast(MixedFormatSqlExtendParser.LastContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitLast(MixedFormatSqlExtendParser.LastContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterStar(MixedFormatSqlExtendParser.StarContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitStar(MixedFormatSqlExtendParser.StarContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterOverlay(MixedFormatSqlExtendParser.OverlayContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitOverlay(MixedFormatSqlExtendParser.OverlayContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSubscript(MixedFormatSqlExtendParser.SubscriptContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSubscript(MixedFormatSqlExtendParser.SubscriptContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSubqueryExpression(MixedFormatSqlExtendParser.SubqueryExpressionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSubqueryExpression(MixedFormatSqlExtendParser.SubqueryExpressionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSubstring(MixedFormatSqlExtendParser.SubstringContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSubstring(MixedFormatSqlExtendParser.SubstringContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterCast(MixedFormatSqlExtendParser.CastContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitCast(MixedFormatSqlExtendParser.CastContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterConstantDefault(MixedFormatSqlExtendParser.ConstantDefaultContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitConstantDefault(MixedFormatSqlExtendParser.ConstantDefaultContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterLambda(MixedFormatSqlExtendParser.LambdaContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitLambda(MixedFormatSqlExtendParser.LambdaContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterParenthesizedExpression(MixedFormatSqlExtendParser.ParenthesizedExpressionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitParenthesizedExpression(MixedFormatSqlExtendParser.ParenthesizedExpressionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterExtract(MixedFormatSqlExtendParser.ExtractContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitExtract(MixedFormatSqlExtendParser.ExtractContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTrim(MixedFormatSqlExtendParser.TrimContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTrim(MixedFormatSqlExtendParser.TrimContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterFunctionCall(MixedFormatSqlExtendParser.FunctionCallContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitFunctionCall(MixedFormatSqlExtendParser.FunctionCallContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSearchedCase(MixedFormatSqlExtendParser.SearchedCaseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSearchedCase(MixedFormatSqlExtendParser.SearchedCaseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPosition(MixedFormatSqlExtendParser.PositionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPosition(MixedFormatSqlExtendParser.PositionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterFirst(MixedFormatSqlExtendParser.FirstContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitFirst(MixedFormatSqlExtendParser.FirstContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterNullLiteral(MixedFormatSqlExtendParser.NullLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitNullLiteral(MixedFormatSqlExtendParser.NullLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterIntervalLiteral(MixedFormatSqlExtendParser.IntervalLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitIntervalLiteral(MixedFormatSqlExtendParser.IntervalLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTypeConstructor(MixedFormatSqlExtendParser.TypeConstructorContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTypeConstructor(MixedFormatSqlExtendParser.TypeConstructorContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterNumericLiteral(MixedFormatSqlExtendParser.NumericLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitNumericLiteral(MixedFormatSqlExtendParser.NumericLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterBooleanLiteral(MixedFormatSqlExtendParser.BooleanLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitBooleanLiteral(MixedFormatSqlExtendParser.BooleanLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterStringLiteral(MixedFormatSqlExtendParser.StringLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitStringLiteral(MixedFormatSqlExtendParser.StringLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterComparisonOperator(MixedFormatSqlExtendParser.ComparisonOperatorContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitComparisonOperator(MixedFormatSqlExtendParser.ComparisonOperatorContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterBooleanValue(MixedFormatSqlExtendParser.BooleanValueContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitBooleanValue(MixedFormatSqlExtendParser.BooleanValueContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterInterval(MixedFormatSqlExtendParser.IntervalContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitInterval(MixedFormatSqlExtendParser.IntervalContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterErrorCapturingMultiUnitsInterval(MixedFormatSqlExtendParser.ErrorCapturingMultiUnitsIntervalContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitErrorCapturingMultiUnitsInterval(MixedFormatSqlExtendParser.ErrorCapturingMultiUnitsIntervalContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterMultiUnitsInterval(MixedFormatSqlExtendParser.MultiUnitsIntervalContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitMultiUnitsInterval(MixedFormatSqlExtendParser.MultiUnitsIntervalContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterErrorCapturingUnitToUnitInterval(MixedFormatSqlExtendParser.ErrorCapturingUnitToUnitIntervalContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitErrorCapturingUnitToUnitInterval(MixedFormatSqlExtendParser.ErrorCapturingUnitToUnitIntervalContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterUnitToUnitInterval(MixedFormatSqlExtendParser.UnitToUnitIntervalContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitUnitToUnitInterval(MixedFormatSqlExtendParser.UnitToUnitIntervalContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterIntervalValue(MixedFormatSqlExtendParser.IntervalValueContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitIntervalValue(MixedFormatSqlExtendParser.IntervalValueContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterColPosition(MixedFormatSqlExtendParser.ColPositionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitColPosition(MixedFormatSqlExtendParser.ColPositionContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterComplexDataType(MixedFormatSqlExtendParser.ComplexDataTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitComplexDataType(MixedFormatSqlExtendParser.ComplexDataTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterYearMonthIntervalDataType(MixedFormatSqlExtendParser.YearMonthIntervalDataTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitYearMonthIntervalDataType(MixedFormatSqlExtendParser.YearMonthIntervalDataTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterDayTimeIntervalDataType(MixedFormatSqlExtendParser.DayTimeIntervalDataTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitDayTimeIntervalDataType(MixedFormatSqlExtendParser.DayTimeIntervalDataTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterPrimitiveDataType(MixedFormatSqlExtendParser.PrimitiveDataTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitPrimitiveDataType(MixedFormatSqlExtendParser.PrimitiveDataTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterColTypeList(MixedFormatSqlExtendParser.ColTypeListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitColTypeList(MixedFormatSqlExtendParser.ColTypeListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterColType(MixedFormatSqlExtendParser.ColTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitColType(MixedFormatSqlExtendParser.ColTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterComplexColTypeList(MixedFormatSqlExtendParser.ComplexColTypeListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitComplexColTypeList(MixedFormatSqlExtendParser.ComplexColTypeListContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterComplexColType(MixedFormatSqlExtendParser.ComplexColTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitComplexColType(MixedFormatSqlExtendParser.ComplexColTypeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterWhenClause(MixedFormatSqlExtendParser.WhenClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitWhenClause(MixedFormatSqlExtendParser.WhenClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterWindowClause(MixedFormatSqlExtendParser.WindowClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitWindowClause(MixedFormatSqlExtendParser.WindowClauseContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterNamedWindow(MixedFormatSqlExtendParser.NamedWindowContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitNamedWindow(MixedFormatSqlExtendParser.NamedWindowContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterWindowRef(MixedFormatSqlExtendParser.WindowRefContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitWindowRef(MixedFormatSqlExtendParser.WindowRefContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterWindowDef(MixedFormatSqlExtendParser.WindowDefContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitWindowDef(MixedFormatSqlExtendParser.WindowDefContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterWindowFrame(MixedFormatSqlExtendParser.WindowFrameContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitWindowFrame(MixedFormatSqlExtendParser.WindowFrameContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterFrameBound(MixedFormatSqlExtendParser.FrameBoundContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitFrameBound(MixedFormatSqlExtendParser.FrameBoundContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterFunctionName(MixedFormatSqlExtendParser.FunctionNameContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitFunctionName(MixedFormatSqlExtendParser.FunctionNameContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterQualifiedName(MixedFormatSqlExtendParser.QualifiedNameContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitQualifiedName(MixedFormatSqlExtendParser.QualifiedNameContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterErrorCapturingIdentifier(MixedFormatSqlExtendParser.ErrorCapturingIdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitErrorCapturingIdentifier(MixedFormatSqlExtendParser.ErrorCapturingIdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterErrorIdent(MixedFormatSqlExtendParser.ErrorIdentContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitErrorIdent(MixedFormatSqlExtendParser.ErrorIdentContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterRealIdent(MixedFormatSqlExtendParser.RealIdentContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitRealIdent(MixedFormatSqlExtendParser.RealIdentContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterIdentifier(MixedFormatSqlExtendParser.IdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitIdentifier(MixedFormatSqlExtendParser.IdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterUnquotedIdentifier(MixedFormatSqlExtendParser.UnquotedIdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitUnquotedIdentifier(MixedFormatSqlExtendParser.UnquotedIdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterQuotedIdentifierAlternative(MixedFormatSqlExtendParser.QuotedIdentifierAlternativeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitQuotedIdentifierAlternative(MixedFormatSqlExtendParser.QuotedIdentifierAlternativeContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterQuotedIdentifier(MixedFormatSqlExtendParser.QuotedIdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitQuotedIdentifier(MixedFormatSqlExtendParser.QuotedIdentifierContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterExponentLiteral(MixedFormatSqlExtendParser.ExponentLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitExponentLiteral(MixedFormatSqlExtendParser.ExponentLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterDecimalLiteral(MixedFormatSqlExtendParser.DecimalLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitDecimalLiteral(MixedFormatSqlExtendParser.DecimalLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterLegacyDecimalLiteral(MixedFormatSqlExtendParser.LegacyDecimalLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitLegacyDecimalLiteral(MixedFormatSqlExtendParser.LegacyDecimalLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterIntegerLiteral(MixedFormatSqlExtendParser.IntegerLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitIntegerLiteral(MixedFormatSqlExtendParser.IntegerLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterBigIntLiteral(MixedFormatSqlExtendParser.BigIntLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitBigIntLiteral(MixedFormatSqlExtendParser.BigIntLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterSmallIntLiteral(MixedFormatSqlExtendParser.SmallIntLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitSmallIntLiteral(MixedFormatSqlExtendParser.SmallIntLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterTinyIntLiteral(MixedFormatSqlExtendParser.TinyIntLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitTinyIntLiteral(MixedFormatSqlExtendParser.TinyIntLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterDoubleLiteral(MixedFormatSqlExtendParser.DoubleLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitDoubleLiteral(MixedFormatSqlExtendParser.DoubleLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterFloatLiteral(MixedFormatSqlExtendParser.FloatLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitFloatLiteral(MixedFormatSqlExtendParser.FloatLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterBigDecimalLiteral(MixedFormatSqlExtendParser.BigDecimalLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitBigDecimalLiteral(MixedFormatSqlExtendParser.BigDecimalLiteralContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterAnsiNonReserved(MixedFormatSqlExtendParser.AnsiNonReservedContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitAnsiNonReserved(MixedFormatSqlExtendParser.AnsiNonReservedContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterStrictNonReserved(MixedFormatSqlExtendParser.StrictNonReservedContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitStrictNonReserved(MixedFormatSqlExtendParser.StrictNonReservedContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterNonReserved(MixedFormatSqlExtendParser.NonReservedContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitNonReserved(MixedFormatSqlExtendParser.NonReservedContext ctx) { } - - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void enterEveryRule(ParserRuleContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void exitEveryRule(ParserRuleContext ctx) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void visitTerminal(TerminalNode node) { } - /** - * {@inheritDoc} - * - *

The default implementation does nothing.

- */ - @Override public void visitErrorNode(ErrorNode node) { } -} \ No newline at end of file diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendBaseVisitor.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendBaseVisitor.java deleted file mode 100644 index 37ab9a9abb..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendBaseVisitor.java +++ /dev/null @@ -1,1225 +0,0 @@ -// Generated from org/apache/amoro/spark/sql/parser/MixedFormatSqlExtend.g4 by ANTLR 4.8 -package org.apache.amoro.spark.sql.parser; -import org.antlr.v4.runtime.tree.AbstractParseTreeVisitor; - -/** - * This class provides an empty implementation of {@link MixedFormatSqlExtendVisitor}, - * which can be extended to create a visitor which only needs to handle a subset - * of the available methods. - * - * @param The return type of the visit operation. Use {@link Void} for - * operations with no return type. - */ -public class MixedFormatSqlExtendBaseVisitor extends AbstractParseTreeVisitor implements MixedFormatSqlExtendVisitor { - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitExtendStatement(MixedFormatSqlExtendParser.ExtendStatementContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitCreateTableWithPk(MixedFormatSqlExtendParser.CreateTableWithPkContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitExplain(MixedFormatSqlExtendParser.ExplainContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitCreateTableHeader(MixedFormatSqlExtendParser.CreateTableHeaderContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitColListWithPk(MixedFormatSqlExtendParser.ColListWithPkContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitColListOnlyPk(MixedFormatSqlExtendParser.ColListOnlyPkContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPrimarySpec(MixedFormatSqlExtendParser.PrimarySpecContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitBucketSpec(MixedFormatSqlExtendParser.BucketSpecContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSkewSpec(MixedFormatSqlExtendParser.SkewSpecContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitLocationSpec(MixedFormatSqlExtendParser.LocationSpecContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitCommentSpec(MixedFormatSqlExtendParser.CommentSpecContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitQuery(MixedFormatSqlExtendParser.QueryContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitCtes(MixedFormatSqlExtendParser.CtesContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitNamedQuery(MixedFormatSqlExtendParser.NamedQueryContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTableProvider(MixedFormatSqlExtendParser.TableProviderContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitCreateTableClauses(MixedFormatSqlExtendParser.CreateTableClausesContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTablePropertyList(MixedFormatSqlExtendParser.TablePropertyListContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTableProperty(MixedFormatSqlExtendParser.TablePropertyContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTablePropertyKey(MixedFormatSqlExtendParser.TablePropertyKeyContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTablePropertyValue(MixedFormatSqlExtendParser.TablePropertyValueContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitConstantList(MixedFormatSqlExtendParser.ConstantListContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitNestedConstantList(MixedFormatSqlExtendParser.NestedConstantListContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitCreateFileFormat(MixedFormatSqlExtendParser.CreateFileFormatContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTableFileFormat(MixedFormatSqlExtendParser.TableFileFormatContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitGenericFileFormat(MixedFormatSqlExtendParser.GenericFileFormatContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitStorageHandler(MixedFormatSqlExtendParser.StorageHandlerContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitQueryOrganization(MixedFormatSqlExtendParser.QueryOrganizationContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitQueryTermDefault(MixedFormatSqlExtendParser.QueryTermDefaultContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSetOperation(MixedFormatSqlExtendParser.SetOperationContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitQueryPrimaryDefault(MixedFormatSqlExtendParser.QueryPrimaryDefaultContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitFromStmt(MixedFormatSqlExtendParser.FromStmtContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTable(MixedFormatSqlExtendParser.TableContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitInlineTableDefault1(MixedFormatSqlExtendParser.InlineTableDefault1Context ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSubquery(MixedFormatSqlExtendParser.SubqueryContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSortItem(MixedFormatSqlExtendParser.SortItemContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitFromStatement(MixedFormatSqlExtendParser.FromStatementContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitFromStatementBody(MixedFormatSqlExtendParser.FromStatementBodyContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTransformQuerySpecification(MixedFormatSqlExtendParser.TransformQuerySpecificationContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitRegularQuerySpecification(MixedFormatSqlExtendParser.RegularQuerySpecificationContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTransformClause(MixedFormatSqlExtendParser.TransformClauseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSelectClause(MixedFormatSqlExtendParser.SelectClauseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitWhereClause(MixedFormatSqlExtendParser.WhereClauseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitHavingClause(MixedFormatSqlExtendParser.HavingClauseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitHint(MixedFormatSqlExtendParser.HintContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitHintStatement(MixedFormatSqlExtendParser.HintStatementContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitFromClause(MixedFormatSqlExtendParser.FromClauseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitAggregationClause(MixedFormatSqlExtendParser.AggregationClauseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitGroupByClause(MixedFormatSqlExtendParser.GroupByClauseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitGroupingAnalytics(MixedFormatSqlExtendParser.GroupingAnalyticsContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitGroupingElement(MixedFormatSqlExtendParser.GroupingElementContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitGroupingSet(MixedFormatSqlExtendParser.GroupingSetContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPivotClause(MixedFormatSqlExtendParser.PivotClauseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPivotColumn(MixedFormatSqlExtendParser.PivotColumnContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPivotValue(MixedFormatSqlExtendParser.PivotValueContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitLateralView(MixedFormatSqlExtendParser.LateralViewContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSetQuantifier(MixedFormatSqlExtendParser.SetQuantifierContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitRelation(MixedFormatSqlExtendParser.RelationContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitJoinRelation(MixedFormatSqlExtendParser.JoinRelationContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitJoinType(MixedFormatSqlExtendParser.JoinTypeContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitJoinCriteria(MixedFormatSqlExtendParser.JoinCriteriaContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSample(MixedFormatSqlExtendParser.SampleContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSampleByPercentile(MixedFormatSqlExtendParser.SampleByPercentileContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSampleByRows(MixedFormatSqlExtendParser.SampleByRowsContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSampleByBucket(MixedFormatSqlExtendParser.SampleByBucketContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSampleByBytes(MixedFormatSqlExtendParser.SampleByBytesContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitIdentifierList(MixedFormatSqlExtendParser.IdentifierListContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitIdentifierSeq(MixedFormatSqlExtendParser.IdentifierSeqContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitOrderedIdentifierList(MixedFormatSqlExtendParser.OrderedIdentifierListContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitOrderedIdentifier(MixedFormatSqlExtendParser.OrderedIdentifierContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTableName(MixedFormatSqlExtendParser.TableNameContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitAliasedQuery(MixedFormatSqlExtendParser.AliasedQueryContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitAliasedRelation(MixedFormatSqlExtendParser.AliasedRelationContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitInlineTableDefault2(MixedFormatSqlExtendParser.InlineTableDefault2Context ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTableValuedFunction(MixedFormatSqlExtendParser.TableValuedFunctionContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitInlineTable(MixedFormatSqlExtendParser.InlineTableContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitFunctionTable(MixedFormatSqlExtendParser.FunctionTableContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTableAlias(MixedFormatSqlExtendParser.TableAliasContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitRowFormatSerde(MixedFormatSqlExtendParser.RowFormatSerdeContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitRowFormatDelimited(MixedFormatSqlExtendParser.RowFormatDelimitedContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitMultipartIdentifier(MixedFormatSqlExtendParser.MultipartIdentifierContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitNamedExpression(MixedFormatSqlExtendParser.NamedExpressionContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitNamedExpressionSeq(MixedFormatSqlExtendParser.NamedExpressionSeqContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPartitionFieldList(MixedFormatSqlExtendParser.PartitionFieldListContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPartitionTransform(MixedFormatSqlExtendParser.PartitionTransformContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPartitionColumn(MixedFormatSqlExtendParser.PartitionColumnContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitIdentityTransform(MixedFormatSqlExtendParser.IdentityTransformContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitApplyTransform(MixedFormatSqlExtendParser.ApplyTransformContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTransformArgument(MixedFormatSqlExtendParser.TransformArgumentContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitExpression(MixedFormatSqlExtendParser.ExpressionContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitExpressionSeq(MixedFormatSqlExtendParser.ExpressionSeqContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitLogicalNot(MixedFormatSqlExtendParser.LogicalNotContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPredicated(MixedFormatSqlExtendParser.PredicatedContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitExists(MixedFormatSqlExtendParser.ExistsContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitLogicalBinary(MixedFormatSqlExtendParser.LogicalBinaryContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPredicate(MixedFormatSqlExtendParser.PredicateContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitValueExpressionDefault(MixedFormatSqlExtendParser.ValueExpressionDefaultContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitComparison(MixedFormatSqlExtendParser.ComparisonContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitArithmeticBinary(MixedFormatSqlExtendParser.ArithmeticBinaryContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitArithmeticUnary(MixedFormatSqlExtendParser.ArithmeticUnaryContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitStruct(MixedFormatSqlExtendParser.StructContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitDereference(MixedFormatSqlExtendParser.DereferenceContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSimpleCase(MixedFormatSqlExtendParser.SimpleCaseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitCurrentLike(MixedFormatSqlExtendParser.CurrentLikeContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitColumnReference(MixedFormatSqlExtendParser.ColumnReferenceContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitRowConstructor(MixedFormatSqlExtendParser.RowConstructorContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitLast(MixedFormatSqlExtendParser.LastContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitStar(MixedFormatSqlExtendParser.StarContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitOverlay(MixedFormatSqlExtendParser.OverlayContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSubscript(MixedFormatSqlExtendParser.SubscriptContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSubqueryExpression(MixedFormatSqlExtendParser.SubqueryExpressionContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSubstring(MixedFormatSqlExtendParser.SubstringContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitCast(MixedFormatSqlExtendParser.CastContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitConstantDefault(MixedFormatSqlExtendParser.ConstantDefaultContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitLambda(MixedFormatSqlExtendParser.LambdaContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitParenthesizedExpression(MixedFormatSqlExtendParser.ParenthesizedExpressionContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitExtract(MixedFormatSqlExtendParser.ExtractContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTrim(MixedFormatSqlExtendParser.TrimContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitFunctionCall(MixedFormatSqlExtendParser.FunctionCallContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSearchedCase(MixedFormatSqlExtendParser.SearchedCaseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPosition(MixedFormatSqlExtendParser.PositionContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitFirst(MixedFormatSqlExtendParser.FirstContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitNullLiteral(MixedFormatSqlExtendParser.NullLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitIntervalLiteral(MixedFormatSqlExtendParser.IntervalLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTypeConstructor(MixedFormatSqlExtendParser.TypeConstructorContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitNumericLiteral(MixedFormatSqlExtendParser.NumericLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitBooleanLiteral(MixedFormatSqlExtendParser.BooleanLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitStringLiteral(MixedFormatSqlExtendParser.StringLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitComparisonOperator(MixedFormatSqlExtendParser.ComparisonOperatorContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitBooleanValue(MixedFormatSqlExtendParser.BooleanValueContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitInterval(MixedFormatSqlExtendParser.IntervalContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitErrorCapturingMultiUnitsInterval(MixedFormatSqlExtendParser.ErrorCapturingMultiUnitsIntervalContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitMultiUnitsInterval(MixedFormatSqlExtendParser.MultiUnitsIntervalContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitErrorCapturingUnitToUnitInterval(MixedFormatSqlExtendParser.ErrorCapturingUnitToUnitIntervalContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitUnitToUnitInterval(MixedFormatSqlExtendParser.UnitToUnitIntervalContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitIntervalValue(MixedFormatSqlExtendParser.IntervalValueContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitColPosition(MixedFormatSqlExtendParser.ColPositionContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitComplexDataType(MixedFormatSqlExtendParser.ComplexDataTypeContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitYearMonthIntervalDataType(MixedFormatSqlExtendParser.YearMonthIntervalDataTypeContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitDayTimeIntervalDataType(MixedFormatSqlExtendParser.DayTimeIntervalDataTypeContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitPrimitiveDataType(MixedFormatSqlExtendParser.PrimitiveDataTypeContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitColTypeList(MixedFormatSqlExtendParser.ColTypeListContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitColType(MixedFormatSqlExtendParser.ColTypeContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitComplexColTypeList(MixedFormatSqlExtendParser.ComplexColTypeListContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitComplexColType(MixedFormatSqlExtendParser.ComplexColTypeContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitWhenClause(MixedFormatSqlExtendParser.WhenClauseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitWindowClause(MixedFormatSqlExtendParser.WindowClauseContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitNamedWindow(MixedFormatSqlExtendParser.NamedWindowContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitWindowRef(MixedFormatSqlExtendParser.WindowRefContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitWindowDef(MixedFormatSqlExtendParser.WindowDefContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitWindowFrame(MixedFormatSqlExtendParser.WindowFrameContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitFrameBound(MixedFormatSqlExtendParser.FrameBoundContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitFunctionName(MixedFormatSqlExtendParser.FunctionNameContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitQualifiedName(MixedFormatSqlExtendParser.QualifiedNameContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitErrorCapturingIdentifier(MixedFormatSqlExtendParser.ErrorCapturingIdentifierContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitErrorIdent(MixedFormatSqlExtendParser.ErrorIdentContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitRealIdent(MixedFormatSqlExtendParser.RealIdentContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitIdentifier(MixedFormatSqlExtendParser.IdentifierContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitUnquotedIdentifier(MixedFormatSqlExtendParser.UnquotedIdentifierContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitQuotedIdentifierAlternative(MixedFormatSqlExtendParser.QuotedIdentifierAlternativeContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitQuotedIdentifier(MixedFormatSqlExtendParser.QuotedIdentifierContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitExponentLiteral(MixedFormatSqlExtendParser.ExponentLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitDecimalLiteral(MixedFormatSqlExtendParser.DecimalLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitLegacyDecimalLiteral(MixedFormatSqlExtendParser.LegacyDecimalLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitIntegerLiteral(MixedFormatSqlExtendParser.IntegerLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitBigIntLiteral(MixedFormatSqlExtendParser.BigIntLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitSmallIntLiteral(MixedFormatSqlExtendParser.SmallIntLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitTinyIntLiteral(MixedFormatSqlExtendParser.TinyIntLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitDoubleLiteral(MixedFormatSqlExtendParser.DoubleLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitFloatLiteral(MixedFormatSqlExtendParser.FloatLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitBigDecimalLiteral(MixedFormatSqlExtendParser.BigDecimalLiteralContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitAnsiNonReserved(MixedFormatSqlExtendParser.AnsiNonReservedContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitStrictNonReserved(MixedFormatSqlExtendParser.StrictNonReservedContext ctx) { return visitChildren(ctx); } - /** - * {@inheritDoc} - * - *

The default implementation returns the result of calling - * {@link #visitChildren} on {@code ctx}.

- */ - @Override public T visitNonReserved(MixedFormatSqlExtendParser.NonReservedContext ctx) { return visitChildren(ctx); } -} \ No newline at end of file diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendLexer.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendLexer.java deleted file mode 100644 index 61f24cee19..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendLexer.java +++ /dev/null @@ -1,1488 +0,0 @@ -// Generated from org/apache/amoro/spark/sql/parser/MixedFormatSqlExtend.g4 by ANTLR 4.8 -package org.apache.amoro.spark.sql.parser; -import org.antlr.v4.runtime.Lexer; -import org.antlr.v4.runtime.CharStream; -import org.antlr.v4.runtime.Token; -import org.antlr.v4.runtime.TokenStream; -import org.antlr.v4.runtime.*; -import org.antlr.v4.runtime.atn.*; -import org.antlr.v4.runtime.dfa.DFA; -import org.antlr.v4.runtime.misc.*; - -@SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"}) -public class MixedFormatSqlExtendLexer extends Lexer { - static { RuntimeMetaData.checkVersion("4.8", RuntimeMetaData.VERSION); } - - protected static final DFA[] _decisionToDFA; - protected static final PredictionContextCache _sharedContextCache = - new PredictionContextCache(); - public static final int - T__0=1, T__1=2, T__2=3, T__3=4, T__4=5, T__5=6, T__6=7, T__7=8, T__8=9, - T__9=10, T__10=11, ADD=12, AFTER=13, ALL=14, ALTER=15, ANALYZE=16, AND=17, - ANTI=18, ANY=19, ARCHIVE=20, ARRAY=21, AS=22, ASC=23, AT=24, AUTHORIZATION=25, - BETWEEN=26, BOTH=27, BUCKET=28, BUCKETS=29, BY=30, CACHE=31, CASCADE=32, - CASE=33, CAST=34, CHANGE=35, CHECK=36, CLEAR=37, CLUSTER=38, CLUSTERED=39, - CODEGEN=40, COLLATE=41, COLLECTION=42, COLUMN=43, COLUMNS=44, COMMENT=45, - COMMIT=46, COMPACT=47, COMPACTIONS=48, COMPUTE=49, CONCATENATE=50, CONSTRAINT=51, - COST=52, CREATE=53, CROSS=54, CUBE=55, CURRENT=56, CURRENT_DATE=57, CURRENT_TIME=58, - CURRENT_TIMESTAMP=59, CURRENT_USER=60, DAY=61, DATA=62, DATABASE=63, DATABASES=64, - DBPROPERTIES=65, DEFINED=66, DELETE=67, DELIMITED=68, DESC=69, DESCRIBE=70, - DFS=71, DIRECTORIES=72, DIRECTORY=73, DISTINCT=74, DISTRIBUTE=75, DIV=76, - DROP=77, ELSE=78, END=79, ESCAPE=80, ESCAPED=81, EXCEPT=82, EXCHANGE=83, - EXISTS=84, EXPLAIN=85, EXPORT=86, EXTENDED=87, EXTERNAL=88, EXTRACT=89, - FALSE=90, FETCH=91, FIELDS=92, FILTER=93, FILEFORMAT=94, FIRST=95, FOLLOWING=96, - FOR=97, FOREIGN=98, FORMAT=99, FORMATTED=100, FROM=101, FULL=102, FUNCTION=103, - FUNCTIONS=104, GLOBAL=105, GRANT=106, GROUP=107, GROUPING=108, HAVING=109, - HOUR=110, IF=111, IGNORE=112, IMPORT=113, IN=114, INDEX=115, INDEXES=116, - INNER=117, INPATH=118, INPUTFORMAT=119, INSERT=120, INTERSECT=121, INTERVAL=122, - INTO=123, IS=124, ITEMS=125, JOIN=126, KEYS=127, LAST=128, LATERAL=129, - LAZY=130, LEADING=131, LEFT=132, LIKE=133, LIMIT=134, LINES=135, LIST=136, - LOAD=137, LOCAL=138, LOCATION=139, LOCK=140, LOCKS=141, LOGICAL=142, MACRO=143, - MAP=144, MATCHED=145, MERGE=146, MINUTE=147, MONTH=148, MSCK=149, NAMESPACE=150, - NAMESPACES=151, NATURAL=152, NO=153, NOT=154, NULL=155, NULLS=156, OF=157, - ON=158, ONLY=159, OPTION=160, OPTIONS=161, OR=162, ORDER=163, OUT=164, - OUTER=165, OUTPUTFORMAT=166, OVER=167, OVERLAPS=168, OVERLAY=169, OVERWRITE=170, - PARTITION=171, PARTITIONED=172, PARTITIONS=173, PERCENTLIT=174, PIVOT=175, - PLACING=176, POSITION=177, PRECEDING=178, PRIMARY=179, PRINCIPALS=180, - PROPERTIES=181, PURGE=182, QUERY=183, RANGE=184, RECORDREADER=185, RECORDWRITER=186, - RECOVER=187, REDUCE=188, REFERENCES=189, REFRESH=190, RENAME=191, REPAIR=192, - REPLACE=193, RESET=194, RESPECT=195, RESTRICT=196, REVOKE=197, RIGHT=198, - RLIKE=199, ROLE=200, ROLES=201, ROLLBACK=202, ROLLUP=203, ROW=204, ROWS=205, - SECOND=206, SCHEMA=207, SELECT=208, SEMI=209, SEPARATED=210, SERDE=211, - SERDEPROPERTIES=212, SESSION_USER=213, SET=214, SETMINUS=215, SETS=216, - SHOW=217, SKEWED=218, SOME=219, SORT=220, SORTED=221, START=222, STATISTICS=223, - STORED=224, STRATIFY=225, STRUCT=226, SUBSTR=227, SUBSTRING=228, SYNC=229, - TABLE=230, TABLES=231, TABLESAMPLE=232, TBLPROPERTIES=233, TEMPORARY=234, - TERMINATED=235, THEN=236, TIME=237, TO=238, TOUCH=239, TRAILING=240, TRANSACTION=241, - TRANSACTIONS=242, TRANSFORM=243, TRIM=244, TRUE=245, TRUNCATE=246, TRY_CAST=247, - TYPE=248, UNARCHIVE=249, UNBOUNDED=250, UNCACHE=251, UNION=252, UNIQUE=253, - UNKNOWN=254, UNLOCK=255, UNSET=256, UPDATE=257, USE=258, USER=259, USING=260, - VALUES=261, VIEW=262, VIEWS=263, WHEN=264, WHERE=265, WINDOW=266, WITH=267, - YEAR=268, ZONE=269, KEY=270, EQ=271, NSEQ=272, NEQ=273, NEQJ=274, LT=275, - LTE=276, GT=277, GTE=278, PLUS=279, MINUS=280, ASTERISK=281, SLASH=282, - PERCENT=283, TILDE=284, AMPERSAND=285, PIPE=286, CONCAT_PIPE=287, HAT=288, - STRING=289, BIGINT_LITERAL=290, SMALLINT_LITERAL=291, TINYINT_LITERAL=292, - INTEGER_VALUE=293, EXPONENT_VALUE=294, DECIMAL_VALUE=295, FLOAT_LITERAL=296, - DOUBLE_LITERAL=297, BIGDECIMAL_LITERAL=298, IDENTIFIER=299, BACKQUOTED_IDENTIFIER=300, - SIMPLE_COMMENT=301, BRACKETED_COMMENT=302, WS=303, UNRECOGNIZED=304; - public static String[] channelNames = { - "DEFAULT_TOKEN_CHANNEL", "HIDDEN" - }; - - public static String[] modeNames = { - "DEFAULT_MODE" - }; - - private static String[] makeRuleNames() { - return new String[] { - "T__0", "T__1", "T__2", "T__3", "T__4", "T__5", "T__6", "T__7", "T__8", - "T__9", "T__10", "ADD", "AFTER", "ALL", "ALTER", "ANALYZE", "AND", "ANTI", - "ANY", "ARCHIVE", "ARRAY", "AS", "ASC", "AT", "AUTHORIZATION", "BETWEEN", - "BOTH", "BUCKET", "BUCKETS", "BY", "CACHE", "CASCADE", "CASE", "CAST", - "CHANGE", "CHECK", "CLEAR", "CLUSTER", "CLUSTERED", "CODEGEN", "COLLATE", - "COLLECTION", "COLUMN", "COLUMNS", "COMMENT", "COMMIT", "COMPACT", "COMPACTIONS", - "COMPUTE", "CONCATENATE", "CONSTRAINT", "COST", "CREATE", "CROSS", "CUBE", - "CURRENT", "CURRENT_DATE", "CURRENT_TIME", "CURRENT_TIMESTAMP", "CURRENT_USER", - "DAY", "DATA", "DATABASE", "DATABASES", "DBPROPERTIES", "DEFINED", "DELETE", - "DELIMITED", "DESC", "DESCRIBE", "DFS", "DIRECTORIES", "DIRECTORY", "DISTINCT", - "DISTRIBUTE", "DIV", "DROP", "ELSE", "END", "ESCAPE", "ESCAPED", "EXCEPT", - "EXCHANGE", "EXISTS", "EXPLAIN", "EXPORT", "EXTENDED", "EXTERNAL", "EXTRACT", - "FALSE", "FETCH", "FIELDS", "FILTER", "FILEFORMAT", "FIRST", "FOLLOWING", - "FOR", "FOREIGN", "FORMAT", "FORMATTED", "FROM", "FULL", "FUNCTION", - "FUNCTIONS", "GLOBAL", "GRANT", "GROUP", "GROUPING", "HAVING", "HOUR", - "IF", "IGNORE", "IMPORT", "IN", "INDEX", "INDEXES", "INNER", "INPATH", - "INPUTFORMAT", "INSERT", "INTERSECT", "INTERVAL", "INTO", "IS", "ITEMS", - "JOIN", "KEYS", "LAST", "LATERAL", "LAZY", "LEADING", "LEFT", "LIKE", - "LIMIT", "LINES", "LIST", "LOAD", "LOCAL", "LOCATION", "LOCK", "LOCKS", - "LOGICAL", "MACRO", "MAP", "MATCHED", "MERGE", "MINUTE", "MONTH", "MSCK", - "NAMESPACE", "NAMESPACES", "NATURAL", "NO", "NOT", "NULL", "NULLS", "OF", - "ON", "ONLY", "OPTION", "OPTIONS", "OR", "ORDER", "OUT", "OUTER", "OUTPUTFORMAT", - "OVER", "OVERLAPS", "OVERLAY", "OVERWRITE", "PARTITION", "PARTITIONED", - "PARTITIONS", "PERCENTLIT", "PIVOT", "PLACING", "POSITION", "PRECEDING", - "PRIMARY", "PRINCIPALS", "PROPERTIES", "PURGE", "QUERY", "RANGE", "RECORDREADER", - "RECORDWRITER", "RECOVER", "REDUCE", "REFERENCES", "REFRESH", "RENAME", - "REPAIR", "REPLACE", "RESET", "RESPECT", "RESTRICT", "REVOKE", "RIGHT", - "RLIKE", "ROLE", "ROLES", "ROLLBACK", "ROLLUP", "ROW", "ROWS", "SECOND", - "SCHEMA", "SELECT", "SEMI", "SEPARATED", "SERDE", "SERDEPROPERTIES", - "SESSION_USER", "SET", "SETMINUS", "SETS", "SHOW", "SKEWED", "SOME", - "SORT", "SORTED", "START", "STATISTICS", "STORED", "STRATIFY", "STRUCT", - "SUBSTR", "SUBSTRING", "SYNC", "TABLE", "TABLES", "TABLESAMPLE", "TBLPROPERTIES", - "TEMPORARY", "TERMINATED", "THEN", "TIME", "TO", "TOUCH", "TRAILING", - "TRANSACTION", "TRANSACTIONS", "TRANSFORM", "TRIM", "TRUE", "TRUNCATE", - "TRY_CAST", "TYPE", "UNARCHIVE", "UNBOUNDED", "UNCACHE", "UNION", "UNIQUE", - "UNKNOWN", "UNLOCK", "UNSET", "UPDATE", "USE", "USER", "USING", "VALUES", - "VIEW", "VIEWS", "WHEN", "WHERE", "WINDOW", "WITH", "YEAR", "ZONE", "KEY", - "EQ", "NSEQ", "NEQ", "NEQJ", "LT", "LTE", "GT", "GTE", "PLUS", "MINUS", - "ASTERISK", "SLASH", "PERCENT", "TILDE", "AMPERSAND", "PIPE", "CONCAT_PIPE", - "HAT", "STRING", "BIGINT_LITERAL", "SMALLINT_LITERAL", "TINYINT_LITERAL", - "INTEGER_VALUE", "EXPONENT_VALUE", "DECIMAL_VALUE", "FLOAT_LITERAL", - "DOUBLE_LITERAL", "BIGDECIMAL_LITERAL", "IDENTIFIER", "BACKQUOTED_IDENTIFIER", - "DECIMAL_DIGITS", "EXPONENT", "DIGIT", "LETTER", "SIMPLE_COMMENT", "BRACKETED_COMMENT", - "WS", "UNRECOGNIZED" - }; - } - public static final String[] ruleNames = makeRuleNames(); - - private static String[] makeLiteralNames() { - return new String[] { - null, "';'", "'('", "','", "')'", "'.'", "'/*+'", "'*/'", "'->'", "'['", - "']'", "':'", "'ADD'", "'AFTER'", "'ALL'", "'ALTER'", "'ANALYZE'", "'AND'", - "'ANTI'", "'ANY'", "'ARCHIVE'", "'ARRAY'", "'AS'", "'ASC'", "'AT'", "'AUTHORIZATION'", - "'BETWEEN'", "'BOTH'", "'BUCKET'", "'BUCKETS'", "'BY'", "'CACHE'", "'CASCADE'", - "'CASE'", "'CAST'", "'CHANGE'", "'CHECK'", "'CLEAR'", "'CLUSTER'", "'CLUSTERED'", - "'CODEGEN'", "'COLLATE'", "'COLLECTION'", "'COLUMN'", "'COLUMNS'", "'COMMENT'", - "'COMMIT'", "'COMPACT'", "'COMPACTIONS'", "'COMPUTE'", "'CONCATENATE'", - "'CONSTRAINT'", "'COST'", "'CREATE'", "'CROSS'", "'CUBE'", "'CURRENT'", - "'CURRENT_DATE'", "'CURRENT_TIME'", "'CURRENT_TIMESTAMP'", "'CURRENT_USER'", - "'DAY'", "'DATA'", "'DATABASE'", null, "'DBPROPERTIES'", "'DEFINED'", - "'DELETE'", "'DELIMITED'", "'DESC'", "'DESCRIBE'", "'DFS'", "'DIRECTORIES'", - "'DIRECTORY'", "'DISTINCT'", "'DISTRIBUTE'", "'DIV'", "'DROP'", "'ELSE'", - "'END'", "'ESCAPE'", "'ESCAPED'", "'EXCEPT'", "'EXCHANGE'", "'EXISTS'", - "'EXPLAIN'", "'EXPORT'", "'EXTENDED'", "'EXTERNAL'", "'EXTRACT'", "'FALSE'", - "'FETCH'", "'FIELDS'", "'FILTER'", "'FILEFORMAT'", "'FIRST'", "'FOLLOWING'", - "'FOR'", "'FOREIGN'", "'FORMAT'", "'FORMATTED'", "'FROM'", "'FULL'", - "'FUNCTION'", "'FUNCTIONS'", "'GLOBAL'", "'GRANT'", "'GROUP'", "'GROUPING'", - "'HAVING'", "'HOUR'", "'IF'", "'IGNORE'", "'IMPORT'", "'IN'", "'INDEX'", - "'INDEXES'", "'INNER'", "'INPATH'", "'INPUTFORMAT'", "'INSERT'", "'INTERSECT'", - "'INTERVAL'", "'INTO'", "'IS'", "'ITEMS'", "'JOIN'", "'KEYS'", "'LAST'", - "'LATERAL'", "'LAZY'", "'LEADING'", "'LEFT'", "'LIKE'", "'LIMIT'", "'LINES'", - "'LIST'", "'LOAD'", "'LOCAL'", "'LOCATION'", "'LOCK'", "'LOCKS'", "'LOGICAL'", - "'MACRO'", "'MAP'", "'MATCHED'", "'MERGE'", "'MINUTE'", "'MONTH'", "'MSCK'", - "'NAMESPACE'", "'NAMESPACES'", "'NATURAL'", "'NO'", null, "'NULL'", "'NULLS'", - "'OF'", "'ON'", "'ONLY'", "'OPTION'", "'OPTIONS'", "'OR'", "'ORDER'", - "'OUT'", "'OUTER'", "'OUTPUTFORMAT'", "'OVER'", "'OVERLAPS'", "'OVERLAY'", - "'OVERWRITE'", "'PARTITION'", "'PARTITIONED'", "'PARTITIONS'", "'PERCENT'", - "'PIVOT'", "'PLACING'", "'POSITION'", "'PRECEDING'", "'PRIMARY'", "'PRINCIPALS'", - "'PROPERTIES'", "'PURGE'", "'QUERY'", "'RANGE'", "'RECORDREADER'", "'RECORDWRITER'", - "'RECOVER'", "'REDUCE'", "'REFERENCES'", "'REFRESH'", "'RENAME'", "'REPAIR'", - "'REPLACE'", "'RESET'", "'RESPECT'", "'RESTRICT'", "'REVOKE'", "'RIGHT'", - null, "'ROLE'", "'ROLES'", "'ROLLBACK'", "'ROLLUP'", "'ROW'", "'ROWS'", - "'SECOND'", "'SCHEMA'", "'SELECT'", "'SEMI'", "'SEPARATED'", "'SERDE'", - "'SERDEPROPERTIES'", "'SESSION_USER'", "'SET'", "'MINUS'", "'SETS'", - "'SHOW'", "'SKEWED'", "'SOME'", "'SORT'", "'SORTED'", "'START'", "'STATISTICS'", - "'STORED'", "'STRATIFY'", "'STRUCT'", "'SUBSTR'", "'SUBSTRING'", "'SYNC'", - "'TABLE'", "'TABLES'", "'TABLESAMPLE'", "'TBLPROPERTIES'", null, "'TERMINATED'", - "'THEN'", "'TIME'", "'TO'", "'TOUCH'", "'TRAILING'", "'TRANSACTION'", - "'TRANSACTIONS'", "'TRANSFORM'", "'TRIM'", "'TRUE'", "'TRUNCATE'", "'TRY_CAST'", - "'TYPE'", "'UNARCHIVE'", "'UNBOUNDED'", "'UNCACHE'", "'UNION'", "'UNIQUE'", - "'UNKNOWN'", "'UNLOCK'", "'UNSET'", "'UPDATE'", "'USE'", "'USER'", "'USING'", - "'VALUES'", "'VIEW'", "'VIEWS'", "'WHEN'", "'WHERE'", "'WINDOW'", "'WITH'", - "'YEAR'", "'ZONE'", "'KEY'", null, "'<=>'", "'<>'", "'!='", "'<'", null, - "'>'", null, "'+'", "'-'", "'*'", "'/'", "'%'", "'~'", "'&'", "'|'", - "'||'", "'^'" - }; - } - private static final String[] _LITERAL_NAMES = makeLiteralNames(); - private static String[] makeSymbolicNames() { - return new String[] { - null, null, null, null, null, null, null, null, null, null, null, null, - "ADD", "AFTER", "ALL", "ALTER", "ANALYZE", "AND", "ANTI", "ANY", "ARCHIVE", - "ARRAY", "AS", "ASC", "AT", "AUTHORIZATION", "BETWEEN", "BOTH", "BUCKET", - "BUCKETS", "BY", "CACHE", "CASCADE", "CASE", "CAST", "CHANGE", "CHECK", - "CLEAR", "CLUSTER", "CLUSTERED", "CODEGEN", "COLLATE", "COLLECTION", - "COLUMN", "COLUMNS", "COMMENT", "COMMIT", "COMPACT", "COMPACTIONS", "COMPUTE", - "CONCATENATE", "CONSTRAINT", "COST", "CREATE", "CROSS", "CUBE", "CURRENT", - "CURRENT_DATE", "CURRENT_TIME", "CURRENT_TIMESTAMP", "CURRENT_USER", - "DAY", "DATA", "DATABASE", "DATABASES", "DBPROPERTIES", "DEFINED", "DELETE", - "DELIMITED", "DESC", "DESCRIBE", "DFS", "DIRECTORIES", "DIRECTORY", "DISTINCT", - "DISTRIBUTE", "DIV", "DROP", "ELSE", "END", "ESCAPE", "ESCAPED", "EXCEPT", - "EXCHANGE", "EXISTS", "EXPLAIN", "EXPORT", "EXTENDED", "EXTERNAL", "EXTRACT", - "FALSE", "FETCH", "FIELDS", "FILTER", "FILEFORMAT", "FIRST", "FOLLOWING", - "FOR", "FOREIGN", "FORMAT", "FORMATTED", "FROM", "FULL", "FUNCTION", - "FUNCTIONS", "GLOBAL", "GRANT", "GROUP", "GROUPING", "HAVING", "HOUR", - "IF", "IGNORE", "IMPORT", "IN", "INDEX", "INDEXES", "INNER", "INPATH", - "INPUTFORMAT", "INSERT", "INTERSECT", "INTERVAL", "INTO", "IS", "ITEMS", - "JOIN", "KEYS", "LAST", "LATERAL", "LAZY", "LEADING", "LEFT", "LIKE", - "LIMIT", "LINES", "LIST", "LOAD", "LOCAL", "LOCATION", "LOCK", "LOCKS", - "LOGICAL", "MACRO", "MAP", "MATCHED", "MERGE", "MINUTE", "MONTH", "MSCK", - "NAMESPACE", "NAMESPACES", "NATURAL", "NO", "NOT", "NULL", "NULLS", "OF", - "ON", "ONLY", "OPTION", "OPTIONS", "OR", "ORDER", "OUT", "OUTER", "OUTPUTFORMAT", - "OVER", "OVERLAPS", "OVERLAY", "OVERWRITE", "PARTITION", "PARTITIONED", - "PARTITIONS", "PERCENTLIT", "PIVOT", "PLACING", "POSITION", "PRECEDING", - "PRIMARY", "PRINCIPALS", "PROPERTIES", "PURGE", "QUERY", "RANGE", "RECORDREADER", - "RECORDWRITER", "RECOVER", "REDUCE", "REFERENCES", "REFRESH", "RENAME", - "REPAIR", "REPLACE", "RESET", "RESPECT", "RESTRICT", "REVOKE", "RIGHT", - "RLIKE", "ROLE", "ROLES", "ROLLBACK", "ROLLUP", "ROW", "ROWS", "SECOND", - "SCHEMA", "SELECT", "SEMI", "SEPARATED", "SERDE", "SERDEPROPERTIES", - "SESSION_USER", "SET", "SETMINUS", "SETS", "SHOW", "SKEWED", "SOME", - "SORT", "SORTED", "START", "STATISTICS", "STORED", "STRATIFY", "STRUCT", - "SUBSTR", "SUBSTRING", "SYNC", "TABLE", "TABLES", "TABLESAMPLE", "TBLPROPERTIES", - "TEMPORARY", "TERMINATED", "THEN", "TIME", "TO", "TOUCH", "TRAILING", - "TRANSACTION", "TRANSACTIONS", "TRANSFORM", "TRIM", "TRUE", "TRUNCATE", - "TRY_CAST", "TYPE", "UNARCHIVE", "UNBOUNDED", "UNCACHE", "UNION", "UNIQUE", - "UNKNOWN", "UNLOCK", "UNSET", "UPDATE", "USE", "USER", "USING", "VALUES", - "VIEW", "VIEWS", "WHEN", "WHERE", "WINDOW", "WITH", "YEAR", "ZONE", "KEY", - "EQ", "NSEQ", "NEQ", "NEQJ", "LT", "LTE", "GT", "GTE", "PLUS", "MINUS", - "ASTERISK", "SLASH", "PERCENT", "TILDE", "AMPERSAND", "PIPE", "CONCAT_PIPE", - "HAT", "STRING", "BIGINT_LITERAL", "SMALLINT_LITERAL", "TINYINT_LITERAL", - "INTEGER_VALUE", "EXPONENT_VALUE", "DECIMAL_VALUE", "FLOAT_LITERAL", - "DOUBLE_LITERAL", "BIGDECIMAL_LITERAL", "IDENTIFIER", "BACKQUOTED_IDENTIFIER", - "SIMPLE_COMMENT", "BRACKETED_COMMENT", "WS", "UNRECOGNIZED" - }; - } - private static final String[] _SYMBOLIC_NAMES = makeSymbolicNames(); - public static final Vocabulary VOCABULARY = new VocabularyImpl(_LITERAL_NAMES, _SYMBOLIC_NAMES); - - /** - * @deprecated Use {@link #VOCABULARY} instead. - */ - @Deprecated - public static final String[] tokenNames; - static { - tokenNames = new String[_SYMBOLIC_NAMES.length]; - for (int i = 0; i < tokenNames.length; i++) { - tokenNames[i] = VOCABULARY.getLiteralName(i); - if (tokenNames[i] == null) { - tokenNames[i] = VOCABULARY.getSymbolicName(i); - } - - if (tokenNames[i] == null) { - tokenNames[i] = ""; - } - } - } - - @Override - @Deprecated - public String[] getTokenNames() { - return tokenNames; - } - - @Override - - public Vocabulary getVocabulary() { - return VOCABULARY; - } - - - /** - * When true, parser should throw ParseExcetion for unclosed bracketed comment. - */ - public boolean has_unclosed_bracketed_comment = false; - - /** - * Verify whether current token is a valid decimal token (which contains dot). - * Returns true if the character that follows the token is not a digit or letter or underscore. - * - * For example: - * For char stream "2.3", "2." is not a valid decimal token, because it is followed by digit '3'. - * For char stream "2.3_", "2.3" is not a valid decimal token, because it is followed by '_'. - * For char stream "2.3W", "2.3" is not a valid decimal token, because it is followed by 'W'. - * For char stream "12.0D 34.E2+0.12 " 12.0D is a valid decimal token because it is followed - * by a space. 34.E2 is a valid decimal token because it is followed by symbol '+' - * which is not a digit or letter or underscore. - */ - public boolean isValidDecimal() { - int nextChar = _input.LA(1); - if (nextChar >= 'A' && nextChar <= 'Z' || nextChar >= '0' && nextChar <= '9' || - nextChar == '_') { - return false; - } else { - return true; - } - } - - /** - * This method will be called when we see '/*' and try to match it as a bracketed comment. - * If the next character is '+', it should be parsed as hint later, and we cannot match - * it as a bracketed comment. - * - * Returns true if the next character is '+'. - */ - public boolean isHint() { - int nextChar = _input.LA(1); - if (nextChar == '+') { - return true; - } else { - return false; - } - } - - /** - * This method will be called when the character stream ends and try to find out the - * unclosed bracketed comment. - * If the method be called, it means the end of the entire character stream match, - * and we set the flag and fail later. - */ - public void markUnclosedComment() { - has_unclosed_bracketed_comment = true; - } - - - public MixedFormatSqlExtendLexer(CharStream input) { - super(input); - _interp = new LexerATNSimulator(this,_ATN,_decisionToDFA,_sharedContextCache); - } - - @Override - public String getGrammarFileName() { return "MixedFormatSqlExtend.g4"; } - - @Override - public String[] getRuleNames() { return ruleNames; } - - @Override - public String getSerializedATN() { return _serializedATN; } - - @Override - public String[] getChannelNames() { return channelNames; } - - @Override - public String[] getModeNames() { return modeNames; } - - @Override - public ATN getATN() { return _ATN; } - - @Override - public void action(RuleContext _localctx, int ruleIndex, int actionIndex) { - switch (ruleIndex) { - case 305: - BRACKETED_COMMENT_action((RuleContext)_localctx, actionIndex); - break; - } - } - private void BRACKETED_COMMENT_action(RuleContext _localctx, int actionIndex) { - switch (actionIndex) { - case 0: - markUnclosedComment(); - break; - } - } - @Override - public boolean sempred(RuleContext _localctx, int ruleIndex, int predIndex) { - switch (ruleIndex) { - case 293: - return EXPONENT_VALUE_sempred((RuleContext)_localctx, predIndex); - case 294: - return DECIMAL_VALUE_sempred((RuleContext)_localctx, predIndex); - case 295: - return FLOAT_LITERAL_sempred((RuleContext)_localctx, predIndex); - case 296: - return DOUBLE_LITERAL_sempred((RuleContext)_localctx, predIndex); - case 297: - return BIGDECIMAL_LITERAL_sempred((RuleContext)_localctx, predIndex); - case 305: - return BRACKETED_COMMENT_sempred((RuleContext)_localctx, predIndex); - } - return true; - } - private boolean EXPONENT_VALUE_sempred(RuleContext _localctx, int predIndex) { - switch (predIndex) { - case 0: - return isValidDecimal(); - } - return true; - } - private boolean DECIMAL_VALUE_sempred(RuleContext _localctx, int predIndex) { - switch (predIndex) { - case 1: - return isValidDecimal(); - } - return true; - } - private boolean FLOAT_LITERAL_sempred(RuleContext _localctx, int predIndex) { - switch (predIndex) { - case 2: - return isValidDecimal(); - } - return true; - } - private boolean DOUBLE_LITERAL_sempred(RuleContext _localctx, int predIndex) { - switch (predIndex) { - case 3: - return isValidDecimal(); - } - return true; - } - private boolean BIGDECIMAL_LITERAL_sempred(RuleContext _localctx, int predIndex) { - switch (predIndex) { - case 4: - return isValidDecimal(); - } - return true; - } - private boolean BRACKETED_COMMENT_sempred(RuleContext _localctx, int predIndex) { - switch (predIndex) { - case 5: - return !isHint(); - } - return true; - } - - private static final int _serializedATNSegments = 2; - private static final String _serializedATNSegment0 = - "\3\u608b\ua72a\u8133\ub9ed\u417c\u3be7\u7786\u5964\2\u0132\u0b00\b\1\4"+ - "\2\t\2\4\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n"+ - "\4\13\t\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22"+ - "\t\22\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\4\30\t\30\4\31"+ - "\t\31\4\32\t\32\4\33\t\33\4\34\t\34\4\35\t\35\4\36\t\36\4\37\t\37\4 \t"+ - " \4!\t!\4\"\t\"\4#\t#\4$\t$\4%\t%\4&\t&\4\'\t\'\4(\t(\4)\t)\4*\t*\4+\t"+ - "+\4,\t,\4-\t-\4.\t.\4/\t/\4\60\t\60\4\61\t\61\4\62\t\62\4\63\t\63\4\64"+ - "\t\64\4\65\t\65\4\66\t\66\4\67\t\67\48\t8\49\t9\4:\t:\4;\t;\4<\t<\4=\t"+ - "=\4>\t>\4?\t?\4@\t@\4A\tA\4B\tB\4C\tC\4D\tD\4E\tE\4F\tF\4G\tG\4H\tH\4"+ - "I\tI\4J\tJ\4K\tK\4L\tL\4M\tM\4N\tN\4O\tO\4P\tP\4Q\tQ\4R\tR\4S\tS\4T\t"+ - "T\4U\tU\4V\tV\4W\tW\4X\tX\4Y\tY\4Z\tZ\4[\t[\4\\\t\\\4]\t]\4^\t^\4_\t_"+ - "\4`\t`\4a\ta\4b\tb\4c\tc\4d\td\4e\te\4f\tf\4g\tg\4h\th\4i\ti\4j\tj\4k"+ - "\tk\4l\tl\4m\tm\4n\tn\4o\to\4p\tp\4q\tq\4r\tr\4s\ts\4t\tt\4u\tu\4v\tv"+ - "\4w\tw\4x\tx\4y\ty\4z\tz\4{\t{\4|\t|\4}\t}\4~\t~\4\177\t\177\4\u0080\t"+ - "\u0080\4\u0081\t\u0081\4\u0082\t\u0082\4\u0083\t\u0083\4\u0084\t\u0084"+ - "\4\u0085\t\u0085\4\u0086\t\u0086\4\u0087\t\u0087\4\u0088\t\u0088\4\u0089"+ - "\t\u0089\4\u008a\t\u008a\4\u008b\t\u008b\4\u008c\t\u008c\4\u008d\t\u008d"+ - "\4\u008e\t\u008e\4\u008f\t\u008f\4\u0090\t\u0090\4\u0091\t\u0091\4\u0092"+ - "\t\u0092\4\u0093\t\u0093\4\u0094\t\u0094\4\u0095\t\u0095\4\u0096\t\u0096"+ - "\4\u0097\t\u0097\4\u0098\t\u0098\4\u0099\t\u0099\4\u009a\t\u009a\4\u009b"+ - "\t\u009b\4\u009c\t\u009c\4\u009d\t\u009d\4\u009e\t\u009e\4\u009f\t\u009f"+ - "\4\u00a0\t\u00a0\4\u00a1\t\u00a1\4\u00a2\t\u00a2\4\u00a3\t\u00a3\4\u00a4"+ - "\t\u00a4\4\u00a5\t\u00a5\4\u00a6\t\u00a6\4\u00a7\t\u00a7\4\u00a8\t\u00a8"+ - "\4\u00a9\t\u00a9\4\u00aa\t\u00aa\4\u00ab\t\u00ab\4\u00ac\t\u00ac\4\u00ad"+ - "\t\u00ad\4\u00ae\t\u00ae\4\u00af\t\u00af\4\u00b0\t\u00b0\4\u00b1\t\u00b1"+ - "\4\u00b2\t\u00b2\4\u00b3\t\u00b3\4\u00b4\t\u00b4\4\u00b5\t\u00b5\4\u00b6"+ - "\t\u00b6\4\u00b7\t\u00b7\4\u00b8\t\u00b8\4\u00b9\t\u00b9\4\u00ba\t\u00ba"+ - "\4\u00bb\t\u00bb\4\u00bc\t\u00bc\4\u00bd\t\u00bd\4\u00be\t\u00be\4\u00bf"+ - "\t\u00bf\4\u00c0\t\u00c0\4\u00c1\t\u00c1\4\u00c2\t\u00c2\4\u00c3\t\u00c3"+ - "\4\u00c4\t\u00c4\4\u00c5\t\u00c5\4\u00c6\t\u00c6\4\u00c7\t\u00c7\4\u00c8"+ - "\t\u00c8\4\u00c9\t\u00c9\4\u00ca\t\u00ca\4\u00cb\t\u00cb\4\u00cc\t\u00cc"+ - "\4\u00cd\t\u00cd\4\u00ce\t\u00ce\4\u00cf\t\u00cf\4\u00d0\t\u00d0\4\u00d1"+ - "\t\u00d1\4\u00d2\t\u00d2\4\u00d3\t\u00d3\4\u00d4\t\u00d4\4\u00d5\t\u00d5"+ - "\4\u00d6\t\u00d6\4\u00d7\t\u00d7\4\u00d8\t\u00d8\4\u00d9\t\u00d9\4\u00da"+ - "\t\u00da\4\u00db\t\u00db\4\u00dc\t\u00dc\4\u00dd\t\u00dd\4\u00de\t\u00de"+ - "\4\u00df\t\u00df\4\u00e0\t\u00e0\4\u00e1\t\u00e1\4\u00e2\t\u00e2\4\u00e3"+ - "\t\u00e3\4\u00e4\t\u00e4\4\u00e5\t\u00e5\4\u00e6\t\u00e6\4\u00e7\t\u00e7"+ - "\4\u00e8\t\u00e8\4\u00e9\t\u00e9\4\u00ea\t\u00ea\4\u00eb\t\u00eb\4\u00ec"+ - "\t\u00ec\4\u00ed\t\u00ed\4\u00ee\t\u00ee\4\u00ef\t\u00ef\4\u00f0\t\u00f0"+ - "\4\u00f1\t\u00f1\4\u00f2\t\u00f2\4\u00f3\t\u00f3\4\u00f4\t\u00f4\4\u00f5"+ - "\t\u00f5\4\u00f6\t\u00f6\4\u00f7\t\u00f7\4\u00f8\t\u00f8\4\u00f9\t\u00f9"+ - "\4\u00fa\t\u00fa\4\u00fb\t\u00fb\4\u00fc\t\u00fc\4\u00fd\t\u00fd\4\u00fe"+ - "\t\u00fe\4\u00ff\t\u00ff\4\u0100\t\u0100\4\u0101\t\u0101\4\u0102\t\u0102"+ - "\4\u0103\t\u0103\4\u0104\t\u0104\4\u0105\t\u0105\4\u0106\t\u0106\4\u0107"+ - "\t\u0107\4\u0108\t\u0108\4\u0109\t\u0109\4\u010a\t\u010a\4\u010b\t\u010b"+ - "\4\u010c\t\u010c\4\u010d\t\u010d\4\u010e\t\u010e\4\u010f\t\u010f\4\u0110"+ - "\t\u0110\4\u0111\t\u0111\4\u0112\t\u0112\4\u0113\t\u0113\4\u0114\t\u0114"+ - "\4\u0115\t\u0115\4\u0116\t\u0116\4\u0117\t\u0117\4\u0118\t\u0118\4\u0119"+ - "\t\u0119\4\u011a\t\u011a\4\u011b\t\u011b\4\u011c\t\u011c\4\u011d\t\u011d"+ - "\4\u011e\t\u011e\4\u011f\t\u011f\4\u0120\t\u0120\4\u0121\t\u0121\4\u0122"+ - "\t\u0122\4\u0123\t\u0123\4\u0124\t\u0124\4\u0125\t\u0125\4\u0126\t\u0126"+ - "\4\u0127\t\u0127\4\u0128\t\u0128\4\u0129\t\u0129\4\u012a\t\u012a\4\u012b"+ - "\t\u012b\4\u012c\t\u012c\4\u012d\t\u012d\4\u012e\t\u012e\4\u012f\t\u012f"+ - "\4\u0130\t\u0130\4\u0131\t\u0131\4\u0132\t\u0132\4\u0133\t\u0133\4\u0134"+ - "\t\u0134\4\u0135\t\u0135\3\2\3\2\3\3\3\3\3\4\3\4\3\5\3\5\3\6\3\6\3\7\3"+ - "\7\3\7\3\7\3\b\3\b\3\b\3\t\3\t\3\t\3\n\3\n\3\13\3\13\3\f\3\f\3\r\3\r\3"+ - "\r\3\r\3\16\3\16\3\16\3\16\3\16\3\16\3\17\3\17\3\17\3\17\3\20\3\20\3\20"+ - "\3\20\3\20\3\20\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\22\3\22\3\22"+ - "\3\22\3\23\3\23\3\23\3\23\3\23\3\24\3\24\3\24\3\24\3\25\3\25\3\25\3\25"+ - "\3\25\3\25\3\25\3\25\3\26\3\26\3\26\3\26\3\26\3\26\3\27\3\27\3\27\3\30"+ - "\3\30\3\30\3\30\3\31\3\31\3\31\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32"+ - "\3\32\3\32\3\32\3\32\3\32\3\32\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33"+ - "\3\34\3\34\3\34\3\34\3\34\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\36\3\36"+ - "\3\36\3\36\3\36\3\36\3\36\3\36\3\37\3\37\3\37\3 \3 \3 \3 \3 \3 \3!\3!"+ - "\3!\3!\3!\3!\3!\3!\3\"\3\"\3\"\3\"\3\"\3#\3#\3#\3#\3#\3$\3$\3$\3$\3$\3"+ - "$\3$\3%\3%\3%\3%\3%\3%\3&\3&\3&\3&\3&\3&\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3"+ - "\'\3(\3(\3(\3(\3(\3(\3(\3(\3(\3(\3)\3)\3)\3)\3)\3)\3)\3)\3*\3*\3*\3*\3"+ - "*\3*\3*\3*\3+\3+\3+\3+\3+\3+\3+\3+\3+\3+\3+\3,\3,\3,\3,\3,\3,\3,\3-\3"+ - "-\3-\3-\3-\3-\3-\3-\3.\3.\3.\3.\3.\3.\3.\3.\3/\3/\3/\3/\3/\3/\3/\3\60"+ - "\3\60\3\60\3\60\3\60\3\60\3\60\3\60\3\61\3\61\3\61\3\61\3\61\3\61\3\61"+ - "\3\61\3\61\3\61\3\61\3\61\3\62\3\62\3\62\3\62\3\62\3\62\3\62\3\62\3\63"+ - "\3\63\3\63\3\63\3\63\3\63\3\63\3\63\3\63\3\63\3\63\3\63\3\64\3\64\3\64"+ - "\3\64\3\64\3\64\3\64\3\64\3\64\3\64\3\64\3\65\3\65\3\65\3\65\3\65\3\66"+ - "\3\66\3\66\3\66\3\66\3\66\3\66\3\67\3\67\3\67\3\67\3\67\3\67\38\38\38"+ - "\38\38\39\39\39\39\39\39\39\39\3:\3:\3:\3:\3:\3:\3:\3:\3:\3:\3:\3:\3:"+ - "\3;\3;\3;\3;\3;\3;\3;\3;\3;\3;\3;\3;\3;\3<\3<\3<\3<\3<\3<\3<\3<\3<\3<"+ - "\3<\3<\3<\3<\3<\3<\3<\3<\3=\3=\3=\3=\3=\3=\3=\3=\3=\3=\3=\3=\3=\3>\3>"+ - "\3>\3>\3?\3?\3?\3?\3?\3@\3@\3@\3@\3@\3@\3@\3@\3@\3A\3A\3A\3A\3A\3A\3A"+ - "\3A\3A\3A\3A\3A\3A\3A\3A\3A\5A\u0417\nA\3B\3B\3B\3B\3B\3B\3B\3B\3B\3B"+ - "\3B\3B\3B\3C\3C\3C\3C\3C\3C\3C\3C\3D\3D\3D\3D\3D\3D\3D\3E\3E\3E\3E\3E"+ - "\3E\3E\3E\3E\3E\3F\3F\3F\3F\3F\3G\3G\3G\3G\3G\3G\3G\3G\3G\3H\3H\3H\3H"+ - "\3I\3I\3I\3I\3I\3I\3I\3I\3I\3I\3I\3I\3J\3J\3J\3J\3J\3J\3J\3J\3J\3J\3K"+ - "\3K\3K\3K\3K\3K\3K\3K\3K\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3M\3M\3M\3M"+ - "\3N\3N\3N\3N\3N\3O\3O\3O\3O\3O\3P\3P\3P\3P\3Q\3Q\3Q\3Q\3Q\3Q\3Q\3R\3R"+ - "\3R\3R\3R\3R\3R\3R\3S\3S\3S\3S\3S\3S\3S\3T\3T\3T\3T\3T\3T\3T\3T\3T\3U"+ - "\3U\3U\3U\3U\3U\3U\3V\3V\3V\3V\3V\3V\3V\3V\3W\3W\3W\3W\3W\3W\3W\3X\3X"+ - "\3X\3X\3X\3X\3X\3X\3X\3Y\3Y\3Y\3Y\3Y\3Y\3Y\3Y\3Y\3Z\3Z\3Z\3Z\3Z\3Z\3Z"+ - "\3Z\3[\3[\3[\3[\3[\3[\3\\\3\\\3\\\3\\\3\\\3\\\3]\3]\3]\3]\3]\3]\3]\3^"+ - "\3^\3^\3^\3^\3^\3^\3_\3_\3_\3_\3_\3_\3_\3_\3_\3_\3_\3`\3`\3`\3`\3`\3`"+ - "\3a\3a\3a\3a\3a\3a\3a\3a\3a\3a\3b\3b\3b\3b\3c\3c\3c\3c\3c\3c\3c\3c\3d"+ - "\3d\3d\3d\3d\3d\3d\3e\3e\3e\3e\3e\3e\3e\3e\3e\3e\3f\3f\3f\3f\3f\3g\3g"+ - "\3g\3g\3g\3h\3h\3h\3h\3h\3h\3h\3h\3h\3i\3i\3i\3i\3i\3i\3i\3i\3i\3i\3j"+ - "\3j\3j\3j\3j\3j\3j\3k\3k\3k\3k\3k\3k\3l\3l\3l\3l\3l\3l\3m\3m\3m\3m\3m"+ - "\3m\3m\3m\3m\3n\3n\3n\3n\3n\3n\3n\3o\3o\3o\3o\3o\3p\3p\3p\3q\3q\3q\3q"+ - "\3q\3q\3q\3r\3r\3r\3r\3r\3r\3r\3s\3s\3s\3t\3t\3t\3t\3t\3t\3u\3u\3u\3u"+ - "\3u\3u\3u\3u\3v\3v\3v\3v\3v\3v\3w\3w\3w\3w\3w\3w\3w\3x\3x\3x\3x\3x\3x"+ - "\3x\3x\3x\3x\3x\3x\3y\3y\3y\3y\3y\3y\3y\3z\3z\3z\3z\3z\3z\3z\3z\3z\3z"+ - "\3{\3{\3{\3{\3{\3{\3{\3{\3{\3|\3|\3|\3|\3|\3}\3}\3}\3~\3~\3~\3~\3~\3~"+ - "\3\177\3\177\3\177\3\177\3\177\3\u0080\3\u0080\3\u0080\3\u0080\3\u0080"+ - "\3\u0081\3\u0081\3\u0081\3\u0081\3\u0081\3\u0082\3\u0082\3\u0082\3\u0082"+ - "\3\u0082\3\u0082\3\u0082\3\u0082\3\u0083\3\u0083\3\u0083\3\u0083\3\u0083"+ - "\3\u0084\3\u0084\3\u0084\3\u0084\3\u0084\3\u0084\3\u0084\3\u0084\3\u0085"+ - "\3\u0085\3\u0085\3\u0085\3\u0085\3\u0086\3\u0086\3\u0086\3\u0086\3\u0086"+ - "\3\u0087\3\u0087\3\u0087\3\u0087\3\u0087\3\u0087\3\u0088\3\u0088\3\u0088"+ - "\3\u0088\3\u0088\3\u0088\3\u0089\3\u0089\3\u0089\3\u0089\3\u0089\3\u008a"+ - "\3\u008a\3\u008a\3\u008a\3\u008a\3\u008b\3\u008b\3\u008b\3\u008b\3\u008b"+ - "\3\u008b\3\u008c\3\u008c\3\u008c\3\u008c\3\u008c\3\u008c\3\u008c\3\u008c"+ - "\3\u008c\3\u008d\3\u008d\3\u008d\3\u008d\3\u008d\3\u008e\3\u008e\3\u008e"+ - "\3\u008e\3\u008e\3\u008e\3\u008f\3\u008f\3\u008f\3\u008f\3\u008f\3\u008f"+ - "\3\u008f\3\u008f\3\u0090\3\u0090\3\u0090\3\u0090\3\u0090\3\u0090\3\u0091"+ - "\3\u0091\3\u0091\3\u0091\3\u0092\3\u0092\3\u0092\3\u0092\3\u0092\3\u0092"+ - "\3\u0092\3\u0092\3\u0093\3\u0093\3\u0093\3\u0093\3\u0093\3\u0093\3\u0094"+ - "\3\u0094\3\u0094\3\u0094\3\u0094\3\u0094\3\u0094\3\u0095\3\u0095\3\u0095"+ - "\3\u0095\3\u0095\3\u0095\3\u0096\3\u0096\3\u0096\3\u0096\3\u0096\3\u0097"+ - "\3\u0097\3\u0097\3\u0097\3\u0097\3\u0097\3\u0097\3\u0097\3\u0097\3\u0097"+ - "\3\u0098\3\u0098\3\u0098\3\u0098\3\u0098\3\u0098\3\u0098\3\u0098\3\u0098"+ - "\3\u0098\3\u0098\3\u0099\3\u0099\3\u0099\3\u0099\3\u0099\3\u0099\3\u0099"+ - "\3\u0099\3\u009a\3\u009a\3\u009a\3\u009b\3\u009b\3\u009b\3\u009b\5\u009b"+ - "\u068a\n\u009b\3\u009c\3\u009c\3\u009c\3\u009c\3\u009c\3\u009d\3\u009d"+ - "\3\u009d\3\u009d\3\u009d\3\u009d\3\u009e\3\u009e\3\u009e\3\u009f\3\u009f"+ - "\3\u009f\3\u00a0\3\u00a0\3\u00a0\3\u00a0\3\u00a0\3\u00a1\3\u00a1\3\u00a1"+ - "\3\u00a1\3\u00a1\3\u00a1\3\u00a1\3\u00a2\3\u00a2\3\u00a2\3\u00a2\3\u00a2"+ - "\3\u00a2\3\u00a2\3\u00a2\3\u00a3\3\u00a3\3\u00a3\3\u00a4\3\u00a4\3\u00a4"+ - "\3\u00a4\3\u00a4\3\u00a4\3\u00a5\3\u00a5\3\u00a5\3\u00a5\3\u00a6\3\u00a6"+ - "\3\u00a6\3\u00a6\3\u00a6\3\u00a6\3\u00a7\3\u00a7\3\u00a7\3\u00a7\3\u00a7"+ - "\3\u00a7\3\u00a7\3\u00a7\3\u00a7\3\u00a7\3\u00a7\3\u00a7\3\u00a7\3\u00a8"+ - "\3\u00a8\3\u00a8\3\u00a8\3\u00a8\3\u00a9\3\u00a9\3\u00a9\3\u00a9\3\u00a9"+ - "\3\u00a9\3\u00a9\3\u00a9\3\u00a9\3\u00aa\3\u00aa\3\u00aa\3\u00aa\3\u00aa"+ - "\3\u00aa\3\u00aa\3\u00aa\3\u00ab\3\u00ab\3\u00ab\3\u00ab\3\u00ab\3\u00ab"+ - "\3\u00ab\3\u00ab\3\u00ab\3\u00ab\3\u00ac\3\u00ac\3\u00ac\3\u00ac\3\u00ac"+ - "\3\u00ac\3\u00ac\3\u00ac\3\u00ac\3\u00ac\3\u00ad\3\u00ad\3\u00ad\3\u00ad"+ - "\3\u00ad\3\u00ad\3\u00ad\3\u00ad\3\u00ad\3\u00ad\3\u00ad\3\u00ad\3\u00ae"+ - "\3\u00ae\3\u00ae\3\u00ae\3\u00ae\3\u00ae\3\u00ae\3\u00ae\3\u00ae\3\u00ae"+ - "\3\u00ae\3\u00af\3\u00af\3\u00af\3\u00af\3\u00af\3\u00af\3\u00af\3\u00af"+ - "\3\u00b0\3\u00b0\3\u00b0\3\u00b0\3\u00b0\3\u00b0\3\u00b1\3\u00b1\3\u00b1"+ - "\3\u00b1\3\u00b1\3\u00b1\3\u00b1\3\u00b1\3\u00b2\3\u00b2\3\u00b2\3\u00b2"+ - "\3\u00b2\3\u00b2\3\u00b2\3\u00b2\3\u00b2\3\u00b3\3\u00b3\3\u00b3\3\u00b3"+ - "\3\u00b3\3\u00b3\3\u00b3\3\u00b3\3\u00b3\3\u00b3\3\u00b4\3\u00b4\3\u00b4"+ - "\3\u00b4\3\u00b4\3\u00b4\3\u00b4\3\u00b4\3\u00b5\3\u00b5\3\u00b5\3\u00b5"+ - "\3\u00b5\3\u00b5\3\u00b5\3\u00b5\3\u00b5\3\u00b5\3\u00b5\3\u00b6\3\u00b6"+ - "\3\u00b6\3\u00b6\3\u00b6\3\u00b6\3\u00b6\3\u00b6\3\u00b6\3\u00b6\3\u00b6"+ - "\3\u00b7\3\u00b7\3\u00b7\3\u00b7\3\u00b7\3\u00b7\3\u00b8\3\u00b8\3\u00b8"+ - "\3\u00b8\3\u00b8\3\u00b8\3\u00b9\3\u00b9\3\u00b9\3\u00b9\3\u00b9\3\u00b9"+ - "\3\u00ba\3\u00ba\3\u00ba\3\u00ba\3\u00ba\3\u00ba\3\u00ba\3\u00ba\3\u00ba"+ - "\3\u00ba\3\u00ba\3\u00ba\3\u00ba\3\u00bb\3\u00bb\3\u00bb\3\u00bb\3\u00bb"+ - "\3\u00bb\3\u00bb\3\u00bb\3\u00bb\3\u00bb\3\u00bb\3\u00bb\3\u00bb\3\u00bc"+ - "\3\u00bc\3\u00bc\3\u00bc\3\u00bc\3\u00bc\3\u00bc\3\u00bc\3\u00bd\3\u00bd"+ - "\3\u00bd\3\u00bd\3\u00bd\3\u00bd\3\u00bd\3\u00be\3\u00be\3\u00be\3\u00be"+ - "\3\u00be\3\u00be\3\u00be\3\u00be\3\u00be\3\u00be\3\u00be\3\u00bf\3\u00bf"+ - "\3\u00bf\3\u00bf\3\u00bf\3\u00bf\3\u00bf\3\u00bf\3\u00c0\3\u00c0\3\u00c0"+ - "\3\u00c0\3\u00c0\3\u00c0\3\u00c0\3\u00c1\3\u00c1\3\u00c1\3\u00c1\3\u00c1"+ - "\3\u00c1\3\u00c1\3\u00c2\3\u00c2\3\u00c2\3\u00c2\3\u00c2\3\u00c2\3\u00c2"+ - "\3\u00c2\3\u00c3\3\u00c3\3\u00c3\3\u00c3\3\u00c3\3\u00c3\3\u00c4\3\u00c4"+ - "\3\u00c4\3\u00c4\3\u00c4\3\u00c4\3\u00c4\3\u00c4\3\u00c5\3\u00c5\3\u00c5"+ - "\3\u00c5\3\u00c5\3\u00c5\3\u00c5\3\u00c5\3\u00c5\3\u00c6\3\u00c6\3\u00c6"+ - "\3\u00c6\3\u00c6\3\u00c6\3\u00c6\3\u00c7\3\u00c7\3\u00c7\3\u00c7\3\u00c7"+ - "\3\u00c7\3\u00c8\3\u00c8\3\u00c8\3\u00c8\3\u00c8\3\u00c8\3\u00c8\3\u00c8"+ - "\3\u00c8\3\u00c8\3\u00c8\5\u00c8\u07ec\n\u00c8\3\u00c9\3\u00c9\3\u00c9"+ - "\3\u00c9\3\u00c9\3\u00ca\3\u00ca\3\u00ca\3\u00ca\3\u00ca\3\u00ca\3\u00cb"+ - "\3\u00cb\3\u00cb\3\u00cb\3\u00cb\3\u00cb\3\u00cb\3\u00cb\3\u00cb\3\u00cc"+ - "\3\u00cc\3\u00cc\3\u00cc\3\u00cc\3\u00cc\3\u00cc\3\u00cd\3\u00cd\3\u00cd"+ - "\3\u00cd\3\u00ce\3\u00ce\3\u00ce\3\u00ce\3\u00ce\3\u00cf\3\u00cf\3\u00cf"+ - "\3\u00cf\3\u00cf\3\u00cf\3\u00cf\3\u00d0\3\u00d0\3\u00d0\3\u00d0\3\u00d0"+ - "\3\u00d0\3\u00d0\3\u00d1\3\u00d1\3\u00d1\3\u00d1\3\u00d1\3\u00d1\3\u00d1"+ - "\3\u00d2\3\u00d2\3\u00d2\3\u00d2\3\u00d2\3\u00d3\3\u00d3\3\u00d3\3\u00d3"+ - "\3\u00d3\3\u00d3\3\u00d3\3\u00d3\3\u00d3\3\u00d3\3\u00d4\3\u00d4\3\u00d4"+ - "\3\u00d4\3\u00d4\3\u00d4\3\u00d5\3\u00d5\3\u00d5\3\u00d5\3\u00d5\3\u00d5"+ - "\3\u00d5\3\u00d5\3\u00d5\3\u00d5\3\u00d5\3\u00d5\3\u00d5\3\u00d5\3\u00d5"+ - "\3\u00d5\3\u00d6\3\u00d6\3\u00d6\3\u00d6\3\u00d6\3\u00d6\3\u00d6\3\u00d6"+ - "\3\u00d6\3\u00d6\3\u00d6\3\u00d6\3\u00d6\3\u00d7\3\u00d7\3\u00d7\3\u00d7"+ - "\3\u00d8\3\u00d8\3\u00d8\3\u00d8\3\u00d8\3\u00d8\3\u00d9\3\u00d9\3\u00d9"+ - "\3\u00d9\3\u00d9\3\u00da\3\u00da\3\u00da\3\u00da\3\u00da\3\u00db\3\u00db"+ - "\3\u00db\3\u00db\3\u00db\3\u00db\3\u00db\3\u00dc\3\u00dc\3\u00dc\3\u00dc"+ - "\3\u00dc\3\u00dd\3\u00dd\3\u00dd\3\u00dd\3\u00dd\3\u00de\3\u00de\3\u00de"+ - "\3\u00de\3\u00de\3\u00de\3\u00de\3\u00df\3\u00df\3\u00df\3\u00df\3\u00df"+ - "\3\u00df\3\u00e0\3\u00e0\3\u00e0\3\u00e0\3\u00e0\3\u00e0\3\u00e0\3\u00e0"+ - "\3\u00e0\3\u00e0\3\u00e0\3\u00e1\3\u00e1\3\u00e1\3\u00e1\3\u00e1\3\u00e1"+ - "\3\u00e1\3\u00e2\3\u00e2\3\u00e2\3\u00e2\3\u00e2\3\u00e2\3\u00e2\3\u00e2"+ - "\3\u00e2\3\u00e3\3\u00e3\3\u00e3\3\u00e3\3\u00e3\3\u00e3\3\u00e3\3\u00e4"+ - "\3\u00e4\3\u00e4\3\u00e4\3\u00e4\3\u00e4\3\u00e4\3\u00e5\3\u00e5\3\u00e5"+ - "\3\u00e5\3\u00e5\3\u00e5\3\u00e5\3\u00e5\3\u00e5\3\u00e5\3\u00e6\3\u00e6"+ - "\3\u00e6\3\u00e6\3\u00e6\3\u00e7\3\u00e7\3\u00e7\3\u00e7\3\u00e7\3\u00e7"+ - "\3\u00e8\3\u00e8\3\u00e8\3\u00e8\3\u00e8\3\u00e8\3\u00e8\3\u00e9\3\u00e9"+ - "\3\u00e9\3\u00e9\3\u00e9\3\u00e9\3\u00e9\3\u00e9\3\u00e9\3\u00e9\3\u00e9"+ - "\3\u00e9\3\u00ea\3\u00ea\3\u00ea\3\u00ea\3\u00ea\3\u00ea\3\u00ea\3\u00ea"+ - "\3\u00ea\3\u00ea\3\u00ea\3\u00ea\3\u00ea\3\u00ea\3\u00eb\3\u00eb\3\u00eb"+ - "\3\u00eb\3\u00eb\3\u00eb\3\u00eb\3\u00eb\3\u00eb\3\u00eb\3\u00eb\3\u00eb"+ - "\3\u00eb\5\u00eb\u08f7\n\u00eb\3\u00ec\3\u00ec\3\u00ec\3\u00ec\3\u00ec"+ - "\3\u00ec\3\u00ec\3\u00ec\3\u00ec\3\u00ec\3\u00ec\3\u00ed\3\u00ed\3\u00ed"+ - "\3\u00ed\3\u00ed\3\u00ee\3\u00ee\3\u00ee\3\u00ee\3\u00ee\3\u00ef\3\u00ef"+ - "\3\u00ef\3\u00f0\3\u00f0\3\u00f0\3\u00f0\3\u00f0\3\u00f0\3\u00f1\3\u00f1"+ - "\3\u00f1\3\u00f1\3\u00f1\3\u00f1\3\u00f1\3\u00f1\3\u00f1\3\u00f2\3\u00f2"+ - "\3\u00f2\3\u00f2\3\u00f2\3\u00f2\3\u00f2\3\u00f2\3\u00f2\3\u00f2\3\u00f2"+ - "\3\u00f2\3\u00f3\3\u00f3\3\u00f3\3\u00f3\3\u00f3\3\u00f3\3\u00f3\3\u00f3"+ - "\3\u00f3\3\u00f3\3\u00f3\3\u00f3\3\u00f3\3\u00f4\3\u00f4\3\u00f4\3\u00f4"+ - "\3\u00f4\3\u00f4\3\u00f4\3\u00f4\3\u00f4\3\u00f4\3\u00f5\3\u00f5\3\u00f5"+ - "\3\u00f5\3\u00f5\3\u00f6\3\u00f6\3\u00f6\3\u00f6\3\u00f6\3\u00f7\3\u00f7"+ - "\3\u00f7\3\u00f7\3\u00f7\3\u00f7\3\u00f7\3\u00f7\3\u00f7\3\u00f8\3\u00f8"+ - "\3\u00f8\3\u00f8\3\u00f8\3\u00f8\3\u00f8\3\u00f8\3\u00f8\3\u00f9\3\u00f9"+ - "\3\u00f9\3\u00f9\3\u00f9\3\u00fa\3\u00fa\3\u00fa\3\u00fa\3\u00fa\3\u00fa"+ - "\3\u00fa\3\u00fa\3\u00fa\3\u00fa\3\u00fb\3\u00fb\3\u00fb\3\u00fb\3\u00fb"+ - "\3\u00fb\3\u00fb\3\u00fb\3\u00fb\3\u00fb\3\u00fc\3\u00fc\3\u00fc\3\u00fc"+ - "\3\u00fc\3\u00fc\3\u00fc\3\u00fc\3\u00fd\3\u00fd\3\u00fd\3\u00fd\3\u00fd"+ - "\3\u00fd\3\u00fe\3\u00fe\3\u00fe\3\u00fe\3\u00fe\3\u00fe\3\u00fe\3\u00ff"+ - "\3\u00ff\3\u00ff\3\u00ff\3\u00ff\3\u00ff\3\u00ff\3\u00ff\3\u0100\3\u0100"+ - "\3\u0100\3\u0100\3\u0100\3\u0100\3\u0100\3\u0101\3\u0101\3\u0101\3\u0101"+ - "\3\u0101\3\u0101\3\u0102\3\u0102\3\u0102\3\u0102\3\u0102\3\u0102\3\u0102"+ - "\3\u0103\3\u0103\3\u0103\3\u0103\3\u0104\3\u0104\3\u0104\3\u0104\3\u0104"+ - "\3\u0105\3\u0105\3\u0105\3\u0105\3\u0105\3\u0105\3\u0106\3\u0106\3\u0106"+ - "\3\u0106\3\u0106\3\u0106\3\u0106\3\u0107\3\u0107\3\u0107\3\u0107\3\u0107"+ - "\3\u0108\3\u0108\3\u0108\3\u0108\3\u0108\3\u0108\3\u0109\3\u0109\3\u0109"+ - "\3\u0109\3\u0109\3\u010a\3\u010a\3\u010a\3\u010a\3\u010a\3\u010a\3\u010b"+ - "\3\u010b\3\u010b\3\u010b\3\u010b\3\u010b\3\u010b\3\u010c\3\u010c\3\u010c"+ - "\3\u010c\3\u010c\3\u010d\3\u010d\3\u010d\3\u010d\3\u010d\3\u010e\3\u010e"+ - "\3\u010e\3\u010e\3\u010e\3\u010f\3\u010f\3\u010f\3\u010f\3\u0110\3\u0110"+ - "\3\u0110\5\u0110\u09f2\n\u0110\3\u0111\3\u0111\3\u0111\3\u0111\3\u0112"+ - "\3\u0112\3\u0112\3\u0113\3\u0113\3\u0113\3\u0114\3\u0114\3\u0115\3\u0115"+ - "\3\u0115\3\u0115\5\u0115\u0a04\n\u0115\3\u0116\3\u0116\3\u0117\3\u0117"+ - "\3\u0117\3\u0117\5\u0117\u0a0c\n\u0117\3\u0118\3\u0118\3\u0119\3\u0119"+ - "\3\u011a\3\u011a\3\u011b\3\u011b\3\u011c\3\u011c\3\u011d\3\u011d\3\u011e"+ - "\3\u011e\3\u011f\3\u011f\3\u0120\3\u0120\3\u0120\3\u0121\3\u0121\3\u0122"+ - "\3\u0122\3\u0122\3\u0122\7\u0122\u0a27\n\u0122\f\u0122\16\u0122\u0a2a"+ - "\13\u0122\3\u0122\3\u0122\3\u0122\3\u0122\3\u0122\7\u0122\u0a31\n\u0122"+ - "\f\u0122\16\u0122\u0a34\13\u0122\3\u0122\5\u0122\u0a37\n\u0122\3\u0123"+ - "\6\u0123\u0a3a\n\u0123\r\u0123\16\u0123\u0a3b\3\u0123\3\u0123\3\u0124"+ - "\6\u0124\u0a41\n\u0124\r\u0124\16\u0124\u0a42\3\u0124\3\u0124\3\u0125"+ - "\6\u0125\u0a48\n\u0125\r\u0125\16\u0125\u0a49\3\u0125\3\u0125\3\u0126"+ - "\6\u0126\u0a4f\n\u0126\r\u0126\16\u0126\u0a50\3\u0127\6\u0127\u0a54\n"+ - "\u0127\r\u0127\16\u0127\u0a55\3\u0127\3\u0127\3\u0127\3\u0127\3\u0127"+ - "\3\u0127\5\u0127\u0a5e\n\u0127\3\u0128\3\u0128\3\u0128\3\u0129\6\u0129"+ - "\u0a64\n\u0129\r\u0129\16\u0129\u0a65\3\u0129\5\u0129\u0a69\n\u0129\3"+ - "\u0129\3\u0129\3\u0129\3\u0129\5\u0129\u0a6f\n\u0129\3\u0129\3\u0129\3"+ - "\u0129\5\u0129\u0a74\n\u0129\3\u012a\6\u012a\u0a77\n\u012a\r\u012a\16"+ - "\u012a\u0a78\3\u012a\5\u012a\u0a7c\n\u012a\3\u012a\3\u012a\3\u012a\3\u012a"+ - "\5\u012a\u0a82\n\u012a\3\u012a\3\u012a\3\u012a\5\u012a\u0a87\n\u012a\3"+ - "\u012b\6\u012b\u0a8a\n\u012b\r\u012b\16\u012b\u0a8b\3\u012b\5\u012b\u0a8f"+ - "\n\u012b\3\u012b\3\u012b\3\u012b\3\u012b\3\u012b\5\u012b\u0a96\n\u012b"+ - "\3\u012b\3\u012b\3\u012b\3\u012b\3\u012b\5\u012b\u0a9d\n\u012b\3\u012c"+ - "\3\u012c\3\u012c\6\u012c\u0aa2\n\u012c\r\u012c\16\u012c\u0aa3\3\u012d"+ - "\3\u012d\3\u012d\3\u012d\7\u012d\u0aaa\n\u012d\f\u012d\16\u012d\u0aad"+ - "\13\u012d\3\u012d\3\u012d\3\u012e\6\u012e\u0ab2\n\u012e\r\u012e\16\u012e"+ - "\u0ab3\3\u012e\3\u012e\7\u012e\u0ab8\n\u012e\f\u012e\16\u012e\u0abb\13"+ - "\u012e\3\u012e\3\u012e\6\u012e\u0abf\n\u012e\r\u012e\16\u012e\u0ac0\5"+ - "\u012e\u0ac3\n\u012e\3\u012f\3\u012f\5\u012f\u0ac7\n\u012f\3\u012f\6\u012f"+ - "\u0aca\n\u012f\r\u012f\16\u012f\u0acb\3\u0130\3\u0130\3\u0131\3\u0131"+ - "\3\u0132\3\u0132\3\u0132\3\u0132\3\u0132\3\u0132\7\u0132\u0ad8\n\u0132"+ - "\f\u0132\16\u0132\u0adb\13\u0132\3\u0132\5\u0132\u0ade\n\u0132\3\u0132"+ - "\5\u0132\u0ae1\n\u0132\3\u0132\3\u0132\3\u0133\3\u0133\3\u0133\3\u0133"+ - "\3\u0133\3\u0133\7\u0133\u0aeb\n\u0133\f\u0133\16\u0133\u0aee\13\u0133"+ - "\3\u0133\3\u0133\3\u0133\3\u0133\5\u0133\u0af4\n\u0133\3\u0133\3\u0133"+ - "\3\u0134\6\u0134\u0af9\n\u0134\r\u0134\16\u0134\u0afa\3\u0134\3\u0134"+ - "\3\u0135\3\u0135\3\u0aec\2\u0136\3\3\5\4\7\5\t\6\13\7\r\b\17\t\21\n\23"+ - "\13\25\f\27\r\31\16\33\17\35\20\37\21!\22#\23%\24\'\25)\26+\27-\30/\31"+ - "\61\32\63\33\65\34\67\359\36;\37= ?!A\"C#E$G%I&K\'M(O)Q*S+U,W-Y.[/]\60"+ - "_\61a\62c\63e\64g\65i\66k\67m8o9q:s;u{?}@\177A\u0081B\u0083C\u0085"+ - "D\u0087E\u0089F\u008bG\u008dH\u008fI\u0091J\u0093K\u0095L\u0097M\u0099"+ - "N\u009bO\u009dP\u009fQ\u00a1R\u00a3S\u00a5T\u00a7U\u00a9V\u00abW\u00ad"+ - "X\u00afY\u00b1Z\u00b3[\u00b5\\\u00b7]\u00b9^\u00bb_\u00bd`\u00bfa\u00c1"+ - "b\u00c3c\u00c5d\u00c7e\u00c9f\u00cbg\u00cdh\u00cfi\u00d1j\u00d3k\u00d5"+ - "l\u00d7m\u00d9n\u00dbo\u00ddp\u00dfq\u00e1r\u00e3s\u00e5t\u00e7u\u00e9"+ - "v\u00ebw\u00edx\u00efy\u00f1z\u00f3{\u00f5|\u00f7}\u00f9~\u00fb\177\u00fd"+ - "\u0080\u00ff\u0081\u0101\u0082\u0103\u0083\u0105\u0084\u0107\u0085\u0109"+ - "\u0086\u010b\u0087\u010d\u0088\u010f\u0089\u0111\u008a\u0113\u008b\u0115"+ - "\u008c\u0117\u008d\u0119\u008e\u011b\u008f\u011d\u0090\u011f\u0091\u0121"+ - "\u0092\u0123\u0093\u0125\u0094\u0127\u0095\u0129\u0096\u012b\u0097\u012d"+ - "\u0098\u012f\u0099\u0131\u009a\u0133\u009b\u0135\u009c\u0137\u009d\u0139"+ - "\u009e\u013b\u009f\u013d\u00a0\u013f\u00a1\u0141\u00a2\u0143\u00a3\u0145"+ - "\u00a4\u0147\u00a5\u0149\u00a6\u014b\u00a7\u014d\u00a8\u014f\u00a9\u0151"+ - "\u00aa\u0153\u00ab\u0155\u00ac\u0157\u00ad\u0159\u00ae\u015b\u00af\u015d"+ - "\u00b0\u015f\u00b1\u0161\u00b2\u0163\u00b3\u0165\u00b4\u0167\u00b5\u0169"+ - "\u00b6\u016b\u00b7\u016d\u00b8\u016f\u00b9\u0171\u00ba\u0173\u00bb\u0175"+ - "\u00bc\u0177\u00bd\u0179\u00be\u017b\u00bf\u017d\u00c0\u017f\u00c1\u0181"+ - "\u00c2\u0183\u00c3\u0185\u00c4\u0187\u00c5\u0189\u00c6\u018b\u00c7\u018d"+ - "\u00c8\u018f\u00c9\u0191\u00ca\u0193\u00cb\u0195\u00cc\u0197\u00cd\u0199"+ - "\u00ce\u019b\u00cf\u019d\u00d0\u019f\u00d1\u01a1\u00d2\u01a3\u00d3\u01a5"+ - "\u00d4\u01a7\u00d5\u01a9\u00d6\u01ab\u00d7\u01ad\u00d8\u01af\u00d9\u01b1"+ - "\u00da\u01b3\u00db\u01b5\u00dc\u01b7\u00dd\u01b9\u00de\u01bb\u00df\u01bd"+ - "\u00e0\u01bf\u00e1\u01c1\u00e2\u01c3\u00e3\u01c5\u00e4\u01c7\u00e5\u01c9"+ - "\u00e6\u01cb\u00e7\u01cd\u00e8\u01cf\u00e9\u01d1\u00ea\u01d3\u00eb\u01d5"+ - "\u00ec\u01d7\u00ed\u01d9\u00ee\u01db\u00ef\u01dd\u00f0\u01df\u00f1\u01e1"+ - "\u00f2\u01e3\u00f3\u01e5\u00f4\u01e7\u00f5\u01e9\u00f6\u01eb\u00f7\u01ed"+ - "\u00f8\u01ef\u00f9\u01f1\u00fa\u01f3\u00fb\u01f5\u00fc\u01f7\u00fd\u01f9"+ - "\u00fe\u01fb\u00ff\u01fd\u0100\u01ff\u0101\u0201\u0102\u0203\u0103\u0205"+ - "\u0104\u0207\u0105\u0209\u0106\u020b\u0107\u020d\u0108\u020f\u0109\u0211"+ - "\u010a\u0213\u010b\u0215\u010c\u0217\u010d\u0219\u010e\u021b\u010f\u021d"+ - "\u0110\u021f\u0111\u0221\u0112\u0223\u0113\u0225\u0114\u0227\u0115\u0229"+ - "\u0116\u022b\u0117\u022d\u0118\u022f\u0119\u0231\u011a\u0233\u011b\u0235"+ - "\u011c\u0237\u011d\u0239\u011e\u023b\u011f\u023d\u0120\u023f\u0121\u0241"+ - "\u0122\u0243\u0123\u0245\u0124\u0247\u0125\u0249\u0126\u024b\u0127\u024d"+ - "\u0128\u024f\u0129\u0251\u012a\u0253\u012b\u0255\u012c\u0257\u012d\u0259"+ - "\u012e\u025b\2\u025d\2\u025f\2\u0261\2\u0263\u012f\u0265\u0130\u0267\u0131"+ - "\u0269\u0132\3\2\n\4\2))^^\4\2$$^^\3\2bb\4\2--//\3\2\62;\3\2C\\\4\2\f"+ - "\f\17\17\5\2\13\f\17\17\"\"\2\u0b2c\2\3\3\2\2\2\2\5\3\2\2\2\2\7\3\2\2"+ - "\2\2\t\3\2\2\2\2\13\3\2\2\2\2\r\3\2\2\2\2\17\3\2\2\2\2\21\3\2\2\2\2\23"+ - "\3\2\2\2\2\25\3\2\2\2\2\27\3\2\2\2\2\31\3\2\2\2\2\33\3\2\2\2\2\35\3\2"+ - "\2\2\2\37\3\2\2\2\2!\3\2\2\2\2#\3\2\2\2\2%\3\2\2\2\2\'\3\2\2\2\2)\3\2"+ - "\2\2\2+\3\2\2\2\2-\3\2\2\2\2/\3\2\2\2\2\61\3\2\2\2\2\63\3\2\2\2\2\65\3"+ - "\2\2\2\2\67\3\2\2\2\29\3\2\2\2\2;\3\2\2\2\2=\3\2\2\2\2?\3\2\2\2\2A\3\2"+ - "\2\2\2C\3\2\2\2\2E\3\2\2\2\2G\3\2\2\2\2I\3\2\2\2\2K\3\2\2\2\2M\3\2\2\2"+ - "\2O\3\2\2\2\2Q\3\2\2\2\2S\3\2\2\2\2U\3\2\2\2\2W\3\2\2\2\2Y\3\2\2\2\2["+ - "\3\2\2\2\2]\3\2\2\2\2_\3\2\2\2\2a\3\2\2\2\2c\3\2\2\2\2e\3\2\2\2\2g\3\2"+ - "\2\2\2i\3\2\2\2\2k\3\2\2\2\2m\3\2\2\2\2o\3\2\2\2\2q\3\2\2\2\2s\3\2\2\2"+ - "\2u\3\2\2\2\2w\3\2\2\2\2y\3\2\2\2\2{\3\2\2\2\2}\3\2\2\2\2\177\3\2\2\2"+ - "\2\u0081\3\2\2\2\2\u0083\3\2\2\2\2\u0085\3\2\2\2\2\u0087\3\2\2\2\2\u0089"+ - "\3\2\2\2\2\u008b\3\2\2\2\2\u008d\3\2\2\2\2\u008f\3\2\2\2\2\u0091\3\2\2"+ - "\2\2\u0093\3\2\2\2\2\u0095\3\2\2\2\2\u0097\3\2\2\2\2\u0099\3\2\2\2\2\u009b"+ - "\3\2\2\2\2\u009d\3\2\2\2\2\u009f\3\2\2\2\2\u00a1\3\2\2\2\2\u00a3\3\2\2"+ - "\2\2\u00a5\3\2\2\2\2\u00a7\3\2\2\2\2\u00a9\3\2\2\2\2\u00ab\3\2\2\2\2\u00ad"+ - "\3\2\2\2\2\u00af\3\2\2\2\2\u00b1\3\2\2\2\2\u00b3\3\2\2\2\2\u00b5\3\2\2"+ - "\2\2\u00b7\3\2\2\2\2\u00b9\3\2\2\2\2\u00bb\3\2\2\2\2\u00bd\3\2\2\2\2\u00bf"+ - "\3\2\2\2\2\u00c1\3\2\2\2\2\u00c3\3\2\2\2\2\u00c5\3\2\2\2\2\u00c7\3\2\2"+ - "\2\2\u00c9\3\2\2\2\2\u00cb\3\2\2\2\2\u00cd\3\2\2\2\2\u00cf\3\2\2\2\2\u00d1"+ - "\3\2\2\2\2\u00d3\3\2\2\2\2\u00d5\3\2\2\2\2\u00d7\3\2\2\2\2\u00d9\3\2\2"+ - "\2\2\u00db\3\2\2\2\2\u00dd\3\2\2\2\2\u00df\3\2\2\2\2\u00e1\3\2\2\2\2\u00e3"+ - "\3\2\2\2\2\u00e5\3\2\2\2\2\u00e7\3\2\2\2\2\u00e9\3\2\2\2\2\u00eb\3\2\2"+ - "\2\2\u00ed\3\2\2\2\2\u00ef\3\2\2\2\2\u00f1\3\2\2\2\2\u00f3\3\2\2\2\2\u00f5"+ - "\3\2\2\2\2\u00f7\3\2\2\2\2\u00f9\3\2\2\2\2\u00fb\3\2\2\2\2\u00fd\3\2\2"+ - "\2\2\u00ff\3\2\2\2\2\u0101\3\2\2\2\2\u0103\3\2\2\2\2\u0105\3\2\2\2\2\u0107"+ - "\3\2\2\2\2\u0109\3\2\2\2\2\u010b\3\2\2\2\2\u010d\3\2\2\2\2\u010f\3\2\2"+ - "\2\2\u0111\3\2\2\2\2\u0113\3\2\2\2\2\u0115\3\2\2\2\2\u0117\3\2\2\2\2\u0119"+ - "\3\2\2\2\2\u011b\3\2\2\2\2\u011d\3\2\2\2\2\u011f\3\2\2\2\2\u0121\3\2\2"+ - "\2\2\u0123\3\2\2\2\2\u0125\3\2\2\2\2\u0127\3\2\2\2\2\u0129\3\2\2\2\2\u012b"+ - "\3\2\2\2\2\u012d\3\2\2\2\2\u012f\3\2\2\2\2\u0131\3\2\2\2\2\u0133\3\2\2"+ - "\2\2\u0135\3\2\2\2\2\u0137\3\2\2\2\2\u0139\3\2\2\2\2\u013b\3\2\2\2\2\u013d"+ - "\3\2\2\2\2\u013f\3\2\2\2\2\u0141\3\2\2\2\2\u0143\3\2\2\2\2\u0145\3\2\2"+ - "\2\2\u0147\3\2\2\2\2\u0149\3\2\2\2\2\u014b\3\2\2\2\2\u014d\3\2\2\2\2\u014f"+ - "\3\2\2\2\2\u0151\3\2\2\2\2\u0153\3\2\2\2\2\u0155\3\2\2\2\2\u0157\3\2\2"+ - "\2\2\u0159\3\2\2\2\2\u015b\3\2\2\2\2\u015d\3\2\2\2\2\u015f\3\2\2\2\2\u0161"+ - "\3\2\2\2\2\u0163\3\2\2\2\2\u0165\3\2\2\2\2\u0167\3\2\2\2\2\u0169\3\2\2"+ - "\2\2\u016b\3\2\2\2\2\u016d\3\2\2\2\2\u016f\3\2\2\2\2\u0171\3\2\2\2\2\u0173"+ - "\3\2\2\2\2\u0175\3\2\2\2\2\u0177\3\2\2\2\2\u0179\3\2\2\2\2\u017b\3\2\2"+ - "\2\2\u017d\3\2\2\2\2\u017f\3\2\2\2\2\u0181\3\2\2\2\2\u0183\3\2\2\2\2\u0185"+ - "\3\2\2\2\2\u0187\3\2\2\2\2\u0189\3\2\2\2\2\u018b\3\2\2\2\2\u018d\3\2\2"+ - "\2\2\u018f\3\2\2\2\2\u0191\3\2\2\2\2\u0193\3\2\2\2\2\u0195\3\2\2\2\2\u0197"+ - "\3\2\2\2\2\u0199\3\2\2\2\2\u019b\3\2\2\2\2\u019d\3\2\2\2\2\u019f\3\2\2"+ - "\2\2\u01a1\3\2\2\2\2\u01a3\3\2\2\2\2\u01a5\3\2\2\2\2\u01a7\3\2\2\2\2\u01a9"+ - "\3\2\2\2\2\u01ab\3\2\2\2\2\u01ad\3\2\2\2\2\u01af\3\2\2\2\2\u01b1\3\2\2"+ - "\2\2\u01b3\3\2\2\2\2\u01b5\3\2\2\2\2\u01b7\3\2\2\2\2\u01b9\3\2\2\2\2\u01bb"+ - "\3\2\2\2\2\u01bd\3\2\2\2\2\u01bf\3\2\2\2\2\u01c1\3\2\2\2\2\u01c3\3\2\2"+ - "\2\2\u01c5\3\2\2\2\2\u01c7\3\2\2\2\2\u01c9\3\2\2\2\2\u01cb\3\2\2\2\2\u01cd"+ - "\3\2\2\2\2\u01cf\3\2\2\2\2\u01d1\3\2\2\2\2\u01d3\3\2\2\2\2\u01d5\3\2\2"+ - "\2\2\u01d7\3\2\2\2\2\u01d9\3\2\2\2\2\u01db\3\2\2\2\2\u01dd\3\2\2\2\2\u01df"+ - "\3\2\2\2\2\u01e1\3\2\2\2\2\u01e3\3\2\2\2\2\u01e5\3\2\2\2\2\u01e7\3\2\2"+ - "\2\2\u01e9\3\2\2\2\2\u01eb\3\2\2\2\2\u01ed\3\2\2\2\2\u01ef\3\2\2\2\2\u01f1"+ - "\3\2\2\2\2\u01f3\3\2\2\2\2\u01f5\3\2\2\2\2\u01f7\3\2\2\2\2\u01f9\3\2\2"+ - "\2\2\u01fb\3\2\2\2\2\u01fd\3\2\2\2\2\u01ff\3\2\2\2\2\u0201\3\2\2\2\2\u0203"+ - "\3\2\2\2\2\u0205\3\2\2\2\2\u0207\3\2\2\2\2\u0209\3\2\2\2\2\u020b\3\2\2"+ - "\2\2\u020d\3\2\2\2\2\u020f\3\2\2\2\2\u0211\3\2\2\2\2\u0213\3\2\2\2\2\u0215"+ - "\3\2\2\2\2\u0217\3\2\2\2\2\u0219\3\2\2\2\2\u021b\3\2\2\2\2\u021d\3\2\2"+ - "\2\2\u021f\3\2\2\2\2\u0221\3\2\2\2\2\u0223\3\2\2\2\2\u0225\3\2\2\2\2\u0227"+ - "\3\2\2\2\2\u0229\3\2\2\2\2\u022b\3\2\2\2\2\u022d\3\2\2\2\2\u022f\3\2\2"+ - "\2\2\u0231\3\2\2\2\2\u0233\3\2\2\2\2\u0235\3\2\2\2\2\u0237\3\2\2\2\2\u0239"+ - "\3\2\2\2\2\u023b\3\2\2\2\2\u023d\3\2\2\2\2\u023f\3\2\2\2\2\u0241\3\2\2"+ - "\2\2\u0243\3\2\2\2\2\u0245\3\2\2\2\2\u0247\3\2\2\2\2\u0249\3\2\2\2\2\u024b"+ - "\3\2\2\2\2\u024d\3\2\2\2\2\u024f\3\2\2\2\2\u0251\3\2\2\2\2\u0253\3\2\2"+ - "\2\2\u0255\3\2\2\2\2\u0257\3\2\2\2\2\u0259\3\2\2\2\2\u0263\3\2\2\2\2\u0265"+ - "\3\2\2\2\2\u0267\3\2\2\2\2\u0269\3\2\2\2\3\u026b\3\2\2\2\5\u026d\3\2\2"+ - "\2\7\u026f\3\2\2\2\t\u0271\3\2\2\2\13\u0273\3\2\2\2\r\u0275\3\2\2\2\17"+ - "\u0279\3\2\2\2\21\u027c\3\2\2\2\23\u027f\3\2\2\2\25\u0281\3\2\2\2\27\u0283"+ - "\3\2\2\2\31\u0285\3\2\2\2\33\u0289\3\2\2\2\35\u028f\3\2\2\2\37\u0293\3"+ - "\2\2\2!\u0299\3\2\2\2#\u02a1\3\2\2\2%\u02a5\3\2\2\2\'\u02aa\3\2\2\2)\u02ae"+ - "\3\2\2\2+\u02b6\3\2\2\2-\u02bc\3\2\2\2/\u02bf\3\2\2\2\61\u02c3\3\2\2\2"+ - "\63\u02c6\3\2\2\2\65\u02d4\3\2\2\2\67\u02dc\3\2\2\29\u02e1\3\2\2\2;\u02e8"+ - "\3\2\2\2=\u02f0\3\2\2\2?\u02f3\3\2\2\2A\u02f9\3\2\2\2C\u0301\3\2\2\2E"+ - "\u0306\3\2\2\2G\u030b\3\2\2\2I\u0312\3\2\2\2K\u0318\3\2\2\2M\u031e\3\2"+ - "\2\2O\u0326\3\2\2\2Q\u0330\3\2\2\2S\u0338\3\2\2\2U\u0340\3\2\2\2W\u034b"+ - "\3\2\2\2Y\u0352\3\2\2\2[\u035a\3\2\2\2]\u0362\3\2\2\2_\u0369\3\2\2\2a"+ - "\u0371\3\2\2\2c\u037d\3\2\2\2e\u0385\3\2\2\2g\u0391\3\2\2\2i\u039c\3\2"+ - "\2\2k\u03a1\3\2\2\2m\u03a8\3\2\2\2o\u03ae\3\2\2\2q\u03b3\3\2\2\2s\u03bb"+ - "\3\2\2\2u\u03c8\3\2\2\2w\u03d5\3\2\2\2y\u03e7\3\2\2\2{\u03f4\3\2\2\2}"+ - "\u03f8\3\2\2\2\177\u03fd\3\2\2\2\u0081\u0416\3\2\2\2\u0083\u0418\3\2\2"+ - "\2\u0085\u0425\3\2\2\2\u0087\u042d\3\2\2\2\u0089\u0434\3\2\2\2\u008b\u043e"+ - "\3\2\2\2\u008d\u0443\3\2\2\2\u008f\u044c\3\2\2\2\u0091\u0450\3\2\2\2\u0093"+ - "\u045c\3\2\2\2\u0095\u0466\3\2\2\2\u0097\u046f\3\2\2\2\u0099\u047a\3\2"+ - "\2\2\u009b\u047e\3\2\2\2\u009d\u0483\3\2\2\2\u009f\u0488\3\2\2\2\u00a1"+ - "\u048c\3\2\2\2\u00a3\u0493\3\2\2\2\u00a5\u049b\3\2\2\2\u00a7\u04a2\3\2"+ - "\2\2\u00a9\u04ab\3\2\2\2\u00ab\u04b2\3\2\2\2\u00ad\u04ba\3\2\2\2\u00af"+ - "\u04c1\3\2\2\2\u00b1\u04ca\3\2\2\2\u00b3\u04d3\3\2\2\2\u00b5\u04db\3\2"+ - "\2\2\u00b7\u04e1\3\2\2\2\u00b9\u04e7\3\2\2\2\u00bb\u04ee\3\2\2\2\u00bd"+ - "\u04f5\3\2\2\2\u00bf\u0500\3\2\2\2\u00c1\u0506\3\2\2\2\u00c3\u0510\3\2"+ - "\2\2\u00c5\u0514\3\2\2\2\u00c7\u051c\3\2\2\2\u00c9\u0523\3\2\2\2\u00cb"+ - "\u052d\3\2\2\2\u00cd\u0532\3\2\2\2\u00cf\u0537\3\2\2\2\u00d1\u0540\3\2"+ - "\2\2\u00d3\u054a\3\2\2\2\u00d5\u0551\3\2\2\2\u00d7\u0557\3\2\2\2\u00d9"+ - "\u055d\3\2\2\2\u00db\u0566\3\2\2\2\u00dd\u056d\3\2\2\2\u00df\u0572\3\2"+ - "\2\2\u00e1\u0575\3\2\2\2\u00e3\u057c\3\2\2\2\u00e5\u0583\3\2\2\2\u00e7"+ - "\u0586\3\2\2\2\u00e9\u058c\3\2\2\2\u00eb\u0594\3\2\2\2\u00ed\u059a\3\2"+ - "\2\2\u00ef\u05a1\3\2\2\2\u00f1\u05ad\3\2\2\2\u00f3\u05b4\3\2\2\2\u00f5"+ - "\u05be\3\2\2\2\u00f7\u05c7\3\2\2\2\u00f9\u05cc\3\2\2\2\u00fb\u05cf\3\2"+ - "\2\2\u00fd\u05d5\3\2\2\2\u00ff\u05da\3\2\2\2\u0101\u05df\3\2\2\2\u0103"+ - "\u05e4\3\2\2\2\u0105\u05ec\3\2\2\2\u0107\u05f1\3\2\2\2\u0109\u05f9\3\2"+ - "\2\2\u010b\u05fe\3\2\2\2\u010d\u0603\3\2\2\2\u010f\u0609\3\2\2\2\u0111"+ - "\u060f\3\2\2\2\u0113\u0614\3\2\2\2\u0115\u0619\3\2\2\2\u0117\u061f\3\2"+ - "\2\2\u0119\u0628\3\2\2\2\u011b\u062d\3\2\2\2\u011d\u0633\3\2\2\2\u011f"+ - "\u063b\3\2\2\2\u0121\u0641\3\2\2\2\u0123\u0645\3\2\2\2\u0125\u064d\3\2"+ - "\2\2\u0127\u0653\3\2\2\2\u0129\u065a\3\2\2\2\u012b\u0660\3\2\2\2\u012d"+ - "\u0665\3\2\2\2\u012f\u066f\3\2\2\2\u0131\u067a\3\2\2\2\u0133\u0682\3\2"+ - "\2\2\u0135\u0689\3\2\2\2\u0137\u068b\3\2\2\2\u0139\u0690\3\2\2\2\u013b"+ - "\u0696\3\2\2\2\u013d\u0699\3\2\2\2\u013f\u069c\3\2\2\2\u0141\u06a1\3\2"+ - "\2\2\u0143\u06a8\3\2\2\2\u0145\u06b0\3\2\2\2\u0147\u06b3\3\2\2\2\u0149"+ - "\u06b9\3\2\2\2\u014b\u06bd\3\2\2\2\u014d\u06c3\3\2\2\2\u014f\u06d0\3\2"+ - "\2\2\u0151\u06d5\3\2\2\2\u0153\u06de\3\2\2\2\u0155\u06e6\3\2\2\2\u0157"+ - "\u06f0\3\2\2\2\u0159\u06fa\3\2\2\2\u015b\u0706\3\2\2\2\u015d\u0711\3\2"+ - "\2\2\u015f\u0719\3\2\2\2\u0161\u071f\3\2\2\2\u0163\u0727\3\2\2\2\u0165"+ - "\u0730\3\2\2\2\u0167\u073a\3\2\2\2\u0169\u0742\3\2\2\2\u016b\u074d\3\2"+ - "\2\2\u016d\u0758\3\2\2\2\u016f\u075e\3\2\2\2\u0171\u0764\3\2\2\2\u0173"+ - "\u076a\3\2\2\2\u0175\u0777\3\2\2\2\u0177\u0784\3\2\2\2\u0179\u078c\3\2"+ - "\2\2\u017b\u0793\3\2\2\2\u017d\u079e\3\2\2\2\u017f\u07a6\3\2\2\2\u0181"+ - "\u07ad\3\2\2\2\u0183\u07b4\3\2\2\2\u0185\u07bc\3\2\2\2\u0187\u07c2\3\2"+ - "\2\2\u0189\u07ca\3\2\2\2\u018b\u07d3\3\2\2\2\u018d\u07da\3\2\2\2\u018f"+ - "\u07eb\3\2\2\2\u0191\u07ed\3\2\2\2\u0193\u07f2\3\2\2\2\u0195\u07f8\3\2"+ - "\2\2\u0197\u0801\3\2\2\2\u0199\u0808\3\2\2\2\u019b\u080c\3\2\2\2\u019d"+ - "\u0811\3\2\2\2\u019f\u0818\3\2\2\2\u01a1\u081f\3\2\2\2\u01a3\u0826\3\2"+ - "\2\2\u01a5\u082b\3\2\2\2\u01a7\u0835\3\2\2\2\u01a9\u083b\3\2\2\2\u01ab"+ - "\u084b\3\2\2\2\u01ad\u0858\3\2\2\2\u01af\u085c\3\2\2\2\u01b1\u0862\3\2"+ - "\2\2\u01b3\u0867\3\2\2\2\u01b5\u086c\3\2\2\2\u01b7\u0873\3\2\2\2\u01b9"+ - "\u0878\3\2\2\2\u01bb\u087d\3\2\2\2\u01bd\u0884\3\2\2\2\u01bf\u088a\3\2"+ - "\2\2\u01c1\u0895\3\2\2\2\u01c3\u089c\3\2\2\2\u01c5\u08a5\3\2\2\2\u01c7"+ - "\u08ac\3\2\2\2\u01c9\u08b3\3\2\2\2\u01cb\u08bd\3\2\2\2\u01cd\u08c2\3\2"+ - "\2\2\u01cf\u08c8\3\2\2\2\u01d1\u08cf\3\2\2\2\u01d3\u08db\3\2\2\2\u01d5"+ - "\u08f6\3\2\2\2\u01d7\u08f8\3\2\2\2\u01d9\u0903\3\2\2\2\u01db\u0908\3\2"+ - "\2\2\u01dd\u090d\3\2\2\2\u01df\u0910\3\2\2\2\u01e1\u0916\3\2\2\2\u01e3"+ - "\u091f\3\2\2\2\u01e5\u092b\3\2\2\2\u01e7\u0938\3\2\2\2\u01e9\u0942\3\2"+ - "\2\2\u01eb\u0947\3\2\2\2\u01ed\u094c\3\2\2\2\u01ef\u0955\3\2\2\2\u01f1"+ - "\u095e\3\2\2\2\u01f3\u0963\3\2\2\2\u01f5\u096d\3\2\2\2\u01f7\u0977\3\2"+ - "\2\2\u01f9\u097f\3\2\2\2\u01fb\u0985\3\2\2\2\u01fd\u098c\3\2\2\2\u01ff"+ - "\u0994\3\2\2\2\u0201\u099b\3\2\2\2\u0203\u09a1\3\2\2\2\u0205\u09a8\3\2"+ - "\2\2\u0207\u09ac\3\2\2\2\u0209\u09b1\3\2\2\2\u020b\u09b7\3\2\2\2\u020d"+ - "\u09be\3\2\2\2\u020f\u09c3\3\2\2\2\u0211\u09c9\3\2\2\2\u0213\u09ce\3\2"+ - "\2\2\u0215\u09d4\3\2\2\2\u0217\u09db\3\2\2\2\u0219\u09e0\3\2\2\2\u021b"+ - "\u09e5\3\2\2\2\u021d\u09ea\3\2\2\2\u021f\u09f1\3\2\2\2\u0221\u09f3\3\2"+ - "\2\2\u0223\u09f7\3\2\2\2\u0225\u09fa\3\2\2\2\u0227\u09fd\3\2\2\2\u0229"+ - "\u0a03\3\2\2\2\u022b\u0a05\3\2\2\2\u022d\u0a0b\3\2\2\2\u022f\u0a0d\3\2"+ - "\2\2\u0231\u0a0f\3\2\2\2\u0233\u0a11\3\2\2\2\u0235\u0a13\3\2\2\2\u0237"+ - "\u0a15\3\2\2\2\u0239\u0a17\3\2\2\2\u023b\u0a19\3\2\2\2\u023d\u0a1b\3\2"+ - "\2\2\u023f\u0a1d\3\2\2\2\u0241\u0a20\3\2\2\2\u0243\u0a36\3\2\2\2\u0245"+ - "\u0a39\3\2\2\2\u0247\u0a40\3\2\2\2\u0249\u0a47\3\2\2\2\u024b\u0a4e\3\2"+ - "\2\2\u024d\u0a5d\3\2\2\2\u024f\u0a5f\3\2\2\2\u0251\u0a73\3\2\2\2\u0253"+ - "\u0a86\3\2\2\2\u0255\u0a9c\3\2\2\2\u0257\u0aa1\3\2\2\2\u0259\u0aa5\3\2"+ - "\2\2\u025b\u0ac2\3\2\2\2\u025d\u0ac4\3\2\2\2\u025f\u0acd\3\2\2\2\u0261"+ - "\u0acf\3\2\2\2\u0263\u0ad1\3\2\2\2\u0265\u0ae4\3\2\2\2\u0267\u0af8\3\2"+ - "\2\2\u0269\u0afe\3\2\2\2\u026b\u026c\7=\2\2\u026c\4\3\2\2\2\u026d\u026e"+ - "\7*\2\2\u026e\6\3\2\2\2\u026f\u0270\7.\2\2\u0270\b\3\2\2\2\u0271\u0272"+ - "\7+\2\2\u0272\n\3\2\2\2\u0273\u0274\7\60\2\2\u0274\f\3\2\2\2\u0275\u0276"+ - "\7\61\2\2\u0276\u0277\7,\2\2\u0277\u0278\7-\2\2\u0278\16\3\2\2\2\u0279"+ - "\u027a\7,\2\2\u027a\u027b\7\61\2\2\u027b\20\3\2\2\2\u027c\u027d\7/\2\2"+ - "\u027d\u027e\7@\2\2\u027e\22\3\2\2\2\u027f\u0280\7]\2\2\u0280\24\3\2\2"+ - "\2\u0281\u0282\7_\2\2\u0282\26\3\2\2\2\u0283\u0284\7<\2\2\u0284\30\3\2"+ - "\2\2\u0285\u0286\7C\2\2\u0286\u0287\7F\2\2\u0287\u0288\7F\2\2\u0288\32"+ - "\3\2\2\2\u0289\u028a\7C\2\2\u028a\u028b\7H\2\2\u028b\u028c\7V\2\2\u028c"+ - "\u028d\7G\2\2\u028d\u028e\7T\2\2\u028e\34\3\2\2\2\u028f\u0290\7C\2\2\u0290"+ - "\u0291\7N\2\2\u0291\u0292\7N\2\2\u0292\36\3\2\2\2\u0293\u0294\7C\2\2\u0294"+ - "\u0295\7N\2\2\u0295\u0296\7V\2\2\u0296\u0297\7G\2\2\u0297\u0298\7T\2\2"+ - "\u0298 \3\2\2\2\u0299\u029a\7C\2\2\u029a\u029b\7P\2\2\u029b\u029c\7C\2"+ - "\2\u029c\u029d\7N\2\2\u029d\u029e\7[\2\2\u029e\u029f\7\\\2\2\u029f\u02a0"+ - "\7G\2\2\u02a0\"\3\2\2\2\u02a1\u02a2\7C\2\2\u02a2\u02a3\7P\2\2\u02a3\u02a4"+ - "\7F\2\2\u02a4$\3\2\2\2\u02a5\u02a6\7C\2\2\u02a6\u02a7\7P\2\2\u02a7\u02a8"+ - "\7V\2\2\u02a8\u02a9\7K\2\2\u02a9&\3\2\2\2\u02aa\u02ab\7C\2\2\u02ab\u02ac"+ - "\7P\2\2\u02ac\u02ad\7[\2\2\u02ad(\3\2\2\2\u02ae\u02af\7C\2\2\u02af\u02b0"+ - "\7T\2\2\u02b0\u02b1\7E\2\2\u02b1\u02b2\7J\2\2\u02b2\u02b3\7K\2\2\u02b3"+ - "\u02b4\7X\2\2\u02b4\u02b5\7G\2\2\u02b5*\3\2\2\2\u02b6\u02b7\7C\2\2\u02b7"+ - "\u02b8\7T\2\2\u02b8\u02b9\7T\2\2\u02b9\u02ba\7C\2\2\u02ba\u02bb\7[\2\2"+ - "\u02bb,\3\2\2\2\u02bc\u02bd\7C\2\2\u02bd\u02be\7U\2\2\u02be.\3\2\2\2\u02bf"+ - "\u02c0\7C\2\2\u02c0\u02c1\7U\2\2\u02c1\u02c2\7E\2\2\u02c2\60\3\2\2\2\u02c3"+ - "\u02c4\7C\2\2\u02c4\u02c5\7V\2\2\u02c5\62\3\2\2\2\u02c6\u02c7\7C\2\2\u02c7"+ - "\u02c8\7W\2\2\u02c8\u02c9\7V\2\2\u02c9\u02ca\7J\2\2\u02ca\u02cb\7Q\2\2"+ - "\u02cb\u02cc\7T\2\2\u02cc\u02cd\7K\2\2\u02cd\u02ce\7\\\2\2\u02ce\u02cf"+ - "\7C\2\2\u02cf\u02d0\7V\2\2\u02d0\u02d1\7K\2\2\u02d1\u02d2\7Q\2\2\u02d2"+ - "\u02d3\7P\2\2\u02d3\64\3\2\2\2\u02d4\u02d5\7D\2\2\u02d5\u02d6\7G\2\2\u02d6"+ - "\u02d7\7V\2\2\u02d7\u02d8\7Y\2\2\u02d8\u02d9\7G\2\2\u02d9\u02da\7G\2\2"+ - "\u02da\u02db\7P\2\2\u02db\66\3\2\2\2\u02dc\u02dd\7D\2\2\u02dd\u02de\7"+ - "Q\2\2\u02de\u02df\7V\2\2\u02df\u02e0\7J\2\2\u02e08\3\2\2\2\u02e1\u02e2"+ - "\7D\2\2\u02e2\u02e3\7W\2\2\u02e3\u02e4\7E\2\2\u02e4\u02e5\7M\2\2\u02e5"+ - "\u02e6\7G\2\2\u02e6\u02e7\7V\2\2\u02e7:\3\2\2\2\u02e8\u02e9\7D\2\2\u02e9"+ - "\u02ea\7W\2\2\u02ea\u02eb\7E\2\2\u02eb\u02ec\7M\2\2\u02ec\u02ed\7G\2\2"+ - "\u02ed\u02ee\7V\2\2\u02ee\u02ef\7U\2\2\u02ef<\3\2\2\2\u02f0\u02f1\7D\2"+ - "\2\u02f1\u02f2\7[\2\2\u02f2>\3\2\2\2\u02f3\u02f4\7E\2\2\u02f4\u02f5\7"+ - "C\2\2\u02f5\u02f6\7E\2\2\u02f6\u02f7\7J\2\2\u02f7\u02f8\7G\2\2\u02f8@"+ - "\3\2\2\2\u02f9\u02fa\7E\2\2\u02fa\u02fb\7C\2\2\u02fb\u02fc\7U\2\2\u02fc"+ - "\u02fd\7E\2\2\u02fd\u02fe\7C\2\2\u02fe\u02ff\7F\2\2\u02ff\u0300\7G\2\2"+ - "\u0300B\3\2\2\2\u0301\u0302\7E\2\2\u0302\u0303\7C\2\2\u0303\u0304\7U\2"+ - "\2\u0304\u0305\7G\2\2\u0305D\3\2\2\2\u0306\u0307\7E\2\2\u0307\u0308\7"+ - "C\2\2\u0308\u0309\7U\2\2\u0309\u030a\7V\2\2\u030aF\3\2\2\2\u030b\u030c"+ - "\7E\2\2\u030c\u030d\7J\2\2\u030d\u030e\7C\2\2\u030e\u030f\7P\2\2\u030f"+ - "\u0310\7I\2\2\u0310\u0311\7G\2\2\u0311H\3\2\2\2\u0312\u0313\7E\2\2\u0313"+ - "\u0314\7J\2\2\u0314\u0315\7G\2\2\u0315\u0316\7E\2\2\u0316\u0317\7M\2\2"+ - "\u0317J\3\2\2\2\u0318\u0319\7E\2\2\u0319\u031a\7N\2\2\u031a\u031b\7G\2"+ - "\2\u031b\u031c\7C\2\2\u031c\u031d\7T\2\2\u031dL\3\2\2\2\u031e\u031f\7"+ - "E\2\2\u031f\u0320\7N\2\2\u0320\u0321\7W\2\2\u0321\u0322\7U\2\2\u0322\u0323"+ - "\7V\2\2\u0323\u0324\7G\2\2\u0324\u0325\7T\2\2\u0325N\3\2\2\2\u0326\u0327"+ - "\7E\2\2\u0327\u0328\7N\2\2\u0328\u0329\7W\2\2\u0329\u032a\7U\2\2\u032a"+ - "\u032b\7V\2\2\u032b\u032c\7G\2\2\u032c\u032d\7T\2\2\u032d\u032e\7G\2\2"+ - "\u032e\u032f\7F\2\2\u032fP\3\2\2\2\u0330\u0331\7E\2\2\u0331\u0332\7Q\2"+ - "\2\u0332\u0333\7F\2\2\u0333\u0334\7G\2\2\u0334\u0335\7I\2\2\u0335\u0336"+ - "\7G\2\2\u0336\u0337\7P\2\2\u0337R\3\2\2\2\u0338\u0339\7E\2\2\u0339\u033a"+ - "\7Q\2\2\u033a\u033b\7N\2\2\u033b\u033c\7N\2\2\u033c\u033d\7C\2\2\u033d"+ - "\u033e\7V\2\2\u033e\u033f\7G\2\2\u033fT\3\2\2\2\u0340\u0341\7E\2\2\u0341"+ - "\u0342\7Q\2\2\u0342\u0343\7N\2\2\u0343\u0344\7N\2\2\u0344\u0345\7G\2\2"+ - "\u0345\u0346\7E\2\2\u0346\u0347\7V\2\2\u0347\u0348\7K\2\2\u0348\u0349"+ - "\7Q\2\2\u0349\u034a\7P\2\2\u034aV\3\2\2\2\u034b\u034c\7E\2\2\u034c\u034d"+ - "\7Q\2\2\u034d\u034e\7N\2\2\u034e\u034f\7W\2\2\u034f\u0350\7O\2\2\u0350"+ - "\u0351\7P\2\2\u0351X\3\2\2\2\u0352\u0353\7E\2\2\u0353\u0354\7Q\2\2\u0354"+ - "\u0355\7N\2\2\u0355\u0356\7W\2\2\u0356\u0357\7O\2\2\u0357\u0358\7P\2\2"+ - "\u0358\u0359\7U\2\2\u0359Z\3\2\2\2\u035a\u035b\7E\2\2\u035b\u035c\7Q\2"+ - "\2\u035c\u035d\7O\2\2\u035d\u035e\7O\2\2\u035e\u035f\7G\2\2\u035f\u0360"+ - "\7P\2\2\u0360\u0361\7V\2\2\u0361\\\3\2\2\2\u0362\u0363\7E\2\2\u0363\u0364"+ - "\7Q\2\2\u0364\u0365\7O\2\2\u0365\u0366\7O\2\2\u0366\u0367\7K\2\2\u0367"+ - "\u0368\7V\2\2\u0368^\3\2\2\2\u0369\u036a\7E\2\2\u036a\u036b\7Q\2\2\u036b"+ - "\u036c\7O\2\2\u036c\u036d\7R\2\2\u036d\u036e\7C\2\2\u036e\u036f\7E\2\2"+ - "\u036f\u0370\7V\2\2\u0370`\3\2\2\2\u0371\u0372\7E\2\2\u0372\u0373\7Q\2"+ - "\2\u0373\u0374\7O\2\2\u0374\u0375\7R\2\2\u0375\u0376\7C\2\2\u0376\u0377"+ - "\7E\2\2\u0377\u0378\7V\2\2\u0378\u0379\7K\2\2\u0379\u037a\7Q\2\2\u037a"+ - "\u037b\7P\2\2\u037b\u037c\7U\2\2\u037cb\3\2\2\2\u037d\u037e\7E\2\2\u037e"+ - "\u037f\7Q\2\2\u037f\u0380\7O\2\2\u0380\u0381\7R\2\2\u0381\u0382\7W\2\2"+ - "\u0382\u0383\7V\2\2\u0383\u0384\7G\2\2\u0384d\3\2\2\2\u0385\u0386\7E\2"+ - "\2\u0386\u0387\7Q\2\2\u0387\u0388\7P\2\2\u0388\u0389\7E\2\2\u0389\u038a"+ - "\7C\2\2\u038a\u038b\7V\2\2\u038b\u038c\7G\2\2\u038c\u038d\7P\2\2\u038d"+ - "\u038e\7C\2\2\u038e\u038f\7V\2\2\u038f\u0390\7G\2\2\u0390f\3\2\2\2\u0391"+ - "\u0392\7E\2\2\u0392\u0393\7Q\2\2\u0393\u0394\7P\2\2\u0394\u0395\7U\2\2"+ - "\u0395\u0396\7V\2\2\u0396\u0397\7T\2\2\u0397\u0398\7C\2\2\u0398\u0399"+ - "\7K\2\2\u0399\u039a\7P\2\2\u039a\u039b\7V\2\2\u039bh\3\2\2\2\u039c\u039d"+ - "\7E\2\2\u039d\u039e\7Q\2\2\u039e\u039f\7U\2\2\u039f\u03a0\7V\2\2\u03a0"+ - "j\3\2\2\2\u03a1\u03a2\7E\2\2\u03a2\u03a3\7T\2\2\u03a3\u03a4\7G\2\2\u03a4"+ - "\u03a5\7C\2\2\u03a5\u03a6\7V\2\2\u03a6\u03a7\7G\2\2\u03a7l\3\2\2\2\u03a8"+ - "\u03a9\7E\2\2\u03a9\u03aa\7T\2\2\u03aa\u03ab\7Q\2\2\u03ab\u03ac\7U\2\2"+ - "\u03ac\u03ad\7U\2\2\u03adn\3\2\2\2\u03ae\u03af\7E\2\2\u03af\u03b0\7W\2"+ - "\2\u03b0\u03b1\7D\2\2\u03b1\u03b2\7G\2\2\u03b2p\3\2\2\2\u03b3\u03b4\7"+ - "E\2\2\u03b4\u03b5\7W\2\2\u03b5\u03b6\7T\2\2\u03b6\u03b7\7T\2\2\u03b7\u03b8"+ - "\7G\2\2\u03b8\u03b9\7P\2\2\u03b9\u03ba\7V\2\2\u03bar\3\2\2\2\u03bb\u03bc"+ - "\7E\2\2\u03bc\u03bd\7W\2\2\u03bd\u03be\7T\2\2\u03be\u03bf\7T\2\2\u03bf"+ - "\u03c0\7G\2\2\u03c0\u03c1\7P\2\2\u03c1\u03c2\7V\2\2\u03c2\u03c3\7a\2\2"+ - "\u03c3\u03c4\7F\2\2\u03c4\u03c5\7C\2\2\u03c5\u03c6\7V\2\2\u03c6\u03c7"+ - "\7G\2\2\u03c7t\3\2\2\2\u03c8\u03c9\7E\2\2\u03c9\u03ca\7W\2\2\u03ca\u03cb"+ - "\7T\2\2\u03cb\u03cc\7T\2\2\u03cc\u03cd\7G\2\2\u03cd\u03ce\7P\2\2\u03ce"+ - "\u03cf\7V\2\2\u03cf\u03d0\7a\2\2\u03d0\u03d1\7V\2\2\u03d1\u03d2\7K\2\2"+ - "\u03d2\u03d3\7O\2\2\u03d3\u03d4\7G\2\2\u03d4v\3\2\2\2\u03d5\u03d6\7E\2"+ - "\2\u03d6\u03d7\7W\2\2\u03d7\u03d8\7T\2\2\u03d8\u03d9\7T\2\2\u03d9\u03da"+ - "\7G\2\2\u03da\u03db\7P\2\2\u03db\u03dc\7V\2\2\u03dc\u03dd\7a\2\2\u03dd"+ - "\u03de\7V\2\2\u03de\u03df\7K\2\2\u03df\u03e0\7O\2\2\u03e0\u03e1\7G\2\2"+ - "\u03e1\u03e2\7U\2\2\u03e2\u03e3\7V\2\2\u03e3\u03e4\7C\2\2\u03e4\u03e5"+ - "\7O\2\2\u03e5\u03e6\7R\2\2\u03e6x\3\2\2\2\u03e7\u03e8\7E\2\2\u03e8\u03e9"+ - "\7W\2\2\u03e9\u03ea\7T\2\2\u03ea\u03eb\7T\2\2\u03eb\u03ec\7G\2\2\u03ec"+ - "\u03ed\7P\2\2\u03ed\u03ee\7V\2\2\u03ee\u03ef\7a\2\2\u03ef\u03f0\7W\2\2"+ - "\u03f0\u03f1\7U\2\2\u03f1\u03f2\7G\2\2\u03f2\u03f3\7T\2\2\u03f3z\3\2\2"+ - "\2\u03f4\u03f5\7F\2\2\u03f5\u03f6\7C\2\2\u03f6\u03f7\7[\2\2\u03f7|\3\2"+ - "\2\2\u03f8\u03f9\7F\2\2\u03f9\u03fa\7C\2\2\u03fa\u03fb\7V\2\2\u03fb\u03fc"+ - "\7C\2\2\u03fc~\3\2\2\2\u03fd\u03fe\7F\2\2\u03fe\u03ff\7C\2\2\u03ff\u0400"+ - "\7V\2\2\u0400\u0401\7C\2\2\u0401\u0402\7D\2\2\u0402\u0403\7C\2\2\u0403"+ - "\u0404\7U\2\2\u0404\u0405\7G\2\2\u0405\u0080\3\2\2\2\u0406\u0407\7F\2"+ - "\2\u0407\u0408\7C\2\2\u0408\u0409\7V\2\2\u0409\u040a\7C\2\2\u040a\u040b"+ - "\7D\2\2\u040b\u040c\7C\2\2\u040c\u040d\7U\2\2\u040d\u040e\7G\2\2\u040e"+ - "\u0417\7U\2\2\u040f\u0410\7U\2\2\u0410\u0411\7E\2\2\u0411\u0412\7J\2\2"+ - "\u0412\u0413\7G\2\2\u0413\u0414\7O\2\2\u0414\u0415\7C\2\2\u0415\u0417"+ - "\7U\2\2\u0416\u0406\3\2\2\2\u0416\u040f\3\2\2\2\u0417\u0082\3\2\2\2\u0418"+ - "\u0419\7F\2\2\u0419\u041a\7D\2\2\u041a\u041b\7R\2\2\u041b\u041c\7T\2\2"+ - "\u041c\u041d\7Q\2\2\u041d\u041e\7R\2\2\u041e\u041f\7G\2\2\u041f\u0420"+ - "\7T\2\2\u0420\u0421\7V\2\2\u0421\u0422\7K\2\2\u0422\u0423\7G\2\2\u0423"+ - "\u0424\7U\2\2\u0424\u0084\3\2\2\2\u0425\u0426\7F\2\2\u0426\u0427\7G\2"+ - "\2\u0427\u0428\7H\2\2\u0428\u0429\7K\2\2\u0429\u042a\7P\2\2\u042a\u042b"+ - "\7G\2\2\u042b\u042c\7F\2\2\u042c\u0086\3\2\2\2\u042d\u042e\7F\2\2\u042e"+ - "\u042f\7G\2\2\u042f\u0430\7N\2\2\u0430\u0431\7G\2\2\u0431\u0432\7V\2\2"+ - "\u0432\u0433\7G\2\2\u0433\u0088\3\2\2\2\u0434\u0435\7F\2\2\u0435\u0436"+ - "\7G\2\2\u0436\u0437\7N\2\2\u0437\u0438\7K\2\2\u0438\u0439\7O\2\2\u0439"+ - "\u043a\7K\2\2\u043a\u043b\7V\2\2\u043b\u043c\7G\2\2\u043c\u043d\7F\2\2"+ - "\u043d\u008a\3\2\2\2\u043e\u043f\7F\2\2\u043f\u0440\7G\2\2\u0440\u0441"+ - "\7U\2\2\u0441\u0442\7E\2\2\u0442\u008c\3\2\2\2\u0443\u0444\7F\2\2\u0444"+ - "\u0445\7G\2\2\u0445\u0446\7U\2\2\u0446\u0447\7E\2\2\u0447\u0448\7T\2\2"+ - "\u0448\u0449\7K\2\2\u0449\u044a\7D\2\2\u044a\u044b\7G\2\2\u044b\u008e"+ - "\3\2\2\2\u044c\u044d\7F\2\2\u044d\u044e\7H\2\2\u044e\u044f\7U\2\2\u044f"+ - "\u0090\3\2\2\2\u0450\u0451\7F\2\2\u0451\u0452\7K\2\2\u0452\u0453\7T\2"+ - "\2\u0453\u0454\7G\2\2\u0454\u0455\7E\2\2\u0455\u0456\7V\2\2\u0456\u0457"+ - "\7Q\2\2\u0457\u0458\7T\2\2\u0458\u0459\7K\2\2\u0459\u045a\7G\2\2\u045a"+ - "\u045b\7U\2\2\u045b\u0092\3\2\2\2\u045c\u045d\7F\2\2\u045d\u045e\7K\2"+ - "\2\u045e\u045f\7T\2\2\u045f\u0460\7G\2\2\u0460\u0461\7E\2\2\u0461\u0462"+ - "\7V\2\2\u0462\u0463\7Q\2\2\u0463\u0464\7T\2\2\u0464\u0465\7[\2\2\u0465"+ - "\u0094\3\2\2\2\u0466\u0467\7F\2\2\u0467\u0468\7K\2\2\u0468\u0469\7U\2"+ - "\2\u0469\u046a\7V\2\2\u046a\u046b\7K\2\2\u046b\u046c\7P\2\2\u046c\u046d"+ - "\7E\2\2\u046d\u046e\7V\2\2\u046e\u0096\3\2\2\2\u046f\u0470\7F\2\2\u0470"+ - "\u0471\7K\2\2\u0471\u0472\7U\2\2\u0472\u0473\7V\2\2\u0473\u0474\7T\2\2"+ - "\u0474\u0475\7K\2\2\u0475\u0476\7D\2\2\u0476\u0477\7W\2\2\u0477\u0478"+ - "\7V\2\2\u0478\u0479\7G\2\2\u0479\u0098\3\2\2\2\u047a\u047b\7F\2\2\u047b"+ - "\u047c\7K\2\2\u047c\u047d\7X\2\2\u047d\u009a\3\2\2\2\u047e\u047f\7F\2"+ - "\2\u047f\u0480\7T\2\2\u0480\u0481\7Q\2\2\u0481\u0482\7R\2\2\u0482\u009c"+ - "\3\2\2\2\u0483\u0484\7G\2\2\u0484\u0485\7N\2\2\u0485\u0486\7U\2\2\u0486"+ - "\u0487\7G\2\2\u0487\u009e\3\2\2\2\u0488\u0489\7G\2\2\u0489\u048a\7P\2"+ - "\2\u048a\u048b\7F\2\2\u048b\u00a0\3\2\2\2\u048c\u048d\7G\2\2\u048d\u048e"+ - "\7U\2\2\u048e\u048f\7E\2\2\u048f\u0490\7C\2\2\u0490\u0491\7R\2\2\u0491"+ - "\u0492\7G\2\2\u0492\u00a2\3\2\2\2\u0493\u0494\7G\2\2\u0494\u0495\7U\2"+ - "\2\u0495\u0496\7E\2\2\u0496\u0497\7C\2\2\u0497\u0498\7R\2\2\u0498\u0499"+ - "\7G\2\2\u0499\u049a\7F\2\2\u049a\u00a4\3\2\2\2\u049b\u049c\7G\2\2\u049c"+ - "\u049d\7Z\2\2\u049d\u049e\7E\2\2\u049e\u049f\7G\2\2\u049f\u04a0\7R\2\2"+ - "\u04a0\u04a1\7V\2\2\u04a1\u00a6\3\2\2\2\u04a2\u04a3\7G\2\2\u04a3\u04a4"+ - "\7Z\2\2\u04a4\u04a5\7E\2\2\u04a5\u04a6\7J\2\2\u04a6\u04a7\7C\2\2\u04a7"+ - "\u04a8\7P\2\2\u04a8\u04a9\7I\2\2\u04a9\u04aa\7G\2\2\u04aa\u00a8\3\2\2"+ - "\2\u04ab\u04ac\7G\2\2\u04ac\u04ad\7Z\2\2\u04ad\u04ae\7K\2\2\u04ae\u04af"+ - "\7U\2\2\u04af\u04b0\7V\2\2\u04b0\u04b1\7U\2\2\u04b1\u00aa\3\2\2\2\u04b2"+ - "\u04b3\7G\2\2\u04b3\u04b4\7Z\2\2\u04b4\u04b5\7R\2\2\u04b5\u04b6\7N\2\2"+ - "\u04b6\u04b7\7C\2\2\u04b7\u04b8\7K\2\2\u04b8\u04b9\7P\2\2\u04b9\u00ac"+ - "\3\2\2\2\u04ba\u04bb\7G\2\2\u04bb\u04bc\7Z\2\2\u04bc\u04bd\7R\2\2\u04bd"+ - "\u04be\7Q\2\2\u04be\u04bf\7T\2\2\u04bf\u04c0\7V\2\2\u04c0\u00ae\3\2\2"+ - "\2\u04c1\u04c2\7G\2\2\u04c2\u04c3\7Z\2\2\u04c3\u04c4\7V\2\2\u04c4\u04c5"+ - "\7G\2\2\u04c5\u04c6\7P\2\2\u04c6\u04c7\7F\2\2\u04c7\u04c8\7G\2\2\u04c8"+ - "\u04c9\7F\2\2\u04c9\u00b0\3\2\2\2\u04ca\u04cb\7G\2\2\u04cb\u04cc\7Z\2"+ - "\2\u04cc\u04cd\7V\2\2\u04cd\u04ce\7G\2\2\u04ce\u04cf\7T\2\2\u04cf\u04d0"+ - "\7P\2\2\u04d0\u04d1\7C\2\2\u04d1\u04d2\7N\2\2\u04d2\u00b2\3\2\2\2\u04d3"+ - "\u04d4\7G\2\2\u04d4\u04d5\7Z\2\2\u04d5\u04d6\7V\2\2\u04d6\u04d7\7T\2\2"+ - "\u04d7\u04d8\7C\2\2\u04d8\u04d9\7E\2\2\u04d9\u04da\7V\2\2\u04da\u00b4"+ - "\3\2\2\2\u04db\u04dc\7H\2\2\u04dc\u04dd\7C\2\2\u04dd\u04de\7N\2\2\u04de"+ - "\u04df\7U\2\2\u04df\u04e0\7G\2\2\u04e0\u00b6\3\2\2\2\u04e1\u04e2\7H\2"+ - "\2\u04e2\u04e3\7G\2\2\u04e3\u04e4\7V\2\2\u04e4\u04e5\7E\2\2\u04e5\u04e6"+ - "\7J\2\2\u04e6\u00b8\3\2\2\2\u04e7\u04e8\7H\2\2\u04e8\u04e9\7K\2\2\u04e9"+ - "\u04ea\7G\2\2\u04ea\u04eb\7N\2\2\u04eb\u04ec\7F\2\2\u04ec\u04ed\7U\2\2"+ - "\u04ed\u00ba\3\2\2\2\u04ee\u04ef\7H\2\2\u04ef\u04f0\7K\2\2\u04f0\u04f1"+ - "\7N\2\2\u04f1\u04f2\7V\2\2\u04f2\u04f3\7G\2\2\u04f3\u04f4\7T\2\2\u04f4"+ - "\u00bc\3\2\2\2\u04f5\u04f6\7H\2\2\u04f6\u04f7\7K\2\2\u04f7\u04f8\7N\2"+ - "\2\u04f8\u04f9\7G\2\2\u04f9\u04fa\7H\2\2\u04fa\u04fb\7Q\2\2\u04fb\u04fc"+ - "\7T\2\2\u04fc\u04fd\7O\2\2\u04fd\u04fe\7C\2\2\u04fe\u04ff\7V\2\2\u04ff"+ - "\u00be\3\2\2\2\u0500\u0501\7H\2\2\u0501\u0502\7K\2\2\u0502\u0503\7T\2"+ - "\2\u0503\u0504\7U\2\2\u0504\u0505\7V\2\2\u0505\u00c0\3\2\2\2\u0506\u0507"+ - "\7H\2\2\u0507\u0508\7Q\2\2\u0508\u0509\7N\2\2\u0509\u050a\7N\2\2\u050a"+ - "\u050b\7Q\2\2\u050b\u050c\7Y\2\2\u050c\u050d\7K\2\2\u050d\u050e\7P\2\2"+ - "\u050e\u050f\7I\2\2\u050f\u00c2\3\2\2\2\u0510\u0511\7H\2\2\u0511\u0512"+ - "\7Q\2\2\u0512\u0513\7T\2\2\u0513\u00c4\3\2\2\2\u0514\u0515\7H\2\2\u0515"+ - "\u0516\7Q\2\2\u0516\u0517\7T\2\2\u0517\u0518\7G\2\2\u0518\u0519\7K\2\2"+ - "\u0519\u051a\7I\2\2\u051a\u051b\7P\2\2\u051b\u00c6\3\2\2\2\u051c\u051d"+ - "\7H\2\2\u051d\u051e\7Q\2\2\u051e\u051f\7T\2\2\u051f\u0520\7O\2\2\u0520"+ - "\u0521\7C\2\2\u0521\u0522\7V\2\2\u0522\u00c8\3\2\2\2\u0523\u0524\7H\2"+ - "\2\u0524\u0525\7Q\2\2\u0525\u0526\7T\2\2\u0526\u0527\7O\2\2\u0527\u0528"+ - "\7C\2\2\u0528\u0529\7V\2\2\u0529\u052a\7V\2\2\u052a\u052b\7G\2\2\u052b"+ - "\u052c\7F\2\2\u052c\u00ca\3\2\2\2\u052d\u052e\7H\2\2\u052e\u052f\7T\2"+ - "\2\u052f\u0530\7Q\2\2\u0530\u0531\7O\2\2\u0531\u00cc\3\2\2\2\u0532\u0533"+ - "\7H\2\2\u0533\u0534\7W\2\2\u0534\u0535\7N\2\2\u0535\u0536\7N\2\2\u0536"+ - "\u00ce\3\2\2\2\u0537\u0538\7H\2\2\u0538\u0539\7W\2\2\u0539\u053a\7P\2"+ - "\2\u053a\u053b\7E\2\2\u053b\u053c\7V\2\2\u053c\u053d\7K\2\2\u053d\u053e"+ - "\7Q\2\2\u053e\u053f\7P\2\2\u053f\u00d0\3\2\2\2\u0540\u0541\7H\2\2\u0541"+ - "\u0542\7W\2\2\u0542\u0543\7P\2\2\u0543\u0544\7E\2\2\u0544\u0545\7V\2\2"+ - "\u0545\u0546\7K\2\2\u0546\u0547\7Q\2\2\u0547\u0548\7P\2\2\u0548\u0549"+ - "\7U\2\2\u0549\u00d2\3\2\2\2\u054a\u054b\7I\2\2\u054b\u054c\7N\2\2\u054c"+ - "\u054d\7Q\2\2\u054d\u054e\7D\2\2\u054e\u054f\7C\2\2\u054f\u0550\7N\2\2"+ - "\u0550\u00d4\3\2\2\2\u0551\u0552\7I\2\2\u0552\u0553\7T\2\2\u0553\u0554"+ - "\7C\2\2\u0554\u0555\7P\2\2\u0555\u0556\7V\2\2\u0556\u00d6\3\2\2\2\u0557"+ - "\u0558\7I\2\2\u0558\u0559\7T\2\2\u0559\u055a\7Q\2\2\u055a\u055b\7W\2\2"+ - "\u055b\u055c\7R\2\2\u055c\u00d8\3\2\2\2\u055d\u055e\7I\2\2\u055e\u055f"+ - "\7T\2\2\u055f\u0560\7Q\2\2\u0560\u0561\7W\2\2\u0561\u0562\7R\2\2\u0562"+ - "\u0563\7K\2\2\u0563\u0564\7P\2\2\u0564\u0565\7I\2\2\u0565\u00da\3\2\2"+ - "\2\u0566\u0567\7J\2\2\u0567\u0568\7C\2\2\u0568\u0569\7X\2\2\u0569\u056a"+ - "\7K\2\2\u056a\u056b\7P\2\2\u056b\u056c\7I\2\2\u056c\u00dc\3\2\2\2\u056d"+ - "\u056e\7J\2\2\u056e\u056f\7Q\2\2\u056f\u0570\7W\2\2\u0570\u0571\7T\2\2"+ - "\u0571\u00de\3\2\2\2\u0572\u0573\7K\2\2\u0573\u0574\7H\2\2\u0574\u00e0"+ - "\3\2\2\2\u0575\u0576\7K\2\2\u0576\u0577\7I\2\2\u0577\u0578\7P\2\2\u0578"+ - "\u0579\7Q\2\2\u0579\u057a\7T\2\2\u057a\u057b\7G\2\2\u057b\u00e2\3\2\2"+ - "\2\u057c\u057d\7K\2\2\u057d\u057e\7O\2\2\u057e\u057f\7R\2\2\u057f\u0580"+ - "\7Q\2\2\u0580\u0581\7T\2\2\u0581\u0582\7V\2\2\u0582\u00e4\3\2\2\2\u0583"+ - "\u0584\7K\2\2\u0584\u0585\7P\2\2\u0585\u00e6\3\2\2\2\u0586\u0587\7K\2"+ - "\2\u0587\u0588\7P\2\2\u0588\u0589\7F\2\2\u0589\u058a\7G\2\2\u058a\u058b"+ - "\7Z\2\2\u058b\u00e8\3\2\2\2\u058c\u058d\7K\2\2\u058d\u058e\7P\2\2\u058e"+ - "\u058f\7F\2\2\u058f\u0590\7G\2\2\u0590\u0591\7Z\2\2\u0591\u0592\7G\2\2"+ - "\u0592\u0593\7U\2\2\u0593\u00ea\3\2\2\2\u0594\u0595\7K\2\2\u0595\u0596"+ - "\7P\2\2\u0596\u0597\7P\2\2\u0597\u0598\7G\2\2\u0598\u0599\7T\2\2\u0599"+ - "\u00ec\3\2\2\2\u059a\u059b\7K\2\2\u059b\u059c\7P\2\2\u059c\u059d\7R\2"+ - "\2\u059d\u059e\7C\2\2\u059e\u059f\7V\2\2\u059f\u05a0\7J\2\2\u05a0\u00ee"+ - "\3\2\2\2\u05a1\u05a2\7K\2\2\u05a2\u05a3\7P\2\2\u05a3\u05a4\7R\2\2\u05a4"+ - "\u05a5\7W\2\2\u05a5\u05a6\7V\2\2\u05a6\u05a7\7H\2\2\u05a7\u05a8\7Q\2\2"+ - "\u05a8\u05a9\7T\2\2\u05a9\u05aa\7O\2\2\u05aa\u05ab\7C\2\2\u05ab\u05ac"+ - "\7V\2\2\u05ac\u00f0\3\2\2\2\u05ad\u05ae\7K\2\2\u05ae\u05af\7P\2\2\u05af"+ - "\u05b0\7U\2\2\u05b0\u05b1\7G\2\2\u05b1\u05b2\7T\2\2\u05b2\u05b3\7V\2\2"+ - "\u05b3\u00f2\3\2\2\2\u05b4\u05b5\7K\2\2\u05b5\u05b6\7P\2\2\u05b6\u05b7"+ - "\7V\2\2\u05b7\u05b8\7G\2\2\u05b8\u05b9\7T\2\2\u05b9\u05ba\7U\2\2\u05ba"+ - "\u05bb\7G\2\2\u05bb\u05bc\7E\2\2\u05bc\u05bd\7V\2\2\u05bd\u00f4\3\2\2"+ - "\2\u05be\u05bf\7K\2\2\u05bf\u05c0\7P\2\2\u05c0\u05c1\7V\2\2\u05c1\u05c2"+ - "\7G\2\2\u05c2\u05c3\7T\2\2\u05c3\u05c4\7X\2\2\u05c4\u05c5\7C\2\2\u05c5"+ - "\u05c6\7N\2\2\u05c6\u00f6\3\2\2\2\u05c7\u05c8\7K\2\2\u05c8\u05c9\7P\2"+ - "\2\u05c9\u05ca\7V\2\2\u05ca\u05cb\7Q\2\2\u05cb\u00f8\3\2\2\2\u05cc\u05cd"+ - "\7K\2\2\u05cd\u05ce\7U\2\2\u05ce\u00fa\3\2\2\2\u05cf\u05d0\7K\2\2\u05d0"+ - "\u05d1\7V\2\2\u05d1\u05d2\7G\2\2\u05d2\u05d3\7O\2\2\u05d3\u05d4\7U\2\2"+ - "\u05d4\u00fc\3\2\2\2\u05d5\u05d6\7L\2\2\u05d6\u05d7\7Q\2\2\u05d7\u05d8"+ - "\7K\2\2\u05d8\u05d9\7P\2\2\u05d9\u00fe\3\2\2\2\u05da\u05db\7M\2\2\u05db"+ - "\u05dc\7G\2\2\u05dc\u05dd\7[\2\2\u05dd\u05de\7U\2\2\u05de\u0100\3\2\2"+ - "\2\u05df\u05e0\7N\2\2\u05e0\u05e1\7C\2\2\u05e1\u05e2\7U\2\2\u05e2\u05e3"+ - "\7V\2\2\u05e3\u0102\3\2\2\2\u05e4\u05e5\7N\2\2\u05e5\u05e6\7C\2\2\u05e6"+ - "\u05e7\7V\2\2\u05e7\u05e8\7G\2\2\u05e8\u05e9\7T\2\2\u05e9\u05ea\7C\2\2"+ - "\u05ea\u05eb\7N\2\2\u05eb\u0104\3\2\2\2\u05ec\u05ed\7N\2\2\u05ed\u05ee"+ - "\7C\2\2\u05ee\u05ef\7\\\2\2\u05ef\u05f0\7[\2\2\u05f0\u0106\3\2\2\2\u05f1"+ - "\u05f2\7N\2\2\u05f2\u05f3\7G\2\2\u05f3\u05f4\7C\2\2\u05f4\u05f5\7F\2\2"+ - "\u05f5\u05f6\7K\2\2\u05f6\u05f7\7P\2\2\u05f7\u05f8\7I\2\2\u05f8\u0108"+ - "\3\2\2\2\u05f9\u05fa\7N\2\2\u05fa\u05fb\7G\2\2\u05fb\u05fc\7H\2\2\u05fc"+ - "\u05fd\7V\2\2\u05fd\u010a\3\2\2\2\u05fe\u05ff\7N\2\2\u05ff\u0600\7K\2"+ - "\2\u0600\u0601\7M\2\2\u0601\u0602\7G\2\2\u0602\u010c\3\2\2\2\u0603\u0604"+ - "\7N\2\2\u0604\u0605\7K\2\2\u0605\u0606\7O\2\2\u0606\u0607\7K\2\2\u0607"+ - "\u0608\7V\2\2\u0608\u010e\3\2\2\2\u0609\u060a\7N\2\2\u060a\u060b\7K\2"+ - "\2\u060b\u060c\7P\2\2\u060c\u060d\7G\2\2\u060d\u060e\7U\2\2\u060e\u0110"+ - "\3\2\2\2\u060f\u0610\7N\2\2\u0610\u0611\7K\2\2\u0611\u0612\7U\2\2\u0612"+ - "\u0613\7V\2\2\u0613\u0112\3\2\2\2\u0614\u0615\7N\2\2\u0615\u0616\7Q\2"+ - "\2\u0616\u0617\7C\2\2\u0617\u0618\7F\2\2\u0618\u0114\3\2\2\2\u0619\u061a"+ - "\7N\2\2\u061a\u061b\7Q\2\2\u061b\u061c\7E\2\2\u061c\u061d\7C\2\2\u061d"+ - "\u061e\7N\2\2\u061e\u0116\3\2\2\2\u061f\u0620\7N\2\2\u0620\u0621\7Q\2"+ - "\2\u0621\u0622\7E\2\2\u0622\u0623\7C\2\2\u0623\u0624\7V\2\2\u0624\u0625"+ - "\7K\2\2\u0625\u0626\7Q\2\2\u0626\u0627\7P\2\2\u0627\u0118\3\2\2\2\u0628"+ - "\u0629\7N\2\2\u0629\u062a\7Q\2\2\u062a\u062b\7E\2\2\u062b\u062c\7M\2\2"+ - "\u062c\u011a\3\2\2\2\u062d\u062e\7N\2\2\u062e\u062f\7Q\2\2\u062f\u0630"+ - "\7E\2\2\u0630\u0631\7M\2\2\u0631\u0632\7U\2\2\u0632\u011c\3\2\2\2\u0633"+ - "\u0634\7N\2\2\u0634\u0635\7Q\2\2\u0635\u0636\7I\2\2\u0636\u0637\7K\2\2"+ - "\u0637\u0638\7E\2\2\u0638\u0639\7C\2\2\u0639\u063a\7N\2\2\u063a\u011e"+ - "\3\2\2\2\u063b\u063c\7O\2\2\u063c\u063d\7C\2\2\u063d\u063e\7E\2\2\u063e"+ - "\u063f\7T\2\2\u063f\u0640\7Q\2\2\u0640\u0120\3\2\2\2\u0641\u0642\7O\2"+ - "\2\u0642\u0643\7C\2\2\u0643\u0644\7R\2\2\u0644\u0122\3\2\2\2\u0645\u0646"+ - "\7O\2\2\u0646\u0647\7C\2\2\u0647\u0648\7V\2\2\u0648\u0649\7E\2\2\u0649"+ - "\u064a\7J\2\2\u064a\u064b\7G\2\2\u064b\u064c\7F\2\2\u064c\u0124\3\2\2"+ - "\2\u064d\u064e\7O\2\2\u064e\u064f\7G\2\2\u064f\u0650\7T\2\2\u0650\u0651"+ - "\7I\2\2\u0651\u0652\7G\2\2\u0652\u0126\3\2\2\2\u0653\u0654\7O\2\2\u0654"+ - "\u0655\7K\2\2\u0655\u0656\7P\2\2\u0656\u0657\7W\2\2\u0657\u0658\7V\2\2"+ - "\u0658\u0659\7G\2\2\u0659\u0128\3\2\2\2\u065a\u065b\7O\2\2\u065b\u065c"+ - "\7Q\2\2\u065c\u065d\7P\2\2\u065d\u065e\7V\2\2\u065e\u065f\7J\2\2\u065f"+ - "\u012a\3\2\2\2\u0660\u0661\7O\2\2\u0661\u0662\7U\2\2\u0662\u0663\7E\2"+ - "\2\u0663\u0664\7M\2\2\u0664\u012c\3\2\2\2\u0665\u0666\7P\2\2\u0666\u0667"+ - "\7C\2\2\u0667\u0668\7O\2\2\u0668\u0669\7G\2\2\u0669\u066a\7U\2\2\u066a"+ - "\u066b\7R\2\2\u066b\u066c\7C\2\2\u066c\u066d\7E\2\2\u066d\u066e\7G\2\2"+ - "\u066e\u012e\3\2\2\2\u066f\u0670\7P\2\2\u0670\u0671\7C\2\2\u0671\u0672"+ - "\7O\2\2\u0672\u0673\7G\2\2\u0673\u0674\7U\2\2\u0674\u0675\7R\2\2\u0675"+ - "\u0676\7C\2\2\u0676\u0677\7E\2\2\u0677\u0678\7G\2\2\u0678\u0679\7U\2\2"+ - "\u0679\u0130\3\2\2\2\u067a\u067b\7P\2\2\u067b\u067c\7C\2\2\u067c\u067d"+ - "\7V\2\2\u067d\u067e\7W\2\2\u067e\u067f\7T\2\2\u067f\u0680\7C\2\2\u0680"+ - "\u0681\7N\2\2\u0681\u0132\3\2\2\2\u0682\u0683\7P\2\2\u0683\u0684\7Q\2"+ - "\2\u0684\u0134\3\2\2\2\u0685\u0686\7P\2\2\u0686\u0687\7Q\2\2\u0687\u068a"+ - "\7V\2\2\u0688\u068a\7#\2\2\u0689\u0685\3\2\2\2\u0689\u0688\3\2\2\2\u068a"+ - "\u0136\3\2\2\2\u068b\u068c\7P\2\2\u068c\u068d\7W\2\2\u068d\u068e\7N\2"+ - "\2\u068e\u068f\7N\2\2\u068f\u0138\3\2\2\2\u0690\u0691\7P\2\2\u0691\u0692"+ - "\7W\2\2\u0692\u0693\7N\2\2\u0693\u0694\7N\2\2\u0694\u0695\7U\2\2\u0695"+ - "\u013a\3\2\2\2\u0696\u0697\7Q\2\2\u0697\u0698\7H\2\2\u0698\u013c\3\2\2"+ - "\2\u0699\u069a\7Q\2\2\u069a\u069b\7P\2\2\u069b\u013e\3\2\2\2\u069c\u069d"+ - "\7Q\2\2\u069d\u069e\7P\2\2\u069e\u069f\7N\2\2\u069f\u06a0\7[\2\2\u06a0"+ - "\u0140\3\2\2\2\u06a1\u06a2\7Q\2\2\u06a2\u06a3\7R\2\2\u06a3\u06a4\7V\2"+ - "\2\u06a4\u06a5\7K\2\2\u06a5\u06a6\7Q\2\2\u06a6\u06a7\7P\2\2\u06a7\u0142"+ - "\3\2\2\2\u06a8\u06a9\7Q\2\2\u06a9\u06aa\7R\2\2\u06aa\u06ab\7V\2\2\u06ab"+ - "\u06ac\7K\2\2\u06ac\u06ad\7Q\2\2\u06ad\u06ae\7P\2\2\u06ae\u06af\7U\2\2"+ - "\u06af\u0144\3\2\2\2\u06b0\u06b1\7Q\2\2\u06b1\u06b2\7T\2\2\u06b2\u0146"+ - "\3\2\2\2\u06b3\u06b4\7Q\2\2\u06b4\u06b5\7T\2\2\u06b5\u06b6\7F\2\2\u06b6"+ - "\u06b7\7G\2\2\u06b7\u06b8\7T\2\2\u06b8\u0148\3\2\2\2\u06b9\u06ba\7Q\2"+ - "\2\u06ba\u06bb\7W\2\2\u06bb\u06bc\7V\2\2\u06bc\u014a\3\2\2\2\u06bd\u06be"+ - "\7Q\2\2\u06be\u06bf\7W\2\2\u06bf\u06c0\7V\2\2\u06c0\u06c1\7G\2\2\u06c1"+ - "\u06c2\7T\2\2\u06c2\u014c\3\2\2\2\u06c3\u06c4\7Q\2\2\u06c4\u06c5\7W\2"+ - "\2\u06c5\u06c6\7V\2\2\u06c6\u06c7\7R\2\2\u06c7\u06c8\7W\2\2\u06c8\u06c9"+ - "\7V\2\2\u06c9\u06ca\7H\2\2\u06ca\u06cb\7Q\2\2\u06cb\u06cc\7T\2\2\u06cc"+ - "\u06cd\7O\2\2\u06cd\u06ce\7C\2\2\u06ce\u06cf\7V\2\2\u06cf\u014e\3\2\2"+ - "\2\u06d0\u06d1\7Q\2\2\u06d1\u06d2\7X\2\2\u06d2\u06d3\7G\2\2\u06d3\u06d4"+ - "\7T\2\2\u06d4\u0150\3\2\2\2\u06d5\u06d6\7Q\2\2\u06d6\u06d7\7X\2\2\u06d7"+ - "\u06d8\7G\2\2\u06d8\u06d9\7T\2\2\u06d9\u06da\7N\2\2\u06da\u06db\7C\2\2"+ - "\u06db\u06dc\7R\2\2\u06dc\u06dd\7U\2\2\u06dd\u0152\3\2\2\2\u06de\u06df"+ - "\7Q\2\2\u06df\u06e0\7X\2\2\u06e0\u06e1\7G\2\2\u06e1\u06e2\7T\2\2\u06e2"+ - "\u06e3\7N\2\2\u06e3\u06e4\7C\2\2\u06e4\u06e5\7[\2\2\u06e5\u0154\3\2\2"+ - "\2\u06e6\u06e7\7Q\2\2\u06e7\u06e8\7X\2\2\u06e8\u06e9\7G\2\2\u06e9\u06ea"+ - "\7T\2\2\u06ea\u06eb\7Y\2\2\u06eb\u06ec\7T\2\2\u06ec\u06ed\7K\2\2\u06ed"+ - "\u06ee\7V\2\2\u06ee\u06ef\7G\2\2\u06ef\u0156\3\2\2\2\u06f0\u06f1\7R\2"+ - "\2\u06f1\u06f2\7C\2\2\u06f2\u06f3\7T\2\2\u06f3\u06f4\7V\2\2\u06f4\u06f5"+ - "\7K\2\2\u06f5\u06f6\7V\2\2\u06f6\u06f7\7K\2\2\u06f7\u06f8\7Q\2\2\u06f8"+ - "\u06f9\7P\2\2\u06f9\u0158\3\2\2\2\u06fa\u06fb\7R\2\2\u06fb\u06fc\7C\2"+ - "\2\u06fc\u06fd\7T\2\2\u06fd\u06fe\7V\2\2\u06fe\u06ff\7K\2\2\u06ff\u0700"+ - "\7V\2\2\u0700\u0701\7K\2\2\u0701\u0702\7Q\2\2\u0702\u0703\7P\2\2\u0703"+ - "\u0704\7G\2\2\u0704\u0705\7F\2\2\u0705\u015a\3\2\2\2\u0706\u0707\7R\2"+ - "\2\u0707\u0708\7C\2\2\u0708\u0709\7T\2\2\u0709\u070a\7V\2\2\u070a\u070b"+ - "\7K\2\2\u070b\u070c\7V\2\2\u070c\u070d\7K\2\2\u070d\u070e\7Q\2\2\u070e"+ - "\u070f\7P\2\2\u070f\u0710\7U\2\2\u0710\u015c\3\2\2\2\u0711\u0712\7R\2"+ - "\2\u0712\u0713\7G\2\2\u0713\u0714\7T\2\2\u0714\u0715\7E\2\2\u0715\u0716"+ - "\7G\2\2\u0716\u0717\7P\2\2\u0717\u0718\7V\2\2\u0718\u015e\3\2\2\2\u0719"+ - "\u071a\7R\2\2\u071a\u071b\7K\2\2\u071b\u071c\7X\2\2\u071c\u071d\7Q\2\2"+ - "\u071d\u071e\7V\2\2\u071e\u0160\3\2\2\2\u071f\u0720\7R\2\2\u0720\u0721"+ - "\7N\2\2\u0721\u0722\7C\2\2\u0722\u0723\7E\2\2\u0723\u0724\7K\2\2\u0724"+ - "\u0725\7P\2\2\u0725\u0726\7I\2\2\u0726\u0162\3\2\2\2\u0727\u0728\7R\2"+ - "\2\u0728\u0729\7Q\2\2\u0729\u072a\7U\2\2\u072a\u072b\7K\2\2\u072b\u072c"+ - "\7V\2\2\u072c\u072d\7K\2\2\u072d\u072e\7Q\2\2\u072e\u072f\7P\2\2\u072f"+ - "\u0164\3\2\2\2\u0730\u0731\7R\2\2\u0731\u0732\7T\2\2\u0732\u0733\7G\2"+ - "\2\u0733\u0734\7E\2\2\u0734\u0735\7G\2\2\u0735\u0736\7F\2\2\u0736\u0737"+ - "\7K\2\2\u0737\u0738\7P\2\2\u0738\u0739\7I\2\2\u0739\u0166\3\2\2\2\u073a"+ - "\u073b\7R\2\2\u073b\u073c\7T\2\2\u073c\u073d\7K\2\2\u073d\u073e\7O\2\2"+ - "\u073e\u073f\7C\2\2\u073f\u0740\7T\2\2\u0740\u0741\7[\2\2\u0741\u0168"+ - "\3\2\2\2\u0742\u0743\7R\2\2\u0743\u0744\7T\2\2\u0744\u0745\7K\2\2\u0745"+ - "\u0746\7P\2\2\u0746\u0747\7E\2\2\u0747\u0748\7K\2\2\u0748\u0749\7R\2\2"+ - "\u0749\u074a\7C\2\2\u074a\u074b\7N\2\2\u074b\u074c\7U\2\2\u074c\u016a"+ - "\3\2\2\2\u074d\u074e\7R\2\2\u074e\u074f\7T\2\2\u074f\u0750\7Q\2\2\u0750"+ - "\u0751\7R\2\2\u0751\u0752\7G\2\2\u0752\u0753\7T\2\2\u0753\u0754\7V\2\2"+ - "\u0754\u0755\7K\2\2\u0755\u0756\7G\2\2\u0756\u0757\7U\2\2\u0757\u016c"+ - "\3\2\2\2\u0758\u0759\7R\2\2\u0759\u075a\7W\2\2\u075a\u075b\7T\2\2\u075b"+ - "\u075c\7I\2\2\u075c\u075d\7G\2\2\u075d\u016e\3\2\2\2\u075e\u075f\7S\2"+ - "\2\u075f\u0760\7W\2\2\u0760\u0761\7G\2\2\u0761\u0762\7T\2\2\u0762\u0763"+ - "\7[\2\2\u0763\u0170\3\2\2\2\u0764\u0765\7T\2\2\u0765\u0766\7C\2\2\u0766"+ - "\u0767\7P\2\2\u0767\u0768\7I\2\2\u0768\u0769\7G\2\2\u0769\u0172\3\2\2"+ - "\2\u076a\u076b\7T\2\2\u076b\u076c\7G\2\2\u076c\u076d\7E\2\2\u076d\u076e"+ - "\7Q\2\2\u076e\u076f\7T\2\2\u076f\u0770\7F\2\2\u0770\u0771\7T\2\2\u0771"+ - "\u0772\7G\2\2\u0772\u0773\7C\2\2\u0773\u0774\7F\2\2\u0774\u0775\7G\2\2"+ - "\u0775\u0776\7T\2\2\u0776\u0174\3\2\2\2\u0777\u0778\7T\2\2\u0778\u0779"+ - "\7G\2\2\u0779\u077a\7E\2\2\u077a\u077b\7Q\2\2\u077b\u077c\7T\2\2\u077c"+ - "\u077d\7F\2\2\u077d\u077e\7Y\2\2\u077e\u077f\7T\2\2\u077f\u0780\7K\2\2"+ - "\u0780\u0781\7V\2\2\u0781\u0782\7G\2\2\u0782\u0783\7T\2\2\u0783\u0176"+ - "\3\2\2\2\u0784\u0785\7T\2\2\u0785\u0786\7G\2\2\u0786\u0787\7E\2\2\u0787"+ - "\u0788\7Q\2\2\u0788\u0789\7X\2\2\u0789\u078a\7G\2\2\u078a\u078b\7T\2\2"+ - "\u078b\u0178\3\2\2\2\u078c\u078d\7T\2\2\u078d\u078e\7G\2\2\u078e\u078f"+ - "\7F\2\2\u078f\u0790\7W\2\2\u0790\u0791\7E\2\2\u0791\u0792\7G\2\2\u0792"+ - "\u017a\3\2\2\2\u0793\u0794\7T\2\2\u0794\u0795\7G\2\2\u0795\u0796\7H\2"+ - "\2\u0796\u0797\7G\2\2\u0797\u0798\7T\2\2\u0798\u0799\7G\2\2\u0799\u079a"+ - "\7P\2\2\u079a\u079b\7E\2\2\u079b\u079c\7G\2\2\u079c\u079d\7U\2\2\u079d"+ - "\u017c\3\2\2\2\u079e\u079f\7T\2\2\u079f\u07a0\7G\2\2\u07a0\u07a1\7H\2"+ - "\2\u07a1\u07a2\7T\2\2\u07a2\u07a3\7G\2\2\u07a3\u07a4\7U\2\2\u07a4\u07a5"+ - "\7J\2\2\u07a5\u017e\3\2\2\2\u07a6\u07a7\7T\2\2\u07a7\u07a8\7G\2\2\u07a8"+ - "\u07a9\7P\2\2\u07a9\u07aa\7C\2\2\u07aa\u07ab\7O\2\2\u07ab\u07ac\7G\2\2"+ - "\u07ac\u0180\3\2\2\2\u07ad\u07ae\7T\2\2\u07ae\u07af\7G\2\2\u07af\u07b0"+ - "\7R\2\2\u07b0\u07b1\7C\2\2\u07b1\u07b2\7K\2\2\u07b2\u07b3\7T\2\2\u07b3"+ - "\u0182\3\2\2\2\u07b4\u07b5\7T\2\2\u07b5\u07b6\7G\2\2\u07b6\u07b7\7R\2"+ - "\2\u07b7\u07b8\7N\2\2\u07b8\u07b9\7C\2\2\u07b9\u07ba\7E\2\2\u07ba\u07bb"+ - "\7G\2\2\u07bb\u0184\3\2\2\2\u07bc\u07bd\7T\2\2\u07bd\u07be\7G\2\2\u07be"+ - "\u07bf\7U\2\2\u07bf\u07c0\7G\2\2\u07c0\u07c1\7V\2\2\u07c1\u0186\3\2\2"+ - "\2\u07c2\u07c3\7T\2\2\u07c3\u07c4\7G\2\2\u07c4\u07c5\7U\2\2\u07c5\u07c6"+ - "\7R\2\2\u07c6\u07c7\7G\2\2\u07c7\u07c8\7E\2\2\u07c8\u07c9\7V\2\2\u07c9"+ - "\u0188\3\2\2\2\u07ca\u07cb\7T\2\2\u07cb\u07cc\7G\2\2\u07cc\u07cd\7U\2"+ - "\2\u07cd\u07ce\7V\2\2\u07ce\u07cf\7T\2\2\u07cf\u07d0\7K\2\2\u07d0\u07d1"+ - "\7E\2\2\u07d1\u07d2\7V\2\2\u07d2\u018a\3\2\2\2\u07d3\u07d4\7T\2\2\u07d4"+ - "\u07d5\7G\2\2\u07d5\u07d6\7X\2\2\u07d6\u07d7\7Q\2\2\u07d7\u07d8\7M\2\2"+ - "\u07d8\u07d9\7G\2\2\u07d9\u018c\3\2\2\2\u07da\u07db\7T\2\2\u07db\u07dc"+ - "\7K\2\2\u07dc\u07dd\7I\2\2\u07dd\u07de\7J\2\2\u07de\u07df\7V\2\2\u07df"+ - "\u018e\3\2\2\2\u07e0\u07e1\7T\2\2\u07e1\u07e2\7N\2\2\u07e2\u07e3\7K\2"+ - "\2\u07e3\u07e4\7M\2\2\u07e4\u07ec\7G\2\2\u07e5\u07e6\7T\2\2\u07e6\u07e7"+ - "\7G\2\2\u07e7\u07e8\7I\2\2\u07e8\u07e9\7G\2\2\u07e9\u07ea\7Z\2\2\u07ea"+ - "\u07ec\7R\2\2\u07eb\u07e0\3\2\2\2\u07eb\u07e5\3\2\2\2\u07ec\u0190\3\2"+ - "\2\2\u07ed\u07ee\7T\2\2\u07ee\u07ef\7Q\2\2\u07ef\u07f0\7N\2\2\u07f0\u07f1"+ - "\7G\2\2\u07f1\u0192\3\2\2\2\u07f2\u07f3\7T\2\2\u07f3\u07f4\7Q\2\2\u07f4"+ - "\u07f5\7N\2\2\u07f5\u07f6\7G\2\2\u07f6\u07f7\7U\2\2\u07f7\u0194\3\2\2"+ - "\2\u07f8\u07f9\7T\2\2\u07f9\u07fa\7Q\2\2\u07fa\u07fb\7N\2\2\u07fb\u07fc"+ - "\7N\2\2\u07fc\u07fd\7D\2\2\u07fd\u07fe\7C\2\2\u07fe\u07ff\7E\2\2\u07ff"+ - "\u0800\7M\2\2\u0800\u0196\3\2\2\2\u0801\u0802\7T\2\2\u0802\u0803\7Q\2"+ - "\2\u0803\u0804\7N\2\2\u0804\u0805\7N\2\2\u0805\u0806\7W\2\2\u0806\u0807"+ - "\7R\2\2\u0807\u0198\3\2\2\2\u0808\u0809\7T\2\2\u0809\u080a\7Q\2\2\u080a"+ - "\u080b\7Y\2\2\u080b\u019a\3\2\2\2\u080c\u080d\7T\2\2\u080d\u080e\7Q\2"+ - "\2\u080e\u080f\7Y\2\2\u080f\u0810\7U\2\2\u0810\u019c\3\2\2\2\u0811\u0812"+ - "\7U\2\2\u0812\u0813\7G\2\2\u0813\u0814\7E\2\2\u0814\u0815\7Q\2\2\u0815"+ - "\u0816\7P\2\2\u0816\u0817\7F\2\2\u0817\u019e\3\2\2\2\u0818\u0819\7U\2"+ - "\2\u0819\u081a\7E\2\2\u081a\u081b\7J\2\2\u081b\u081c\7G\2\2\u081c\u081d"+ - "\7O\2\2\u081d\u081e\7C\2\2\u081e\u01a0\3\2\2\2\u081f\u0820\7U\2\2\u0820"+ - "\u0821\7G\2\2\u0821\u0822\7N\2\2\u0822\u0823\7G\2\2\u0823\u0824\7E\2\2"+ - "\u0824\u0825\7V\2\2\u0825\u01a2\3\2\2\2\u0826\u0827\7U\2\2\u0827\u0828"+ - "\7G\2\2\u0828\u0829\7O\2\2\u0829\u082a\7K\2\2\u082a\u01a4\3\2\2\2\u082b"+ - "\u082c\7U\2\2\u082c\u082d\7G\2\2\u082d\u082e\7R\2\2\u082e\u082f\7C\2\2"+ - "\u082f\u0830\7T\2\2\u0830\u0831\7C\2\2\u0831\u0832\7V\2\2\u0832\u0833"+ - "\7G\2\2\u0833\u0834\7F\2\2\u0834\u01a6\3\2\2\2\u0835\u0836\7U\2\2\u0836"+ - "\u0837\7G\2\2\u0837\u0838\7T\2\2\u0838\u0839\7F\2\2\u0839\u083a\7G\2\2"+ - "\u083a\u01a8\3\2\2\2\u083b\u083c\7U\2\2\u083c\u083d\7G\2\2\u083d\u083e"+ - "\7T\2\2\u083e\u083f\7F\2\2\u083f\u0840\7G\2\2\u0840\u0841\7R\2\2\u0841"+ - "\u0842\7T\2\2\u0842\u0843\7Q\2\2\u0843\u0844\7R\2\2\u0844\u0845\7G\2\2"+ - "\u0845\u0846\7T\2\2\u0846\u0847\7V\2\2\u0847\u0848\7K\2\2\u0848\u0849"+ - "\7G\2\2\u0849\u084a\7U\2\2\u084a\u01aa\3\2\2\2\u084b\u084c\7U\2\2\u084c"+ - "\u084d\7G\2\2\u084d\u084e\7U\2\2\u084e\u084f\7U\2\2\u084f\u0850\7K\2\2"+ - "\u0850\u0851\7Q\2\2\u0851\u0852\7P\2\2\u0852\u0853\7a\2\2\u0853\u0854"+ - "\7W\2\2\u0854\u0855\7U\2\2\u0855\u0856\7G\2\2\u0856\u0857\7T\2\2\u0857"+ - "\u01ac\3\2\2\2\u0858\u0859\7U\2\2\u0859\u085a\7G\2\2\u085a\u085b\7V\2"+ - "\2\u085b\u01ae\3\2\2\2\u085c\u085d\7O\2\2\u085d\u085e\7K\2\2\u085e\u085f"+ - "\7P\2\2\u085f\u0860\7W\2\2\u0860\u0861\7U\2\2\u0861\u01b0\3\2\2\2\u0862"+ - "\u0863\7U\2\2\u0863\u0864\7G\2\2\u0864\u0865\7V\2\2\u0865\u0866\7U\2\2"+ - "\u0866\u01b2\3\2\2\2\u0867\u0868\7U\2\2\u0868\u0869\7J\2\2\u0869\u086a"+ - "\7Q\2\2\u086a\u086b\7Y\2\2\u086b\u01b4\3\2\2\2\u086c\u086d\7U\2\2\u086d"+ - "\u086e\7M\2\2\u086e\u086f\7G\2\2\u086f\u0870\7Y\2\2\u0870\u0871\7G\2\2"+ - "\u0871\u0872\7F\2\2\u0872\u01b6\3\2\2\2\u0873\u0874\7U\2\2\u0874\u0875"+ - "\7Q\2\2\u0875\u0876\7O\2\2\u0876\u0877\7G\2\2\u0877\u01b8\3\2\2\2\u0878"+ - "\u0879\7U\2\2\u0879\u087a\7Q\2\2\u087a\u087b\7T\2\2\u087b\u087c\7V\2\2"+ - "\u087c\u01ba\3\2\2\2\u087d\u087e\7U\2\2\u087e\u087f\7Q\2\2\u087f\u0880"+ - "\7T\2\2\u0880\u0881\7V\2\2\u0881\u0882\7G\2\2\u0882\u0883\7F\2\2\u0883"+ - "\u01bc\3\2\2\2\u0884\u0885\7U\2\2\u0885\u0886\7V\2\2\u0886\u0887\7C\2"+ - "\2\u0887\u0888\7T\2\2\u0888\u0889\7V\2\2\u0889\u01be\3\2\2\2\u088a\u088b"+ - "\7U\2\2\u088b\u088c\7V\2\2\u088c\u088d\7C\2\2\u088d\u088e\7V\2\2\u088e"+ - "\u088f\7K\2\2\u088f\u0890\7U\2\2\u0890\u0891\7V\2\2\u0891\u0892\7K\2\2"+ - "\u0892\u0893\7E\2\2\u0893\u0894\7U\2\2\u0894\u01c0\3\2\2\2\u0895\u0896"+ - "\7U\2\2\u0896\u0897\7V\2\2\u0897\u0898\7Q\2\2\u0898\u0899\7T\2\2\u0899"+ - "\u089a\7G\2\2\u089a\u089b\7F\2\2\u089b\u01c2\3\2\2\2\u089c\u089d\7U\2"+ - "\2\u089d\u089e\7V\2\2\u089e\u089f\7T\2\2\u089f\u08a0\7C\2\2\u08a0\u08a1"+ - "\7V\2\2\u08a1\u08a2\7K\2\2\u08a2\u08a3\7H\2\2\u08a3\u08a4\7[\2\2\u08a4"+ - "\u01c4\3\2\2\2\u08a5\u08a6\7U\2\2\u08a6\u08a7\7V\2\2\u08a7\u08a8\7T\2"+ - "\2\u08a8\u08a9\7W\2\2\u08a9\u08aa\7E\2\2\u08aa\u08ab\7V\2\2\u08ab\u01c6"+ - "\3\2\2\2\u08ac\u08ad\7U\2\2\u08ad\u08ae\7W\2\2\u08ae\u08af\7D\2\2\u08af"+ - "\u08b0\7U\2\2\u08b0\u08b1\7V\2\2\u08b1\u08b2\7T\2\2\u08b2\u01c8\3\2\2"+ - "\2\u08b3\u08b4\7U\2\2\u08b4\u08b5\7W\2\2\u08b5\u08b6\7D\2\2\u08b6\u08b7"+ - "\7U\2\2\u08b7\u08b8\7V\2\2\u08b8\u08b9\7T\2\2\u08b9\u08ba\7K\2\2\u08ba"+ - "\u08bb\7P\2\2\u08bb\u08bc\7I\2\2\u08bc\u01ca\3\2\2\2\u08bd\u08be\7U\2"+ - "\2\u08be\u08bf\7[\2\2\u08bf\u08c0\7P\2\2\u08c0\u08c1\7E\2\2\u08c1\u01cc"+ - "\3\2\2\2\u08c2\u08c3\7V\2\2\u08c3\u08c4\7C\2\2\u08c4\u08c5\7D\2\2\u08c5"+ - "\u08c6\7N\2\2\u08c6\u08c7\7G\2\2\u08c7\u01ce\3\2\2\2\u08c8\u08c9\7V\2"+ - "\2\u08c9\u08ca\7C\2\2\u08ca\u08cb\7D\2\2\u08cb\u08cc\7N\2\2\u08cc\u08cd"+ - "\7G\2\2\u08cd\u08ce\7U\2\2\u08ce\u01d0\3\2\2\2\u08cf\u08d0\7V\2\2\u08d0"+ - "\u08d1\7C\2\2\u08d1\u08d2\7D\2\2\u08d2\u08d3\7N\2\2\u08d3\u08d4\7G\2\2"+ - "\u08d4\u08d5\7U\2\2\u08d5\u08d6\7C\2\2\u08d6\u08d7\7O\2\2\u08d7\u08d8"+ - "\7R\2\2\u08d8\u08d9\7N\2\2\u08d9\u08da\7G\2\2\u08da\u01d2\3\2\2\2\u08db"+ - "\u08dc\7V\2\2\u08dc\u08dd\7D\2\2\u08dd\u08de\7N\2\2\u08de\u08df\7R\2\2"+ - "\u08df\u08e0\7T\2\2\u08e0\u08e1\7Q\2\2\u08e1\u08e2\7R\2\2\u08e2\u08e3"+ - "\7G\2\2\u08e3\u08e4\7T\2\2\u08e4\u08e5\7V\2\2\u08e5\u08e6\7K\2\2\u08e6"+ - "\u08e7\7G\2\2\u08e7\u08e8\7U\2\2\u08e8\u01d4\3\2\2\2\u08e9\u08ea\7V\2"+ - "\2\u08ea\u08eb\7G\2\2\u08eb\u08ec\7O\2\2\u08ec\u08ed\7R\2\2\u08ed\u08ee"+ - "\7Q\2\2\u08ee\u08ef\7T\2\2\u08ef\u08f0\7C\2\2\u08f0\u08f1\7T\2\2\u08f1"+ - "\u08f7\7[\2\2\u08f2\u08f3\7V\2\2\u08f3\u08f4\7G\2\2\u08f4\u08f5\7O\2\2"+ - "\u08f5\u08f7\7R\2\2\u08f6\u08e9\3\2\2\2\u08f6\u08f2\3\2\2\2\u08f7\u01d6"+ - "\3\2\2\2\u08f8\u08f9\7V\2\2\u08f9\u08fa\7G\2\2\u08fa\u08fb\7T\2\2\u08fb"+ - "\u08fc\7O\2\2\u08fc\u08fd\7K\2\2\u08fd\u08fe\7P\2\2\u08fe\u08ff\7C\2\2"+ - "\u08ff\u0900\7V\2\2\u0900\u0901\7G\2\2\u0901\u0902\7F\2\2\u0902\u01d8"+ - "\3\2\2\2\u0903\u0904\7V\2\2\u0904\u0905\7J\2\2\u0905\u0906\7G\2\2\u0906"+ - "\u0907\7P\2\2\u0907\u01da\3\2\2\2\u0908\u0909\7V\2\2\u0909\u090a\7K\2"+ - "\2\u090a\u090b\7O\2\2\u090b\u090c\7G\2\2\u090c\u01dc\3\2\2\2\u090d\u090e"+ - "\7V\2\2\u090e\u090f\7Q\2\2\u090f\u01de\3\2\2\2\u0910\u0911\7V\2\2\u0911"+ - "\u0912\7Q\2\2\u0912\u0913\7W\2\2\u0913\u0914\7E\2\2\u0914\u0915\7J\2\2"+ - "\u0915\u01e0\3\2\2\2\u0916\u0917\7V\2\2\u0917\u0918\7T\2\2\u0918\u0919"+ - "\7C\2\2\u0919\u091a\7K\2\2\u091a\u091b\7N\2\2\u091b\u091c\7K\2\2\u091c"+ - "\u091d\7P\2\2\u091d\u091e\7I\2\2\u091e\u01e2\3\2\2\2\u091f\u0920\7V\2"+ - "\2\u0920\u0921\7T\2\2\u0921\u0922\7C\2\2\u0922\u0923\7P\2\2\u0923\u0924"+ - "\7U\2\2\u0924\u0925\7C\2\2\u0925\u0926\7E\2\2\u0926\u0927\7V\2\2\u0927"+ - "\u0928\7K\2\2\u0928\u0929\7Q\2\2\u0929\u092a\7P\2\2\u092a\u01e4\3\2\2"+ - "\2\u092b\u092c\7V\2\2\u092c\u092d\7T\2\2\u092d\u092e\7C\2\2\u092e\u092f"+ - "\7P\2\2\u092f\u0930\7U\2\2\u0930\u0931\7C\2\2\u0931\u0932\7E\2\2\u0932"+ - "\u0933\7V\2\2\u0933\u0934\7K\2\2\u0934\u0935\7Q\2\2\u0935\u0936\7P\2\2"+ - "\u0936\u0937\7U\2\2\u0937\u01e6\3\2\2\2\u0938\u0939\7V\2\2\u0939\u093a"+ - "\7T\2\2\u093a\u093b\7C\2\2\u093b\u093c\7P\2\2\u093c\u093d\7U\2\2\u093d"+ - "\u093e\7H\2\2\u093e\u093f\7Q\2\2\u093f\u0940\7T\2\2\u0940\u0941\7O\2\2"+ - "\u0941\u01e8\3\2\2\2\u0942\u0943\7V\2\2\u0943\u0944\7T\2\2\u0944\u0945"+ - "\7K\2\2\u0945\u0946\7O\2\2\u0946\u01ea\3\2\2\2\u0947\u0948\7V\2\2\u0948"+ - "\u0949\7T\2\2\u0949\u094a\7W\2\2\u094a\u094b\7G\2\2\u094b\u01ec\3\2\2"+ - "\2\u094c\u094d\7V\2\2\u094d\u094e\7T\2\2\u094e\u094f\7W\2\2\u094f\u0950"+ - "\7P\2\2\u0950\u0951\7E\2\2\u0951\u0952\7C\2\2\u0952\u0953\7V\2\2\u0953"+ - "\u0954\7G\2\2\u0954\u01ee\3\2\2\2\u0955\u0956\7V\2\2\u0956\u0957\7T\2"+ - "\2\u0957\u0958\7[\2\2\u0958\u0959\7a\2\2\u0959\u095a\7E\2\2\u095a\u095b"+ - "\7C\2\2\u095b\u095c\7U\2\2\u095c\u095d\7V\2\2\u095d\u01f0\3\2\2\2\u095e"+ - "\u095f\7V\2\2\u095f\u0960\7[\2\2\u0960\u0961\7R\2\2\u0961\u0962\7G\2\2"+ - "\u0962\u01f2\3\2\2\2\u0963\u0964\7W\2\2\u0964\u0965\7P\2\2\u0965\u0966"+ - "\7C\2\2\u0966\u0967\7T\2\2\u0967\u0968\7E\2\2\u0968\u0969\7J\2\2\u0969"+ - "\u096a\7K\2\2\u096a\u096b\7X\2\2\u096b\u096c\7G\2\2\u096c\u01f4\3\2\2"+ - "\2\u096d\u096e\7W\2\2\u096e\u096f\7P\2\2\u096f\u0970\7D\2\2\u0970\u0971"+ - "\7Q\2\2\u0971\u0972\7W\2\2\u0972\u0973\7P\2\2\u0973\u0974\7F\2\2\u0974"+ - "\u0975\7G\2\2\u0975\u0976\7F\2\2\u0976\u01f6\3\2\2\2\u0977\u0978\7W\2"+ - "\2\u0978\u0979\7P\2\2\u0979\u097a\7E\2\2\u097a\u097b\7C\2\2\u097b\u097c"+ - "\7E\2\2\u097c\u097d\7J\2\2\u097d\u097e\7G\2\2\u097e\u01f8\3\2\2\2\u097f"+ - "\u0980\7W\2\2\u0980\u0981\7P\2\2\u0981\u0982\7K\2\2\u0982\u0983\7Q\2\2"+ - "\u0983\u0984\7P\2\2\u0984\u01fa\3\2\2\2\u0985\u0986\7W\2\2\u0986\u0987"+ - "\7P\2\2\u0987\u0988\7K\2\2\u0988\u0989\7S\2\2\u0989\u098a\7W\2\2\u098a"+ - "\u098b\7G\2\2\u098b\u01fc\3\2\2\2\u098c\u098d\7W\2\2\u098d\u098e\7P\2"+ - "\2\u098e\u098f\7M\2\2\u098f\u0990\7P\2\2\u0990\u0991\7Q\2\2\u0991\u0992"+ - "\7Y\2\2\u0992\u0993\7P\2\2\u0993\u01fe\3\2\2\2\u0994\u0995\7W\2\2\u0995"+ - "\u0996\7P\2\2\u0996\u0997\7N\2\2\u0997\u0998\7Q\2\2\u0998\u0999\7E\2\2"+ - "\u0999\u099a\7M\2\2\u099a\u0200\3\2\2\2\u099b\u099c\7W\2\2\u099c\u099d"+ - "\7P\2\2\u099d\u099e\7U\2\2\u099e\u099f\7G\2\2\u099f\u09a0\7V\2\2\u09a0"+ - "\u0202\3\2\2\2\u09a1\u09a2\7W\2\2\u09a2\u09a3\7R\2\2\u09a3\u09a4\7F\2"+ - "\2\u09a4\u09a5\7C\2\2\u09a5\u09a6\7V\2\2\u09a6\u09a7\7G\2\2\u09a7\u0204"+ - "\3\2\2\2\u09a8\u09a9\7W\2\2\u09a9\u09aa\7U\2\2\u09aa\u09ab\7G\2\2\u09ab"+ - "\u0206\3\2\2\2\u09ac\u09ad\7W\2\2\u09ad\u09ae\7U\2\2\u09ae\u09af\7G\2"+ - "\2\u09af\u09b0\7T\2\2\u09b0\u0208\3\2\2\2\u09b1\u09b2\7W\2\2\u09b2\u09b3"+ - "\7U\2\2\u09b3\u09b4\7K\2\2\u09b4\u09b5\7P\2\2\u09b5\u09b6\7I\2\2\u09b6"+ - "\u020a\3\2\2\2\u09b7\u09b8\7X\2\2\u09b8\u09b9\7C\2\2\u09b9\u09ba\7N\2"+ - "\2\u09ba\u09bb\7W\2\2\u09bb\u09bc\7G\2\2\u09bc\u09bd\7U\2\2\u09bd\u020c"+ - "\3\2\2\2\u09be\u09bf\7X\2\2\u09bf\u09c0\7K\2\2\u09c0\u09c1\7G\2\2\u09c1"+ - "\u09c2\7Y\2\2\u09c2\u020e\3\2\2\2\u09c3\u09c4\7X\2\2\u09c4\u09c5\7K\2"+ - "\2\u09c5\u09c6\7G\2\2\u09c6\u09c7\7Y\2\2\u09c7\u09c8\7U\2\2\u09c8\u0210"+ - "\3\2\2\2\u09c9\u09ca\7Y\2\2\u09ca\u09cb\7J\2\2\u09cb\u09cc\7G\2\2\u09cc"+ - "\u09cd\7P\2\2\u09cd\u0212\3\2\2\2\u09ce\u09cf\7Y\2\2\u09cf\u09d0\7J\2"+ - "\2\u09d0\u09d1\7G\2\2\u09d1\u09d2\7T\2\2\u09d2\u09d3\7G\2\2\u09d3\u0214"+ - "\3\2\2\2\u09d4\u09d5\7Y\2\2\u09d5\u09d6\7K\2\2\u09d6\u09d7\7P\2\2\u09d7"+ - "\u09d8\7F\2\2\u09d8\u09d9\7Q\2\2\u09d9\u09da\7Y\2\2\u09da\u0216\3\2\2"+ - "\2\u09db\u09dc\7Y\2\2\u09dc\u09dd\7K\2\2\u09dd\u09de\7V\2\2\u09de\u09df"+ - "\7J\2\2\u09df\u0218\3\2\2\2\u09e0\u09e1\7[\2\2\u09e1\u09e2\7G\2\2\u09e2"+ - "\u09e3\7C\2\2\u09e3\u09e4\7T\2\2\u09e4\u021a\3\2\2\2\u09e5\u09e6\7\\\2"+ - "\2\u09e6\u09e7\7Q\2\2\u09e7\u09e8\7P\2\2\u09e8\u09e9\7G\2\2\u09e9\u021c"+ - "\3\2\2\2\u09ea\u09eb\7M\2\2\u09eb\u09ec\7G\2\2\u09ec\u09ed\7[\2\2\u09ed"+ - "\u021e\3\2\2\2\u09ee\u09f2\7?\2\2\u09ef\u09f0\7?\2\2\u09f0\u09f2\7?\2"+ - "\2\u09f1\u09ee\3\2\2\2\u09f1\u09ef\3\2\2\2\u09f2\u0220\3\2\2\2\u09f3\u09f4"+ - "\7>\2\2\u09f4\u09f5\7?\2\2\u09f5\u09f6\7@\2\2\u09f6\u0222\3\2\2\2\u09f7"+ - "\u09f8\7>\2\2\u09f8\u09f9\7@\2\2\u09f9\u0224\3\2\2\2\u09fa\u09fb\7#\2"+ - "\2\u09fb\u09fc\7?\2\2\u09fc\u0226\3\2\2\2\u09fd\u09fe\7>\2\2\u09fe\u0228"+ - "\3\2\2\2\u09ff\u0a00\7>\2\2\u0a00\u0a04\7?\2\2\u0a01\u0a02\7#\2\2\u0a02"+ - "\u0a04\7@\2\2\u0a03\u09ff\3\2\2\2\u0a03\u0a01\3\2\2\2\u0a04\u022a\3\2"+ - "\2\2\u0a05\u0a06\7@\2\2\u0a06\u022c\3\2\2\2\u0a07\u0a08\7@\2\2\u0a08\u0a0c"+ - "\7?\2\2\u0a09\u0a0a\7#\2\2\u0a0a\u0a0c\7>\2\2\u0a0b\u0a07\3\2\2\2\u0a0b"+ - "\u0a09\3\2\2\2\u0a0c\u022e\3\2\2\2\u0a0d\u0a0e\7-\2\2\u0a0e\u0230\3\2"+ - "\2\2\u0a0f\u0a10\7/\2\2\u0a10\u0232\3\2\2\2\u0a11\u0a12\7,\2\2\u0a12\u0234"+ - "\3\2\2\2\u0a13"; - private static final String _serializedATNSegment1 = - "\u0a14\7\61\2\2\u0a14\u0236\3\2\2\2\u0a15\u0a16\7\'\2\2\u0a16\u0238\3"+ - "\2\2\2\u0a17\u0a18\7\u0080\2\2\u0a18\u023a\3\2\2\2\u0a19\u0a1a\7(\2\2"+ - "\u0a1a\u023c\3\2\2\2\u0a1b\u0a1c\7~\2\2\u0a1c\u023e\3\2\2\2\u0a1d\u0a1e"+ - "\7~\2\2\u0a1e\u0a1f\7~\2\2\u0a1f\u0240\3\2\2\2\u0a20\u0a21\7`\2\2\u0a21"+ - "\u0242\3\2\2\2\u0a22\u0a28\7)\2\2\u0a23\u0a27\n\2\2\2\u0a24\u0a25\7^\2"+ - "\2\u0a25\u0a27\13\2\2\2\u0a26\u0a23\3\2\2\2\u0a26\u0a24\3\2\2\2\u0a27"+ - "\u0a2a\3\2\2\2\u0a28\u0a26\3\2\2\2\u0a28\u0a29\3\2\2\2\u0a29\u0a2b\3\2"+ - "\2\2\u0a2a\u0a28\3\2\2\2\u0a2b\u0a37\7)\2\2\u0a2c\u0a32\7$\2\2\u0a2d\u0a31"+ - "\n\3\2\2\u0a2e\u0a2f\7^\2\2\u0a2f\u0a31\13\2\2\2\u0a30\u0a2d\3\2\2\2\u0a30"+ - "\u0a2e\3\2\2\2\u0a31\u0a34\3\2\2\2\u0a32\u0a30\3\2\2\2\u0a32\u0a33\3\2"+ - "\2\2\u0a33\u0a35\3\2\2\2\u0a34\u0a32\3\2\2\2\u0a35\u0a37\7$\2\2\u0a36"+ - "\u0a22\3\2\2\2\u0a36\u0a2c\3\2\2\2\u0a37\u0244\3\2\2\2\u0a38\u0a3a\5\u025f"+ - "\u0130\2\u0a39\u0a38\3\2\2\2\u0a3a\u0a3b\3\2\2\2\u0a3b\u0a39\3\2\2\2\u0a3b"+ - "\u0a3c\3\2\2\2\u0a3c\u0a3d\3\2\2\2\u0a3d\u0a3e\7N\2\2\u0a3e\u0246\3\2"+ - "\2\2\u0a3f\u0a41\5\u025f\u0130\2\u0a40\u0a3f\3\2\2\2\u0a41\u0a42\3\2\2"+ - "\2\u0a42\u0a40\3\2\2\2\u0a42\u0a43\3\2\2\2\u0a43\u0a44\3\2\2\2\u0a44\u0a45"+ - "\7U\2\2\u0a45\u0248\3\2\2\2\u0a46\u0a48\5\u025f\u0130\2\u0a47\u0a46\3"+ - "\2\2\2\u0a48\u0a49\3\2\2\2\u0a49\u0a47\3\2\2\2\u0a49\u0a4a\3\2\2\2\u0a4a"+ - "\u0a4b\3\2\2\2\u0a4b\u0a4c\7[\2\2\u0a4c\u024a\3\2\2\2\u0a4d\u0a4f\5\u025f"+ - "\u0130\2\u0a4e\u0a4d\3\2\2\2\u0a4f\u0a50\3\2\2\2\u0a50\u0a4e\3\2\2\2\u0a50"+ - "\u0a51\3\2\2\2\u0a51\u024c\3\2\2\2\u0a52\u0a54\5\u025f\u0130\2\u0a53\u0a52"+ - "\3\2\2\2\u0a54\u0a55\3\2\2\2\u0a55\u0a53\3\2\2\2\u0a55\u0a56\3\2\2\2\u0a56"+ - "\u0a57\3\2\2\2\u0a57\u0a58\5\u025d\u012f\2\u0a58\u0a5e\3\2\2\2\u0a59\u0a5a"+ - "\5\u025b\u012e\2\u0a5a\u0a5b\5\u025d\u012f\2\u0a5b\u0a5c\6\u0127\2\2\u0a5c"+ - "\u0a5e\3\2\2\2\u0a5d\u0a53\3\2\2\2\u0a5d\u0a59\3\2\2\2\u0a5e\u024e\3\2"+ - "\2\2\u0a5f\u0a60\5\u025b\u012e\2\u0a60\u0a61\6\u0128\3\2\u0a61\u0250\3"+ - "\2\2\2\u0a62\u0a64\5\u025f\u0130\2\u0a63\u0a62\3\2\2\2\u0a64\u0a65\3\2"+ - "\2\2\u0a65\u0a63\3\2\2\2\u0a65\u0a66\3\2\2\2\u0a66\u0a68\3\2\2\2\u0a67"+ - "\u0a69\5\u025d\u012f\2\u0a68\u0a67\3\2\2\2\u0a68\u0a69\3\2\2\2\u0a69\u0a6a"+ - "\3\2\2\2\u0a6a\u0a6b\7H\2\2\u0a6b\u0a74\3\2\2\2\u0a6c\u0a6e\5\u025b\u012e"+ - "\2\u0a6d\u0a6f\5\u025d\u012f\2\u0a6e\u0a6d\3\2\2\2\u0a6e\u0a6f\3\2\2\2"+ - "\u0a6f\u0a70\3\2\2\2\u0a70\u0a71\7H\2\2\u0a71\u0a72\6\u0129\4\2\u0a72"+ - "\u0a74\3\2\2\2\u0a73\u0a63\3\2\2\2\u0a73\u0a6c\3\2\2\2\u0a74\u0252\3\2"+ - "\2\2\u0a75\u0a77\5\u025f\u0130\2\u0a76\u0a75\3\2\2\2\u0a77\u0a78\3\2\2"+ - "\2\u0a78\u0a76\3\2\2\2\u0a78\u0a79\3\2\2\2\u0a79\u0a7b\3\2\2\2\u0a7a\u0a7c"+ - "\5\u025d\u012f\2\u0a7b\u0a7a\3\2\2\2\u0a7b\u0a7c\3\2\2\2\u0a7c\u0a7d\3"+ - "\2\2\2\u0a7d\u0a7e\7F\2\2\u0a7e\u0a87\3\2\2\2\u0a7f\u0a81\5\u025b\u012e"+ - "\2\u0a80\u0a82\5\u025d\u012f\2\u0a81\u0a80\3\2\2\2\u0a81\u0a82\3\2\2\2"+ - "\u0a82\u0a83\3\2\2\2\u0a83\u0a84\7F\2\2\u0a84\u0a85\6\u012a\5\2\u0a85"+ - "\u0a87\3\2\2\2\u0a86\u0a76\3\2\2\2\u0a86\u0a7f\3\2\2\2\u0a87\u0254\3\2"+ - "\2\2\u0a88\u0a8a\5\u025f\u0130\2\u0a89\u0a88\3\2\2\2\u0a8a\u0a8b\3\2\2"+ - "\2\u0a8b\u0a89\3\2\2\2\u0a8b\u0a8c\3\2\2\2\u0a8c\u0a8e\3\2\2\2\u0a8d\u0a8f"+ - "\5\u025d\u012f\2\u0a8e\u0a8d\3\2\2\2\u0a8e\u0a8f\3\2\2\2\u0a8f\u0a90\3"+ - "\2\2\2\u0a90\u0a91\7D\2\2\u0a91\u0a92\7F\2\2\u0a92\u0a9d\3\2\2\2\u0a93"+ - "\u0a95\5\u025b\u012e\2\u0a94\u0a96\5\u025d\u012f\2\u0a95\u0a94\3\2\2\2"+ - "\u0a95\u0a96\3\2\2\2\u0a96\u0a97\3\2\2\2\u0a97\u0a98\7D\2\2\u0a98\u0a99"+ - "\7F\2\2\u0a99\u0a9a\3\2\2\2\u0a9a\u0a9b\6\u012b\6\2\u0a9b\u0a9d\3\2\2"+ - "\2\u0a9c\u0a89\3\2\2\2\u0a9c\u0a93\3\2\2\2\u0a9d\u0256\3\2\2\2\u0a9e\u0aa2"+ - "\5\u0261\u0131\2\u0a9f\u0aa2\5\u025f\u0130\2\u0aa0\u0aa2\7a\2\2\u0aa1"+ - "\u0a9e\3\2\2\2\u0aa1\u0a9f\3\2\2\2\u0aa1\u0aa0\3\2\2\2\u0aa2\u0aa3\3\2"+ - "\2\2\u0aa3\u0aa1\3\2\2\2\u0aa3\u0aa4\3\2\2\2\u0aa4\u0258\3\2\2\2\u0aa5"+ - "\u0aab\7b\2\2\u0aa6\u0aaa\n\4\2\2\u0aa7\u0aa8\7b\2\2\u0aa8\u0aaa\7b\2"+ - "\2\u0aa9\u0aa6\3\2\2\2\u0aa9\u0aa7\3\2\2\2\u0aaa\u0aad\3\2\2\2\u0aab\u0aa9"+ - "\3\2\2\2\u0aab\u0aac\3\2\2\2\u0aac\u0aae\3\2\2\2\u0aad\u0aab\3\2\2\2\u0aae"+ - "\u0aaf\7b\2\2\u0aaf\u025a\3\2\2\2\u0ab0\u0ab2\5\u025f\u0130\2\u0ab1\u0ab0"+ - "\3\2\2\2\u0ab2\u0ab3\3\2\2\2\u0ab3\u0ab1\3\2\2\2\u0ab3\u0ab4\3\2\2\2\u0ab4"+ - "\u0ab5\3\2\2\2\u0ab5\u0ab9\7\60\2\2\u0ab6\u0ab8\5\u025f\u0130\2\u0ab7"+ - "\u0ab6\3\2\2\2\u0ab8\u0abb\3\2\2\2\u0ab9\u0ab7\3\2\2\2\u0ab9\u0aba\3\2"+ - "\2\2\u0aba\u0ac3\3\2\2\2\u0abb\u0ab9\3\2\2\2\u0abc\u0abe\7\60\2\2\u0abd"+ - "\u0abf\5\u025f\u0130\2\u0abe\u0abd\3\2\2\2\u0abf\u0ac0\3\2\2\2\u0ac0\u0abe"+ - "\3\2\2\2\u0ac0\u0ac1\3\2\2\2\u0ac1\u0ac3\3\2\2\2\u0ac2\u0ab1\3\2\2\2\u0ac2"+ - "\u0abc\3\2\2\2\u0ac3\u025c\3\2\2\2\u0ac4\u0ac6\7G\2\2\u0ac5\u0ac7\t\5"+ - "\2\2\u0ac6\u0ac5\3\2\2\2\u0ac6\u0ac7\3\2\2\2\u0ac7\u0ac9\3\2\2\2\u0ac8"+ - "\u0aca\5\u025f\u0130\2\u0ac9\u0ac8\3\2\2\2\u0aca\u0acb\3\2\2\2\u0acb\u0ac9"+ - "\3\2\2\2\u0acb\u0acc\3\2\2\2\u0acc\u025e\3\2\2\2\u0acd\u0ace\t\6\2\2\u0ace"+ - "\u0260\3\2\2\2\u0acf\u0ad0\t\7\2\2\u0ad0\u0262\3\2\2\2\u0ad1\u0ad2\7/"+ - "\2\2\u0ad2\u0ad3\7/\2\2\u0ad3\u0ad9\3\2\2\2\u0ad4\u0ad5\7^\2\2\u0ad5\u0ad8"+ - "\7\f\2\2\u0ad6\u0ad8\n\b\2\2\u0ad7\u0ad4\3\2\2\2\u0ad7\u0ad6\3\2\2\2\u0ad8"+ - "\u0adb\3\2\2\2\u0ad9\u0ad7\3\2\2\2\u0ad9\u0ada\3\2\2\2\u0ada\u0add\3\2"+ - "\2\2\u0adb\u0ad9\3\2\2\2\u0adc\u0ade\7\17\2\2\u0add\u0adc\3\2\2\2\u0add"+ - "\u0ade\3\2\2\2\u0ade\u0ae0\3\2\2\2\u0adf\u0ae1\7\f\2\2\u0ae0\u0adf\3\2"+ - "\2\2\u0ae0\u0ae1\3\2\2\2\u0ae1\u0ae2\3\2\2\2\u0ae2\u0ae3\b\u0132\2\2\u0ae3"+ - "\u0264\3\2\2\2\u0ae4\u0ae5\7\61\2\2\u0ae5\u0ae6\7,\2\2\u0ae6\u0ae7\3\2"+ - "\2\2\u0ae7\u0aec\6\u0133\7\2\u0ae8\u0aeb\5\u0265\u0133\2\u0ae9\u0aeb\13"+ - "\2\2\2\u0aea\u0ae8\3\2\2\2\u0aea\u0ae9\3\2\2\2\u0aeb\u0aee\3\2\2\2\u0aec"+ - "\u0aed\3\2\2\2\u0aec\u0aea\3\2\2\2\u0aed\u0af3\3\2\2\2\u0aee\u0aec\3\2"+ - "\2\2\u0aef\u0af0\7,\2\2\u0af0\u0af4\7\61\2\2\u0af1\u0af2\b\u0133\3\2\u0af2"+ - "\u0af4\7\2\2\3\u0af3\u0aef\3\2\2\2\u0af3\u0af1\3\2\2\2\u0af4\u0af5\3\2"+ - "\2\2\u0af5\u0af6\b\u0133\2\2\u0af6\u0266\3\2\2\2\u0af7\u0af9\t\t\2\2\u0af8"+ - "\u0af7\3\2\2\2\u0af9\u0afa\3\2\2\2\u0afa\u0af8\3\2\2\2\u0afa\u0afb\3\2"+ - "\2\2\u0afb\u0afc\3\2\2\2\u0afc\u0afd\b\u0134\2\2\u0afd\u0268\3\2\2\2\u0afe"+ - "\u0aff\13\2\2\2\u0aff\u026a\3\2\2\2\63\2\u0416\u0689\u07eb\u08f6\u09f1"+ - "\u0a03\u0a0b\u0a26\u0a28\u0a30\u0a32\u0a36\u0a3b\u0a42\u0a49\u0a50\u0a55"+ - "\u0a5d\u0a65\u0a68\u0a6e\u0a73\u0a78\u0a7b\u0a81\u0a86\u0a8b\u0a8e\u0a95"+ - "\u0a9c\u0aa1\u0aa3\u0aa9\u0aab\u0ab3\u0ab9\u0ac0\u0ac2\u0ac6\u0acb\u0ad7"+ - "\u0ad9\u0add\u0ae0\u0aea\u0aec\u0af3\u0afa\4\2\3\2\3\u0133\2"; - public static final String _serializedATN = Utils.join( - new String[] { - _serializedATNSegment0, - _serializedATNSegment1 - }, - "" - ); - public static final ATN _ATN = - new ATNDeserializer().deserialize(_serializedATN.toCharArray()); - static { - _decisionToDFA = new DFA[_ATN.getNumberOfDecisions()]; - for (int i = 0; i < _ATN.getNumberOfDecisions(); i++) { - _decisionToDFA[i] = new DFA(_ATN.getDecisionState(i), i); - } - } -} \ No newline at end of file diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendListener.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendListener.java deleted file mode 100644 index e4a74f3213..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendListener.java +++ /dev/null @@ -1,1912 +0,0 @@ -// Generated from org/apache/amoro/spark/sql/parser/MixedFormatSqlExtend.g4 by ANTLR 4.8 -package org.apache.amoro.spark.sql.parser; -import org.antlr.v4.runtime.tree.ParseTreeListener; - -/** - * This interface defines a complete listener for a parse tree produced by - * {@link MixedFormatSqlExtendParser}. - */ -public interface MixedFormatSqlExtendListener extends ParseTreeListener { - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#extendStatement}. - * @param ctx the parse tree - */ - void enterExtendStatement(MixedFormatSqlExtendParser.ExtendStatementContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#extendStatement}. - * @param ctx the parse tree - */ - void exitExtendStatement(MixedFormatSqlExtendParser.ExtendStatementContext ctx); - /** - * Enter a parse tree produced by the {@code createTableWithPk} - * labeled alternative in {@link MixedFormatSqlExtendParser#statement}. - * @param ctx the parse tree - */ - void enterCreateTableWithPk(MixedFormatSqlExtendParser.CreateTableWithPkContext ctx); - /** - * Exit a parse tree produced by the {@code createTableWithPk} - * labeled alternative in {@link MixedFormatSqlExtendParser#statement}. - * @param ctx the parse tree - */ - void exitCreateTableWithPk(MixedFormatSqlExtendParser.CreateTableWithPkContext ctx); - /** - * Enter a parse tree produced by the {@code explain} - * labeled alternative in {@link MixedFormatSqlExtendParser#statement}. - * @param ctx the parse tree - */ - void enterExplain(MixedFormatSqlExtendParser.ExplainContext ctx); - /** - * Exit a parse tree produced by the {@code explain} - * labeled alternative in {@link MixedFormatSqlExtendParser#statement}. - * @param ctx the parse tree - */ - void exitExplain(MixedFormatSqlExtendParser.ExplainContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#createTableHeader}. - * @param ctx the parse tree - */ - void enterCreateTableHeader(MixedFormatSqlExtendParser.CreateTableHeaderContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#createTableHeader}. - * @param ctx the parse tree - */ - void exitCreateTableHeader(MixedFormatSqlExtendParser.CreateTableHeaderContext ctx); - /** - * Enter a parse tree produced by the {@code colListWithPk} - * labeled alternative in {@link MixedFormatSqlExtendParser#colListAndPk}. - * @param ctx the parse tree - */ - void enterColListWithPk(MixedFormatSqlExtendParser.ColListWithPkContext ctx); - /** - * Exit a parse tree produced by the {@code colListWithPk} - * labeled alternative in {@link MixedFormatSqlExtendParser#colListAndPk}. - * @param ctx the parse tree - */ - void exitColListWithPk(MixedFormatSqlExtendParser.ColListWithPkContext ctx); - /** - * Enter a parse tree produced by the {@code colListOnlyPk} - * labeled alternative in {@link MixedFormatSqlExtendParser#colListAndPk}. - * @param ctx the parse tree - */ - void enterColListOnlyPk(MixedFormatSqlExtendParser.ColListOnlyPkContext ctx); - /** - * Exit a parse tree produced by the {@code colListOnlyPk} - * labeled alternative in {@link MixedFormatSqlExtendParser#colListAndPk}. - * @param ctx the parse tree - */ - void exitColListOnlyPk(MixedFormatSqlExtendParser.ColListOnlyPkContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#primarySpec}. - * @param ctx the parse tree - */ - void enterPrimarySpec(MixedFormatSqlExtendParser.PrimarySpecContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#primarySpec}. - * @param ctx the parse tree - */ - void exitPrimarySpec(MixedFormatSqlExtendParser.PrimarySpecContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#bucketSpec}. - * @param ctx the parse tree - */ - void enterBucketSpec(MixedFormatSqlExtendParser.BucketSpecContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#bucketSpec}. - * @param ctx the parse tree - */ - void exitBucketSpec(MixedFormatSqlExtendParser.BucketSpecContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#skewSpec}. - * @param ctx the parse tree - */ - void enterSkewSpec(MixedFormatSqlExtendParser.SkewSpecContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#skewSpec}. - * @param ctx the parse tree - */ - void exitSkewSpec(MixedFormatSqlExtendParser.SkewSpecContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#locationSpec}. - * @param ctx the parse tree - */ - void enterLocationSpec(MixedFormatSqlExtendParser.LocationSpecContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#locationSpec}. - * @param ctx the parse tree - */ - void exitLocationSpec(MixedFormatSqlExtendParser.LocationSpecContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#commentSpec}. - * @param ctx the parse tree - */ - void enterCommentSpec(MixedFormatSqlExtendParser.CommentSpecContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#commentSpec}. - * @param ctx the parse tree - */ - void exitCommentSpec(MixedFormatSqlExtendParser.CommentSpecContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#query}. - * @param ctx the parse tree - */ - void enterQuery(MixedFormatSqlExtendParser.QueryContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#query}. - * @param ctx the parse tree - */ - void exitQuery(MixedFormatSqlExtendParser.QueryContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#ctes}. - * @param ctx the parse tree - */ - void enterCtes(MixedFormatSqlExtendParser.CtesContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#ctes}. - * @param ctx the parse tree - */ - void exitCtes(MixedFormatSqlExtendParser.CtesContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#namedQuery}. - * @param ctx the parse tree - */ - void enterNamedQuery(MixedFormatSqlExtendParser.NamedQueryContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#namedQuery}. - * @param ctx the parse tree - */ - void exitNamedQuery(MixedFormatSqlExtendParser.NamedQueryContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#tableProvider}. - * @param ctx the parse tree - */ - void enterTableProvider(MixedFormatSqlExtendParser.TableProviderContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#tableProvider}. - * @param ctx the parse tree - */ - void exitTableProvider(MixedFormatSqlExtendParser.TableProviderContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#createTableClauses}. - * @param ctx the parse tree - */ - void enterCreateTableClauses(MixedFormatSqlExtendParser.CreateTableClausesContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#createTableClauses}. - * @param ctx the parse tree - */ - void exitCreateTableClauses(MixedFormatSqlExtendParser.CreateTableClausesContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#tablePropertyList}. - * @param ctx the parse tree - */ - void enterTablePropertyList(MixedFormatSqlExtendParser.TablePropertyListContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#tablePropertyList}. - * @param ctx the parse tree - */ - void exitTablePropertyList(MixedFormatSqlExtendParser.TablePropertyListContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#tableProperty}. - * @param ctx the parse tree - */ - void enterTableProperty(MixedFormatSqlExtendParser.TablePropertyContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#tableProperty}. - * @param ctx the parse tree - */ - void exitTableProperty(MixedFormatSqlExtendParser.TablePropertyContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#tablePropertyKey}. - * @param ctx the parse tree - */ - void enterTablePropertyKey(MixedFormatSqlExtendParser.TablePropertyKeyContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#tablePropertyKey}. - * @param ctx the parse tree - */ - void exitTablePropertyKey(MixedFormatSqlExtendParser.TablePropertyKeyContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#tablePropertyValue}. - * @param ctx the parse tree - */ - void enterTablePropertyValue(MixedFormatSqlExtendParser.TablePropertyValueContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#tablePropertyValue}. - * @param ctx the parse tree - */ - void exitTablePropertyValue(MixedFormatSqlExtendParser.TablePropertyValueContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#constantList}. - * @param ctx the parse tree - */ - void enterConstantList(MixedFormatSqlExtendParser.ConstantListContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#constantList}. - * @param ctx the parse tree - */ - void exitConstantList(MixedFormatSqlExtendParser.ConstantListContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#nestedConstantList}. - * @param ctx the parse tree - */ - void enterNestedConstantList(MixedFormatSqlExtendParser.NestedConstantListContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#nestedConstantList}. - * @param ctx the parse tree - */ - void exitNestedConstantList(MixedFormatSqlExtendParser.NestedConstantListContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#createFileFormat}. - * @param ctx the parse tree - */ - void enterCreateFileFormat(MixedFormatSqlExtendParser.CreateFileFormatContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#createFileFormat}. - * @param ctx the parse tree - */ - void exitCreateFileFormat(MixedFormatSqlExtendParser.CreateFileFormatContext ctx); - /** - * Enter a parse tree produced by the {@code tableFileFormat} - * labeled alternative in {@link MixedFormatSqlExtendParser#fileFormat}. - * @param ctx the parse tree - */ - void enterTableFileFormat(MixedFormatSqlExtendParser.TableFileFormatContext ctx); - /** - * Exit a parse tree produced by the {@code tableFileFormat} - * labeled alternative in {@link MixedFormatSqlExtendParser#fileFormat}. - * @param ctx the parse tree - */ - void exitTableFileFormat(MixedFormatSqlExtendParser.TableFileFormatContext ctx); - /** - * Enter a parse tree produced by the {@code genericFileFormat} - * labeled alternative in {@link MixedFormatSqlExtendParser#fileFormat}. - * @param ctx the parse tree - */ - void enterGenericFileFormat(MixedFormatSqlExtendParser.GenericFileFormatContext ctx); - /** - * Exit a parse tree produced by the {@code genericFileFormat} - * labeled alternative in {@link MixedFormatSqlExtendParser#fileFormat}. - * @param ctx the parse tree - */ - void exitGenericFileFormat(MixedFormatSqlExtendParser.GenericFileFormatContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#storageHandler}. - * @param ctx the parse tree - */ - void enterStorageHandler(MixedFormatSqlExtendParser.StorageHandlerContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#storageHandler}. - * @param ctx the parse tree - */ - void exitStorageHandler(MixedFormatSqlExtendParser.StorageHandlerContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#queryOrganization}. - * @param ctx the parse tree - */ - void enterQueryOrganization(MixedFormatSqlExtendParser.QueryOrganizationContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#queryOrganization}. - * @param ctx the parse tree - */ - void exitQueryOrganization(MixedFormatSqlExtendParser.QueryOrganizationContext ctx); - /** - * Enter a parse tree produced by the {@code queryTermDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryTerm}. - * @param ctx the parse tree - */ - void enterQueryTermDefault(MixedFormatSqlExtendParser.QueryTermDefaultContext ctx); - /** - * Exit a parse tree produced by the {@code queryTermDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryTerm}. - * @param ctx the parse tree - */ - void exitQueryTermDefault(MixedFormatSqlExtendParser.QueryTermDefaultContext ctx); - /** - * Enter a parse tree produced by the {@code setOperation} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryTerm}. - * @param ctx the parse tree - */ - void enterSetOperation(MixedFormatSqlExtendParser.SetOperationContext ctx); - /** - * Exit a parse tree produced by the {@code setOperation} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryTerm}. - * @param ctx the parse tree - */ - void exitSetOperation(MixedFormatSqlExtendParser.SetOperationContext ctx); - /** - * Enter a parse tree produced by the {@code queryPrimaryDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - */ - void enterQueryPrimaryDefault(MixedFormatSqlExtendParser.QueryPrimaryDefaultContext ctx); - /** - * Exit a parse tree produced by the {@code queryPrimaryDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - */ - void exitQueryPrimaryDefault(MixedFormatSqlExtendParser.QueryPrimaryDefaultContext ctx); - /** - * Enter a parse tree produced by the {@code fromStmt} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - */ - void enterFromStmt(MixedFormatSqlExtendParser.FromStmtContext ctx); - /** - * Exit a parse tree produced by the {@code fromStmt} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - */ - void exitFromStmt(MixedFormatSqlExtendParser.FromStmtContext ctx); - /** - * Enter a parse tree produced by the {@code table} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - */ - void enterTable(MixedFormatSqlExtendParser.TableContext ctx); - /** - * Exit a parse tree produced by the {@code table} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - */ - void exitTable(MixedFormatSqlExtendParser.TableContext ctx); - /** - * Enter a parse tree produced by the {@code inlineTableDefault1} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - */ - void enterInlineTableDefault1(MixedFormatSqlExtendParser.InlineTableDefault1Context ctx); - /** - * Exit a parse tree produced by the {@code inlineTableDefault1} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - */ - void exitInlineTableDefault1(MixedFormatSqlExtendParser.InlineTableDefault1Context ctx); - /** - * Enter a parse tree produced by the {@code subquery} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - */ - void enterSubquery(MixedFormatSqlExtendParser.SubqueryContext ctx); - /** - * Exit a parse tree produced by the {@code subquery} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - */ - void exitSubquery(MixedFormatSqlExtendParser.SubqueryContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#sortItem}. - * @param ctx the parse tree - */ - void enterSortItem(MixedFormatSqlExtendParser.SortItemContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#sortItem}. - * @param ctx the parse tree - */ - void exitSortItem(MixedFormatSqlExtendParser.SortItemContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#fromStatement}. - * @param ctx the parse tree - */ - void enterFromStatement(MixedFormatSqlExtendParser.FromStatementContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#fromStatement}. - * @param ctx the parse tree - */ - void exitFromStatement(MixedFormatSqlExtendParser.FromStatementContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#fromStatementBody}. - * @param ctx the parse tree - */ - void enterFromStatementBody(MixedFormatSqlExtendParser.FromStatementBodyContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#fromStatementBody}. - * @param ctx the parse tree - */ - void exitFromStatementBody(MixedFormatSqlExtendParser.FromStatementBodyContext ctx); - /** - * Enter a parse tree produced by the {@code transformQuerySpecification} - * labeled alternative in {@link MixedFormatSqlExtendParser#querySpecification}. - * @param ctx the parse tree - */ - void enterTransformQuerySpecification(MixedFormatSqlExtendParser.TransformQuerySpecificationContext ctx); - /** - * Exit a parse tree produced by the {@code transformQuerySpecification} - * labeled alternative in {@link MixedFormatSqlExtendParser#querySpecification}. - * @param ctx the parse tree - */ - void exitTransformQuerySpecification(MixedFormatSqlExtendParser.TransformQuerySpecificationContext ctx); - /** - * Enter a parse tree produced by the {@code regularQuerySpecification} - * labeled alternative in {@link MixedFormatSqlExtendParser#querySpecification}. - * @param ctx the parse tree - */ - void enterRegularQuerySpecification(MixedFormatSqlExtendParser.RegularQuerySpecificationContext ctx); - /** - * Exit a parse tree produced by the {@code regularQuerySpecification} - * labeled alternative in {@link MixedFormatSqlExtendParser#querySpecification}. - * @param ctx the parse tree - */ - void exitRegularQuerySpecification(MixedFormatSqlExtendParser.RegularQuerySpecificationContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#transformClause}. - * @param ctx the parse tree - */ - void enterTransformClause(MixedFormatSqlExtendParser.TransformClauseContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#transformClause}. - * @param ctx the parse tree - */ - void exitTransformClause(MixedFormatSqlExtendParser.TransformClauseContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#selectClause}. - * @param ctx the parse tree - */ - void enterSelectClause(MixedFormatSqlExtendParser.SelectClauseContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#selectClause}. - * @param ctx the parse tree - */ - void exitSelectClause(MixedFormatSqlExtendParser.SelectClauseContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#whereClause}. - * @param ctx the parse tree - */ - void enterWhereClause(MixedFormatSqlExtendParser.WhereClauseContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#whereClause}. - * @param ctx the parse tree - */ - void exitWhereClause(MixedFormatSqlExtendParser.WhereClauseContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#havingClause}. - * @param ctx the parse tree - */ - void enterHavingClause(MixedFormatSqlExtendParser.HavingClauseContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#havingClause}. - * @param ctx the parse tree - */ - void exitHavingClause(MixedFormatSqlExtendParser.HavingClauseContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#hint}. - * @param ctx the parse tree - */ - void enterHint(MixedFormatSqlExtendParser.HintContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#hint}. - * @param ctx the parse tree - */ - void exitHint(MixedFormatSqlExtendParser.HintContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#hintStatement}. - * @param ctx the parse tree - */ - void enterHintStatement(MixedFormatSqlExtendParser.HintStatementContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#hintStatement}. - * @param ctx the parse tree - */ - void exitHintStatement(MixedFormatSqlExtendParser.HintStatementContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#fromClause}. - * @param ctx the parse tree - */ - void enterFromClause(MixedFormatSqlExtendParser.FromClauseContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#fromClause}. - * @param ctx the parse tree - */ - void exitFromClause(MixedFormatSqlExtendParser.FromClauseContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#aggregationClause}. - * @param ctx the parse tree - */ - void enterAggregationClause(MixedFormatSqlExtendParser.AggregationClauseContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#aggregationClause}. - * @param ctx the parse tree - */ - void exitAggregationClause(MixedFormatSqlExtendParser.AggregationClauseContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#groupByClause}. - * @param ctx the parse tree - */ - void enterGroupByClause(MixedFormatSqlExtendParser.GroupByClauseContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#groupByClause}. - * @param ctx the parse tree - */ - void exitGroupByClause(MixedFormatSqlExtendParser.GroupByClauseContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#groupingAnalytics}. - * @param ctx the parse tree - */ - void enterGroupingAnalytics(MixedFormatSqlExtendParser.GroupingAnalyticsContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#groupingAnalytics}. - * @param ctx the parse tree - */ - void exitGroupingAnalytics(MixedFormatSqlExtendParser.GroupingAnalyticsContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#groupingElement}. - * @param ctx the parse tree - */ - void enterGroupingElement(MixedFormatSqlExtendParser.GroupingElementContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#groupingElement}. - * @param ctx the parse tree - */ - void exitGroupingElement(MixedFormatSqlExtendParser.GroupingElementContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#groupingSet}. - * @param ctx the parse tree - */ - void enterGroupingSet(MixedFormatSqlExtendParser.GroupingSetContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#groupingSet}. - * @param ctx the parse tree - */ - void exitGroupingSet(MixedFormatSqlExtendParser.GroupingSetContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#pivotClause}. - * @param ctx the parse tree - */ - void enterPivotClause(MixedFormatSqlExtendParser.PivotClauseContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#pivotClause}. - * @param ctx the parse tree - */ - void exitPivotClause(MixedFormatSqlExtendParser.PivotClauseContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#pivotColumn}. - * @param ctx the parse tree - */ - void enterPivotColumn(MixedFormatSqlExtendParser.PivotColumnContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#pivotColumn}. - * @param ctx the parse tree - */ - void exitPivotColumn(MixedFormatSqlExtendParser.PivotColumnContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#pivotValue}. - * @param ctx the parse tree - */ - void enterPivotValue(MixedFormatSqlExtendParser.PivotValueContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#pivotValue}. - * @param ctx the parse tree - */ - void exitPivotValue(MixedFormatSqlExtendParser.PivotValueContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#lateralView}. - * @param ctx the parse tree - */ - void enterLateralView(MixedFormatSqlExtendParser.LateralViewContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#lateralView}. - * @param ctx the parse tree - */ - void exitLateralView(MixedFormatSqlExtendParser.LateralViewContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#setQuantifier}. - * @param ctx the parse tree - */ - void enterSetQuantifier(MixedFormatSqlExtendParser.SetQuantifierContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#setQuantifier}. - * @param ctx the parse tree - */ - void exitSetQuantifier(MixedFormatSqlExtendParser.SetQuantifierContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#relation}. - * @param ctx the parse tree - */ - void enterRelation(MixedFormatSqlExtendParser.RelationContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#relation}. - * @param ctx the parse tree - */ - void exitRelation(MixedFormatSqlExtendParser.RelationContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#joinRelation}. - * @param ctx the parse tree - */ - void enterJoinRelation(MixedFormatSqlExtendParser.JoinRelationContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#joinRelation}. - * @param ctx the parse tree - */ - void exitJoinRelation(MixedFormatSqlExtendParser.JoinRelationContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#joinType}. - * @param ctx the parse tree - */ - void enterJoinType(MixedFormatSqlExtendParser.JoinTypeContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#joinType}. - * @param ctx the parse tree - */ - void exitJoinType(MixedFormatSqlExtendParser.JoinTypeContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#joinCriteria}. - * @param ctx the parse tree - */ - void enterJoinCriteria(MixedFormatSqlExtendParser.JoinCriteriaContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#joinCriteria}. - * @param ctx the parse tree - */ - void exitJoinCriteria(MixedFormatSqlExtendParser.JoinCriteriaContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#sample}. - * @param ctx the parse tree - */ - void enterSample(MixedFormatSqlExtendParser.SampleContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#sample}. - * @param ctx the parse tree - */ - void exitSample(MixedFormatSqlExtendParser.SampleContext ctx); - /** - * Enter a parse tree produced by the {@code sampleByPercentile} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - */ - void enterSampleByPercentile(MixedFormatSqlExtendParser.SampleByPercentileContext ctx); - /** - * Exit a parse tree produced by the {@code sampleByPercentile} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - */ - void exitSampleByPercentile(MixedFormatSqlExtendParser.SampleByPercentileContext ctx); - /** - * Enter a parse tree produced by the {@code sampleByRows} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - */ - void enterSampleByRows(MixedFormatSqlExtendParser.SampleByRowsContext ctx); - /** - * Exit a parse tree produced by the {@code sampleByRows} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - */ - void exitSampleByRows(MixedFormatSqlExtendParser.SampleByRowsContext ctx); - /** - * Enter a parse tree produced by the {@code sampleByBucket} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - */ - void enterSampleByBucket(MixedFormatSqlExtendParser.SampleByBucketContext ctx); - /** - * Exit a parse tree produced by the {@code sampleByBucket} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - */ - void exitSampleByBucket(MixedFormatSqlExtendParser.SampleByBucketContext ctx); - /** - * Enter a parse tree produced by the {@code sampleByBytes} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - */ - void enterSampleByBytes(MixedFormatSqlExtendParser.SampleByBytesContext ctx); - /** - * Exit a parse tree produced by the {@code sampleByBytes} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - */ - void exitSampleByBytes(MixedFormatSqlExtendParser.SampleByBytesContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#identifierList}. - * @param ctx the parse tree - */ - void enterIdentifierList(MixedFormatSqlExtendParser.IdentifierListContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#identifierList}. - * @param ctx the parse tree - */ - void exitIdentifierList(MixedFormatSqlExtendParser.IdentifierListContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#identifierSeq}. - * @param ctx the parse tree - */ - void enterIdentifierSeq(MixedFormatSqlExtendParser.IdentifierSeqContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#identifierSeq}. - * @param ctx the parse tree - */ - void exitIdentifierSeq(MixedFormatSqlExtendParser.IdentifierSeqContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#orderedIdentifierList}. - * @param ctx the parse tree - */ - void enterOrderedIdentifierList(MixedFormatSqlExtendParser.OrderedIdentifierListContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#orderedIdentifierList}. - * @param ctx the parse tree - */ - void exitOrderedIdentifierList(MixedFormatSqlExtendParser.OrderedIdentifierListContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#orderedIdentifier}. - * @param ctx the parse tree - */ - void enterOrderedIdentifier(MixedFormatSqlExtendParser.OrderedIdentifierContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#orderedIdentifier}. - * @param ctx the parse tree - */ - void exitOrderedIdentifier(MixedFormatSqlExtendParser.OrderedIdentifierContext ctx); - /** - * Enter a parse tree produced by the {@code tableName} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - */ - void enterTableName(MixedFormatSqlExtendParser.TableNameContext ctx); - /** - * Exit a parse tree produced by the {@code tableName} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - */ - void exitTableName(MixedFormatSqlExtendParser.TableNameContext ctx); - /** - * Enter a parse tree produced by the {@code aliasedQuery} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - */ - void enterAliasedQuery(MixedFormatSqlExtendParser.AliasedQueryContext ctx); - /** - * Exit a parse tree produced by the {@code aliasedQuery} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - */ - void exitAliasedQuery(MixedFormatSqlExtendParser.AliasedQueryContext ctx); - /** - * Enter a parse tree produced by the {@code aliasedRelation} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - */ - void enterAliasedRelation(MixedFormatSqlExtendParser.AliasedRelationContext ctx); - /** - * Exit a parse tree produced by the {@code aliasedRelation} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - */ - void exitAliasedRelation(MixedFormatSqlExtendParser.AliasedRelationContext ctx); - /** - * Enter a parse tree produced by the {@code inlineTableDefault2} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - */ - void enterInlineTableDefault2(MixedFormatSqlExtendParser.InlineTableDefault2Context ctx); - /** - * Exit a parse tree produced by the {@code inlineTableDefault2} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - */ - void exitInlineTableDefault2(MixedFormatSqlExtendParser.InlineTableDefault2Context ctx); - /** - * Enter a parse tree produced by the {@code tableValuedFunction} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - */ - void enterTableValuedFunction(MixedFormatSqlExtendParser.TableValuedFunctionContext ctx); - /** - * Exit a parse tree produced by the {@code tableValuedFunction} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - */ - void exitTableValuedFunction(MixedFormatSqlExtendParser.TableValuedFunctionContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#inlineTable}. - * @param ctx the parse tree - */ - void enterInlineTable(MixedFormatSqlExtendParser.InlineTableContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#inlineTable}. - * @param ctx the parse tree - */ - void exitInlineTable(MixedFormatSqlExtendParser.InlineTableContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#functionTable}. - * @param ctx the parse tree - */ - void enterFunctionTable(MixedFormatSqlExtendParser.FunctionTableContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#functionTable}. - * @param ctx the parse tree - */ - void exitFunctionTable(MixedFormatSqlExtendParser.FunctionTableContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#tableAlias}. - * @param ctx the parse tree - */ - void enterTableAlias(MixedFormatSqlExtendParser.TableAliasContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#tableAlias}. - * @param ctx the parse tree - */ - void exitTableAlias(MixedFormatSqlExtendParser.TableAliasContext ctx); - /** - * Enter a parse tree produced by the {@code rowFormatSerde} - * labeled alternative in {@link MixedFormatSqlExtendParser#rowFormat}. - * @param ctx the parse tree - */ - void enterRowFormatSerde(MixedFormatSqlExtendParser.RowFormatSerdeContext ctx); - /** - * Exit a parse tree produced by the {@code rowFormatSerde} - * labeled alternative in {@link MixedFormatSqlExtendParser#rowFormat}. - * @param ctx the parse tree - */ - void exitRowFormatSerde(MixedFormatSqlExtendParser.RowFormatSerdeContext ctx); - /** - * Enter a parse tree produced by the {@code rowFormatDelimited} - * labeled alternative in {@link MixedFormatSqlExtendParser#rowFormat}. - * @param ctx the parse tree - */ - void enterRowFormatDelimited(MixedFormatSqlExtendParser.RowFormatDelimitedContext ctx); - /** - * Exit a parse tree produced by the {@code rowFormatDelimited} - * labeled alternative in {@link MixedFormatSqlExtendParser#rowFormat}. - * @param ctx the parse tree - */ - void exitRowFormatDelimited(MixedFormatSqlExtendParser.RowFormatDelimitedContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#multipartIdentifier}. - * @param ctx the parse tree - */ - void enterMultipartIdentifier(MixedFormatSqlExtendParser.MultipartIdentifierContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#multipartIdentifier}. - * @param ctx the parse tree - */ - void exitMultipartIdentifier(MixedFormatSqlExtendParser.MultipartIdentifierContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#namedExpression}. - * @param ctx the parse tree - */ - void enterNamedExpression(MixedFormatSqlExtendParser.NamedExpressionContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#namedExpression}. - * @param ctx the parse tree - */ - void exitNamedExpression(MixedFormatSqlExtendParser.NamedExpressionContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#namedExpressionSeq}. - * @param ctx the parse tree - */ - void enterNamedExpressionSeq(MixedFormatSqlExtendParser.NamedExpressionSeqContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#namedExpressionSeq}. - * @param ctx the parse tree - */ - void exitNamedExpressionSeq(MixedFormatSqlExtendParser.NamedExpressionSeqContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#partitionFieldList}. - * @param ctx the parse tree - */ - void enterPartitionFieldList(MixedFormatSqlExtendParser.PartitionFieldListContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#partitionFieldList}. - * @param ctx the parse tree - */ - void exitPartitionFieldList(MixedFormatSqlExtendParser.PartitionFieldListContext ctx); - /** - * Enter a parse tree produced by the {@code partitionTransform} - * labeled alternative in {@link MixedFormatSqlExtendParser#partitionField}. - * @param ctx the parse tree - */ - void enterPartitionTransform(MixedFormatSqlExtendParser.PartitionTransformContext ctx); - /** - * Exit a parse tree produced by the {@code partitionTransform} - * labeled alternative in {@link MixedFormatSqlExtendParser#partitionField}. - * @param ctx the parse tree - */ - void exitPartitionTransform(MixedFormatSqlExtendParser.PartitionTransformContext ctx); - /** - * Enter a parse tree produced by the {@code partitionColumn} - * labeled alternative in {@link MixedFormatSqlExtendParser#partitionField}. - * @param ctx the parse tree - */ - void enterPartitionColumn(MixedFormatSqlExtendParser.PartitionColumnContext ctx); - /** - * Exit a parse tree produced by the {@code partitionColumn} - * labeled alternative in {@link MixedFormatSqlExtendParser#partitionField}. - * @param ctx the parse tree - */ - void exitPartitionColumn(MixedFormatSqlExtendParser.PartitionColumnContext ctx); - /** - * Enter a parse tree produced by the {@code identityTransform} - * labeled alternative in {@link MixedFormatSqlExtendParser#transform}. - * @param ctx the parse tree - */ - void enterIdentityTransform(MixedFormatSqlExtendParser.IdentityTransformContext ctx); - /** - * Exit a parse tree produced by the {@code identityTransform} - * labeled alternative in {@link MixedFormatSqlExtendParser#transform}. - * @param ctx the parse tree - */ - void exitIdentityTransform(MixedFormatSqlExtendParser.IdentityTransformContext ctx); - /** - * Enter a parse tree produced by the {@code applyTransform} - * labeled alternative in {@link MixedFormatSqlExtendParser#transform}. - * @param ctx the parse tree - */ - void enterApplyTransform(MixedFormatSqlExtendParser.ApplyTransformContext ctx); - /** - * Exit a parse tree produced by the {@code applyTransform} - * labeled alternative in {@link MixedFormatSqlExtendParser#transform}. - * @param ctx the parse tree - */ - void exitApplyTransform(MixedFormatSqlExtendParser.ApplyTransformContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#transformArgument}. - * @param ctx the parse tree - */ - void enterTransformArgument(MixedFormatSqlExtendParser.TransformArgumentContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#transformArgument}. - * @param ctx the parse tree - */ - void exitTransformArgument(MixedFormatSqlExtendParser.TransformArgumentContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#expression}. - * @param ctx the parse tree - */ - void enterExpression(MixedFormatSqlExtendParser.ExpressionContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#expression}. - * @param ctx the parse tree - */ - void exitExpression(MixedFormatSqlExtendParser.ExpressionContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#expressionSeq}. - * @param ctx the parse tree - */ - void enterExpressionSeq(MixedFormatSqlExtendParser.ExpressionSeqContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#expressionSeq}. - * @param ctx the parse tree - */ - void exitExpressionSeq(MixedFormatSqlExtendParser.ExpressionSeqContext ctx); - /** - * Enter a parse tree produced by the {@code logicalNot} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - */ - void enterLogicalNot(MixedFormatSqlExtendParser.LogicalNotContext ctx); - /** - * Exit a parse tree produced by the {@code logicalNot} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - */ - void exitLogicalNot(MixedFormatSqlExtendParser.LogicalNotContext ctx); - /** - * Enter a parse tree produced by the {@code predicated} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - */ - void enterPredicated(MixedFormatSqlExtendParser.PredicatedContext ctx); - /** - * Exit a parse tree produced by the {@code predicated} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - */ - void exitPredicated(MixedFormatSqlExtendParser.PredicatedContext ctx); - /** - * Enter a parse tree produced by the {@code exists} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - */ - void enterExists(MixedFormatSqlExtendParser.ExistsContext ctx); - /** - * Exit a parse tree produced by the {@code exists} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - */ - void exitExists(MixedFormatSqlExtendParser.ExistsContext ctx); - /** - * Enter a parse tree produced by the {@code logicalBinary} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - */ - void enterLogicalBinary(MixedFormatSqlExtendParser.LogicalBinaryContext ctx); - /** - * Exit a parse tree produced by the {@code logicalBinary} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - */ - void exitLogicalBinary(MixedFormatSqlExtendParser.LogicalBinaryContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#predicate}. - * @param ctx the parse tree - */ - void enterPredicate(MixedFormatSqlExtendParser.PredicateContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#predicate}. - * @param ctx the parse tree - */ - void exitPredicate(MixedFormatSqlExtendParser.PredicateContext ctx); - /** - * Enter a parse tree produced by the {@code valueExpressionDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - */ - void enterValueExpressionDefault(MixedFormatSqlExtendParser.ValueExpressionDefaultContext ctx); - /** - * Exit a parse tree produced by the {@code valueExpressionDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - */ - void exitValueExpressionDefault(MixedFormatSqlExtendParser.ValueExpressionDefaultContext ctx); - /** - * Enter a parse tree produced by the {@code comparison} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - */ - void enterComparison(MixedFormatSqlExtendParser.ComparisonContext ctx); - /** - * Exit a parse tree produced by the {@code comparison} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - */ - void exitComparison(MixedFormatSqlExtendParser.ComparisonContext ctx); - /** - * Enter a parse tree produced by the {@code arithmeticBinary} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - */ - void enterArithmeticBinary(MixedFormatSqlExtendParser.ArithmeticBinaryContext ctx); - /** - * Exit a parse tree produced by the {@code arithmeticBinary} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - */ - void exitArithmeticBinary(MixedFormatSqlExtendParser.ArithmeticBinaryContext ctx); - /** - * Enter a parse tree produced by the {@code arithmeticUnary} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - */ - void enterArithmeticUnary(MixedFormatSqlExtendParser.ArithmeticUnaryContext ctx); - /** - * Exit a parse tree produced by the {@code arithmeticUnary} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - */ - void exitArithmeticUnary(MixedFormatSqlExtendParser.ArithmeticUnaryContext ctx); - /** - * Enter a parse tree produced by the {@code struct} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterStruct(MixedFormatSqlExtendParser.StructContext ctx); - /** - * Exit a parse tree produced by the {@code struct} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitStruct(MixedFormatSqlExtendParser.StructContext ctx); - /** - * Enter a parse tree produced by the {@code dereference} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterDereference(MixedFormatSqlExtendParser.DereferenceContext ctx); - /** - * Exit a parse tree produced by the {@code dereference} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitDereference(MixedFormatSqlExtendParser.DereferenceContext ctx); - /** - * Enter a parse tree produced by the {@code simpleCase} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterSimpleCase(MixedFormatSqlExtendParser.SimpleCaseContext ctx); - /** - * Exit a parse tree produced by the {@code simpleCase} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitSimpleCase(MixedFormatSqlExtendParser.SimpleCaseContext ctx); - /** - * Enter a parse tree produced by the {@code currentLike} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterCurrentLike(MixedFormatSqlExtendParser.CurrentLikeContext ctx); - /** - * Exit a parse tree produced by the {@code currentLike} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitCurrentLike(MixedFormatSqlExtendParser.CurrentLikeContext ctx); - /** - * Enter a parse tree produced by the {@code columnReference} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterColumnReference(MixedFormatSqlExtendParser.ColumnReferenceContext ctx); - /** - * Exit a parse tree produced by the {@code columnReference} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitColumnReference(MixedFormatSqlExtendParser.ColumnReferenceContext ctx); - /** - * Enter a parse tree produced by the {@code rowConstructor} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterRowConstructor(MixedFormatSqlExtendParser.RowConstructorContext ctx); - /** - * Exit a parse tree produced by the {@code rowConstructor} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitRowConstructor(MixedFormatSqlExtendParser.RowConstructorContext ctx); - /** - * Enter a parse tree produced by the {@code last} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterLast(MixedFormatSqlExtendParser.LastContext ctx); - /** - * Exit a parse tree produced by the {@code last} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitLast(MixedFormatSqlExtendParser.LastContext ctx); - /** - * Enter a parse tree produced by the {@code star} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterStar(MixedFormatSqlExtendParser.StarContext ctx); - /** - * Exit a parse tree produced by the {@code star} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitStar(MixedFormatSqlExtendParser.StarContext ctx); - /** - * Enter a parse tree produced by the {@code overlay} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterOverlay(MixedFormatSqlExtendParser.OverlayContext ctx); - /** - * Exit a parse tree produced by the {@code overlay} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitOverlay(MixedFormatSqlExtendParser.OverlayContext ctx); - /** - * Enter a parse tree produced by the {@code subscript} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterSubscript(MixedFormatSqlExtendParser.SubscriptContext ctx); - /** - * Exit a parse tree produced by the {@code subscript} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitSubscript(MixedFormatSqlExtendParser.SubscriptContext ctx); - /** - * Enter a parse tree produced by the {@code subqueryExpression} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterSubqueryExpression(MixedFormatSqlExtendParser.SubqueryExpressionContext ctx); - /** - * Exit a parse tree produced by the {@code subqueryExpression} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitSubqueryExpression(MixedFormatSqlExtendParser.SubqueryExpressionContext ctx); - /** - * Enter a parse tree produced by the {@code substring} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterSubstring(MixedFormatSqlExtendParser.SubstringContext ctx); - /** - * Exit a parse tree produced by the {@code substring} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitSubstring(MixedFormatSqlExtendParser.SubstringContext ctx); - /** - * Enter a parse tree produced by the {@code cast} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterCast(MixedFormatSqlExtendParser.CastContext ctx); - /** - * Exit a parse tree produced by the {@code cast} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitCast(MixedFormatSqlExtendParser.CastContext ctx); - /** - * Enter a parse tree produced by the {@code constantDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterConstantDefault(MixedFormatSqlExtendParser.ConstantDefaultContext ctx); - /** - * Exit a parse tree produced by the {@code constantDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitConstantDefault(MixedFormatSqlExtendParser.ConstantDefaultContext ctx); - /** - * Enter a parse tree produced by the {@code lambda} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterLambda(MixedFormatSqlExtendParser.LambdaContext ctx); - /** - * Exit a parse tree produced by the {@code lambda} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitLambda(MixedFormatSqlExtendParser.LambdaContext ctx); - /** - * Enter a parse tree produced by the {@code parenthesizedExpression} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterParenthesizedExpression(MixedFormatSqlExtendParser.ParenthesizedExpressionContext ctx); - /** - * Exit a parse tree produced by the {@code parenthesizedExpression} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitParenthesizedExpression(MixedFormatSqlExtendParser.ParenthesizedExpressionContext ctx); - /** - * Enter a parse tree produced by the {@code extract} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterExtract(MixedFormatSqlExtendParser.ExtractContext ctx); - /** - * Exit a parse tree produced by the {@code extract} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitExtract(MixedFormatSqlExtendParser.ExtractContext ctx); - /** - * Enter a parse tree produced by the {@code trim} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterTrim(MixedFormatSqlExtendParser.TrimContext ctx); - /** - * Exit a parse tree produced by the {@code trim} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitTrim(MixedFormatSqlExtendParser.TrimContext ctx); - /** - * Enter a parse tree produced by the {@code functionCall} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterFunctionCall(MixedFormatSqlExtendParser.FunctionCallContext ctx); - /** - * Exit a parse tree produced by the {@code functionCall} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitFunctionCall(MixedFormatSqlExtendParser.FunctionCallContext ctx); - /** - * Enter a parse tree produced by the {@code searchedCase} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterSearchedCase(MixedFormatSqlExtendParser.SearchedCaseContext ctx); - /** - * Exit a parse tree produced by the {@code searchedCase} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitSearchedCase(MixedFormatSqlExtendParser.SearchedCaseContext ctx); - /** - * Enter a parse tree produced by the {@code position} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterPosition(MixedFormatSqlExtendParser.PositionContext ctx); - /** - * Exit a parse tree produced by the {@code position} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitPosition(MixedFormatSqlExtendParser.PositionContext ctx); - /** - * Enter a parse tree produced by the {@code first} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void enterFirst(MixedFormatSqlExtendParser.FirstContext ctx); - /** - * Exit a parse tree produced by the {@code first} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - */ - void exitFirst(MixedFormatSqlExtendParser.FirstContext ctx); - /** - * Enter a parse tree produced by the {@code nullLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void enterNullLiteral(MixedFormatSqlExtendParser.NullLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code nullLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void exitNullLiteral(MixedFormatSqlExtendParser.NullLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code intervalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void enterIntervalLiteral(MixedFormatSqlExtendParser.IntervalLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code intervalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void exitIntervalLiteral(MixedFormatSqlExtendParser.IntervalLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code typeConstructor} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void enterTypeConstructor(MixedFormatSqlExtendParser.TypeConstructorContext ctx); - /** - * Exit a parse tree produced by the {@code typeConstructor} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void exitTypeConstructor(MixedFormatSqlExtendParser.TypeConstructorContext ctx); - /** - * Enter a parse tree produced by the {@code numericLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void enterNumericLiteral(MixedFormatSqlExtendParser.NumericLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code numericLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void exitNumericLiteral(MixedFormatSqlExtendParser.NumericLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code booleanLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void enterBooleanLiteral(MixedFormatSqlExtendParser.BooleanLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code booleanLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void exitBooleanLiteral(MixedFormatSqlExtendParser.BooleanLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code stringLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void enterStringLiteral(MixedFormatSqlExtendParser.StringLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code stringLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - */ - void exitStringLiteral(MixedFormatSqlExtendParser.StringLiteralContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#comparisonOperator}. - * @param ctx the parse tree - */ - void enterComparisonOperator(MixedFormatSqlExtendParser.ComparisonOperatorContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#comparisonOperator}. - * @param ctx the parse tree - */ - void exitComparisonOperator(MixedFormatSqlExtendParser.ComparisonOperatorContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#booleanValue}. - * @param ctx the parse tree - */ - void enterBooleanValue(MixedFormatSqlExtendParser.BooleanValueContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#booleanValue}. - * @param ctx the parse tree - */ - void exitBooleanValue(MixedFormatSqlExtendParser.BooleanValueContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#interval}. - * @param ctx the parse tree - */ - void enterInterval(MixedFormatSqlExtendParser.IntervalContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#interval}. - * @param ctx the parse tree - */ - void exitInterval(MixedFormatSqlExtendParser.IntervalContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#errorCapturingMultiUnitsInterval}. - * @param ctx the parse tree - */ - void enterErrorCapturingMultiUnitsInterval(MixedFormatSqlExtendParser.ErrorCapturingMultiUnitsIntervalContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#errorCapturingMultiUnitsInterval}. - * @param ctx the parse tree - */ - void exitErrorCapturingMultiUnitsInterval(MixedFormatSqlExtendParser.ErrorCapturingMultiUnitsIntervalContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#multiUnitsInterval}. - * @param ctx the parse tree - */ - void enterMultiUnitsInterval(MixedFormatSqlExtendParser.MultiUnitsIntervalContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#multiUnitsInterval}. - * @param ctx the parse tree - */ - void exitMultiUnitsInterval(MixedFormatSqlExtendParser.MultiUnitsIntervalContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#errorCapturingUnitToUnitInterval}. - * @param ctx the parse tree - */ - void enterErrorCapturingUnitToUnitInterval(MixedFormatSqlExtendParser.ErrorCapturingUnitToUnitIntervalContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#errorCapturingUnitToUnitInterval}. - * @param ctx the parse tree - */ - void exitErrorCapturingUnitToUnitInterval(MixedFormatSqlExtendParser.ErrorCapturingUnitToUnitIntervalContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#unitToUnitInterval}. - * @param ctx the parse tree - */ - void enterUnitToUnitInterval(MixedFormatSqlExtendParser.UnitToUnitIntervalContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#unitToUnitInterval}. - * @param ctx the parse tree - */ - void exitUnitToUnitInterval(MixedFormatSqlExtendParser.UnitToUnitIntervalContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#intervalValue}. - * @param ctx the parse tree - */ - void enterIntervalValue(MixedFormatSqlExtendParser.IntervalValueContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#intervalValue}. - * @param ctx the parse tree - */ - void exitIntervalValue(MixedFormatSqlExtendParser.IntervalValueContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#colPosition}. - * @param ctx the parse tree - */ - void enterColPosition(MixedFormatSqlExtendParser.ColPositionContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#colPosition}. - * @param ctx the parse tree - */ - void exitColPosition(MixedFormatSqlExtendParser.ColPositionContext ctx); - /** - * Enter a parse tree produced by the {@code complexDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - */ - void enterComplexDataType(MixedFormatSqlExtendParser.ComplexDataTypeContext ctx); - /** - * Exit a parse tree produced by the {@code complexDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - */ - void exitComplexDataType(MixedFormatSqlExtendParser.ComplexDataTypeContext ctx); - /** - * Enter a parse tree produced by the {@code yearMonthIntervalDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - */ - void enterYearMonthIntervalDataType(MixedFormatSqlExtendParser.YearMonthIntervalDataTypeContext ctx); - /** - * Exit a parse tree produced by the {@code yearMonthIntervalDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - */ - void exitYearMonthIntervalDataType(MixedFormatSqlExtendParser.YearMonthIntervalDataTypeContext ctx); - /** - * Enter a parse tree produced by the {@code dayTimeIntervalDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - */ - void enterDayTimeIntervalDataType(MixedFormatSqlExtendParser.DayTimeIntervalDataTypeContext ctx); - /** - * Exit a parse tree produced by the {@code dayTimeIntervalDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - */ - void exitDayTimeIntervalDataType(MixedFormatSqlExtendParser.DayTimeIntervalDataTypeContext ctx); - /** - * Enter a parse tree produced by the {@code primitiveDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - */ - void enterPrimitiveDataType(MixedFormatSqlExtendParser.PrimitiveDataTypeContext ctx); - /** - * Exit a parse tree produced by the {@code primitiveDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - */ - void exitPrimitiveDataType(MixedFormatSqlExtendParser.PrimitiveDataTypeContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#colTypeList}. - * @param ctx the parse tree - */ - void enterColTypeList(MixedFormatSqlExtendParser.ColTypeListContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#colTypeList}. - * @param ctx the parse tree - */ - void exitColTypeList(MixedFormatSqlExtendParser.ColTypeListContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#colType}. - * @param ctx the parse tree - */ - void enterColType(MixedFormatSqlExtendParser.ColTypeContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#colType}. - * @param ctx the parse tree - */ - void exitColType(MixedFormatSqlExtendParser.ColTypeContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#complexColTypeList}. - * @param ctx the parse tree - */ - void enterComplexColTypeList(MixedFormatSqlExtendParser.ComplexColTypeListContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#complexColTypeList}. - * @param ctx the parse tree - */ - void exitComplexColTypeList(MixedFormatSqlExtendParser.ComplexColTypeListContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#complexColType}. - * @param ctx the parse tree - */ - void enterComplexColType(MixedFormatSqlExtendParser.ComplexColTypeContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#complexColType}. - * @param ctx the parse tree - */ - void exitComplexColType(MixedFormatSqlExtendParser.ComplexColTypeContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#whenClause}. - * @param ctx the parse tree - */ - void enterWhenClause(MixedFormatSqlExtendParser.WhenClauseContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#whenClause}. - * @param ctx the parse tree - */ - void exitWhenClause(MixedFormatSqlExtendParser.WhenClauseContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#windowClause}. - * @param ctx the parse tree - */ - void enterWindowClause(MixedFormatSqlExtendParser.WindowClauseContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#windowClause}. - * @param ctx the parse tree - */ - void exitWindowClause(MixedFormatSqlExtendParser.WindowClauseContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#namedWindow}. - * @param ctx the parse tree - */ - void enterNamedWindow(MixedFormatSqlExtendParser.NamedWindowContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#namedWindow}. - * @param ctx the parse tree - */ - void exitNamedWindow(MixedFormatSqlExtendParser.NamedWindowContext ctx); - /** - * Enter a parse tree produced by the {@code windowRef} - * labeled alternative in {@link MixedFormatSqlExtendParser#windowSpec}. - * @param ctx the parse tree - */ - void enterWindowRef(MixedFormatSqlExtendParser.WindowRefContext ctx); - /** - * Exit a parse tree produced by the {@code windowRef} - * labeled alternative in {@link MixedFormatSqlExtendParser#windowSpec}. - * @param ctx the parse tree - */ - void exitWindowRef(MixedFormatSqlExtendParser.WindowRefContext ctx); - /** - * Enter a parse tree produced by the {@code windowDef} - * labeled alternative in {@link MixedFormatSqlExtendParser#windowSpec}. - * @param ctx the parse tree - */ - void enterWindowDef(MixedFormatSqlExtendParser.WindowDefContext ctx); - /** - * Exit a parse tree produced by the {@code windowDef} - * labeled alternative in {@link MixedFormatSqlExtendParser#windowSpec}. - * @param ctx the parse tree - */ - void exitWindowDef(MixedFormatSqlExtendParser.WindowDefContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#windowFrame}. - * @param ctx the parse tree - */ - void enterWindowFrame(MixedFormatSqlExtendParser.WindowFrameContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#windowFrame}. - * @param ctx the parse tree - */ - void exitWindowFrame(MixedFormatSqlExtendParser.WindowFrameContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#frameBound}. - * @param ctx the parse tree - */ - void enterFrameBound(MixedFormatSqlExtendParser.FrameBoundContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#frameBound}. - * @param ctx the parse tree - */ - void exitFrameBound(MixedFormatSqlExtendParser.FrameBoundContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#functionName}. - * @param ctx the parse tree - */ - void enterFunctionName(MixedFormatSqlExtendParser.FunctionNameContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#functionName}. - * @param ctx the parse tree - */ - void exitFunctionName(MixedFormatSqlExtendParser.FunctionNameContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#qualifiedName}. - * @param ctx the parse tree - */ - void enterQualifiedName(MixedFormatSqlExtendParser.QualifiedNameContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#qualifiedName}. - * @param ctx the parse tree - */ - void exitQualifiedName(MixedFormatSqlExtendParser.QualifiedNameContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#errorCapturingIdentifier}. - * @param ctx the parse tree - */ - void enterErrorCapturingIdentifier(MixedFormatSqlExtendParser.ErrorCapturingIdentifierContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#errorCapturingIdentifier}. - * @param ctx the parse tree - */ - void exitErrorCapturingIdentifier(MixedFormatSqlExtendParser.ErrorCapturingIdentifierContext ctx); - /** - * Enter a parse tree produced by the {@code errorIdent} - * labeled alternative in {@link MixedFormatSqlExtendParser#errorCapturingIdentifierExtra}. - * @param ctx the parse tree - */ - void enterErrorIdent(MixedFormatSqlExtendParser.ErrorIdentContext ctx); - /** - * Exit a parse tree produced by the {@code errorIdent} - * labeled alternative in {@link MixedFormatSqlExtendParser#errorCapturingIdentifierExtra}. - * @param ctx the parse tree - */ - void exitErrorIdent(MixedFormatSqlExtendParser.ErrorIdentContext ctx); - /** - * Enter a parse tree produced by the {@code realIdent} - * labeled alternative in {@link MixedFormatSqlExtendParser#errorCapturingIdentifierExtra}. - * @param ctx the parse tree - */ - void enterRealIdent(MixedFormatSqlExtendParser.RealIdentContext ctx); - /** - * Exit a parse tree produced by the {@code realIdent} - * labeled alternative in {@link MixedFormatSqlExtendParser#errorCapturingIdentifierExtra}. - * @param ctx the parse tree - */ - void exitRealIdent(MixedFormatSqlExtendParser.RealIdentContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#identifier}. - * @param ctx the parse tree - */ - void enterIdentifier(MixedFormatSqlExtendParser.IdentifierContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#identifier}. - * @param ctx the parse tree - */ - void exitIdentifier(MixedFormatSqlExtendParser.IdentifierContext ctx); - /** - * Enter a parse tree produced by the {@code unquotedIdentifier} - * labeled alternative in {@link MixedFormatSqlExtendParser#strictIdentifier}. - * @param ctx the parse tree - */ - void enterUnquotedIdentifier(MixedFormatSqlExtendParser.UnquotedIdentifierContext ctx); - /** - * Exit a parse tree produced by the {@code unquotedIdentifier} - * labeled alternative in {@link MixedFormatSqlExtendParser#strictIdentifier}. - * @param ctx the parse tree - */ - void exitUnquotedIdentifier(MixedFormatSqlExtendParser.UnquotedIdentifierContext ctx); - /** - * Enter a parse tree produced by the {@code quotedIdentifierAlternative} - * labeled alternative in {@link MixedFormatSqlExtendParser#strictIdentifier}. - * @param ctx the parse tree - */ - void enterQuotedIdentifierAlternative(MixedFormatSqlExtendParser.QuotedIdentifierAlternativeContext ctx); - /** - * Exit a parse tree produced by the {@code quotedIdentifierAlternative} - * labeled alternative in {@link MixedFormatSqlExtendParser#strictIdentifier}. - * @param ctx the parse tree - */ - void exitQuotedIdentifierAlternative(MixedFormatSqlExtendParser.QuotedIdentifierAlternativeContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#quotedIdentifier}. - * @param ctx the parse tree - */ - void enterQuotedIdentifier(MixedFormatSqlExtendParser.QuotedIdentifierContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#quotedIdentifier}. - * @param ctx the parse tree - */ - void exitQuotedIdentifier(MixedFormatSqlExtendParser.QuotedIdentifierContext ctx); - /** - * Enter a parse tree produced by the {@code exponentLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void enterExponentLiteral(MixedFormatSqlExtendParser.ExponentLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code exponentLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void exitExponentLiteral(MixedFormatSqlExtendParser.ExponentLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code decimalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void enterDecimalLiteral(MixedFormatSqlExtendParser.DecimalLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code decimalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void exitDecimalLiteral(MixedFormatSqlExtendParser.DecimalLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code legacyDecimalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void enterLegacyDecimalLiteral(MixedFormatSqlExtendParser.LegacyDecimalLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code legacyDecimalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void exitLegacyDecimalLiteral(MixedFormatSqlExtendParser.LegacyDecimalLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code integerLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void enterIntegerLiteral(MixedFormatSqlExtendParser.IntegerLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code integerLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void exitIntegerLiteral(MixedFormatSqlExtendParser.IntegerLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code bigIntLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void enterBigIntLiteral(MixedFormatSqlExtendParser.BigIntLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code bigIntLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void exitBigIntLiteral(MixedFormatSqlExtendParser.BigIntLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code smallIntLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void enterSmallIntLiteral(MixedFormatSqlExtendParser.SmallIntLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code smallIntLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void exitSmallIntLiteral(MixedFormatSqlExtendParser.SmallIntLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code tinyIntLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void enterTinyIntLiteral(MixedFormatSqlExtendParser.TinyIntLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code tinyIntLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void exitTinyIntLiteral(MixedFormatSqlExtendParser.TinyIntLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code doubleLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void enterDoubleLiteral(MixedFormatSqlExtendParser.DoubleLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code doubleLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void exitDoubleLiteral(MixedFormatSqlExtendParser.DoubleLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code floatLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void enterFloatLiteral(MixedFormatSqlExtendParser.FloatLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code floatLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void exitFloatLiteral(MixedFormatSqlExtendParser.FloatLiteralContext ctx); - /** - * Enter a parse tree produced by the {@code bigDecimalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void enterBigDecimalLiteral(MixedFormatSqlExtendParser.BigDecimalLiteralContext ctx); - /** - * Exit a parse tree produced by the {@code bigDecimalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - */ - void exitBigDecimalLiteral(MixedFormatSqlExtendParser.BigDecimalLiteralContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#ansiNonReserved}. - * @param ctx the parse tree - */ - void enterAnsiNonReserved(MixedFormatSqlExtendParser.AnsiNonReservedContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#ansiNonReserved}. - * @param ctx the parse tree - */ - void exitAnsiNonReserved(MixedFormatSqlExtendParser.AnsiNonReservedContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#strictNonReserved}. - * @param ctx the parse tree - */ - void enterStrictNonReserved(MixedFormatSqlExtendParser.StrictNonReservedContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#strictNonReserved}. - * @param ctx the parse tree - */ - void exitStrictNonReserved(MixedFormatSqlExtendParser.StrictNonReservedContext ctx); - /** - * Enter a parse tree produced by {@link MixedFormatSqlExtendParser#nonReserved}. - * @param ctx the parse tree - */ - void enterNonReserved(MixedFormatSqlExtendParser.NonReservedContext ctx); - /** - * Exit a parse tree produced by {@link MixedFormatSqlExtendParser#nonReserved}. - * @param ctx the parse tree - */ - void exitNonReserved(MixedFormatSqlExtendParser.NonReservedContext ctx); -} \ No newline at end of file diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendParser.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendParser.java deleted file mode 100644 index 6ac3f87c2e..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendParser.java +++ /dev/null @@ -1,13947 +0,0 @@ -// Generated from org/apache/amoro/spark/sql/parser/MixedFormatSqlExtend.g4 by ANTLR 4.8 -package org.apache.amoro.spark.sql.parser; -import org.antlr.v4.runtime.atn.*; -import org.antlr.v4.runtime.dfa.DFA; -import org.antlr.v4.runtime.*; -import org.antlr.v4.runtime.misc.*; -import org.antlr.v4.runtime.tree.*; -import java.util.List; -import java.util.Iterator; -import java.util.ArrayList; - -@SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"}) -public class MixedFormatSqlExtendParser extends Parser { - static { RuntimeMetaData.checkVersion("4.8", RuntimeMetaData.VERSION); } - - protected static final DFA[] _decisionToDFA; - protected static final PredictionContextCache _sharedContextCache = - new PredictionContextCache(); - public static final int - T__0=1, T__1=2, T__2=3, T__3=4, T__4=5, T__5=6, T__6=7, T__7=8, T__8=9, - T__9=10, T__10=11, ADD=12, AFTER=13, ALL=14, ALTER=15, ANALYZE=16, AND=17, - ANTI=18, ANY=19, ARCHIVE=20, ARRAY=21, AS=22, ASC=23, AT=24, AUTHORIZATION=25, - BETWEEN=26, BOTH=27, BUCKET=28, BUCKETS=29, BY=30, CACHE=31, CASCADE=32, - CASE=33, CAST=34, CHANGE=35, CHECK=36, CLEAR=37, CLUSTER=38, CLUSTERED=39, - CODEGEN=40, COLLATE=41, COLLECTION=42, COLUMN=43, COLUMNS=44, COMMENT=45, - COMMIT=46, COMPACT=47, COMPACTIONS=48, COMPUTE=49, CONCATENATE=50, CONSTRAINT=51, - COST=52, CREATE=53, CROSS=54, CUBE=55, CURRENT=56, CURRENT_DATE=57, CURRENT_TIME=58, - CURRENT_TIMESTAMP=59, CURRENT_USER=60, DAY=61, DATA=62, DATABASE=63, DATABASES=64, - DBPROPERTIES=65, DEFINED=66, DELETE=67, DELIMITED=68, DESC=69, DESCRIBE=70, - DFS=71, DIRECTORIES=72, DIRECTORY=73, DISTINCT=74, DISTRIBUTE=75, DIV=76, - DROP=77, ELSE=78, END=79, ESCAPE=80, ESCAPED=81, EXCEPT=82, EXCHANGE=83, - EXISTS=84, EXPLAIN=85, EXPORT=86, EXTENDED=87, EXTERNAL=88, EXTRACT=89, - FALSE=90, FETCH=91, FIELDS=92, FILTER=93, FILEFORMAT=94, FIRST=95, FOLLOWING=96, - FOR=97, FOREIGN=98, FORMAT=99, FORMATTED=100, FROM=101, FULL=102, FUNCTION=103, - FUNCTIONS=104, GLOBAL=105, GRANT=106, GROUP=107, GROUPING=108, HAVING=109, - HOUR=110, IF=111, IGNORE=112, IMPORT=113, IN=114, INDEX=115, INDEXES=116, - INNER=117, INPATH=118, INPUTFORMAT=119, INSERT=120, INTERSECT=121, INTERVAL=122, - INTO=123, IS=124, ITEMS=125, JOIN=126, KEYS=127, LAST=128, LATERAL=129, - LAZY=130, LEADING=131, LEFT=132, LIKE=133, LIMIT=134, LINES=135, LIST=136, - LOAD=137, LOCAL=138, LOCATION=139, LOCK=140, LOCKS=141, LOGICAL=142, MACRO=143, - MAP=144, MATCHED=145, MERGE=146, MINUTE=147, MONTH=148, MSCK=149, NAMESPACE=150, - NAMESPACES=151, NATURAL=152, NO=153, NOT=154, NULL=155, NULLS=156, OF=157, - ON=158, ONLY=159, OPTION=160, OPTIONS=161, OR=162, ORDER=163, OUT=164, - OUTER=165, OUTPUTFORMAT=166, OVER=167, OVERLAPS=168, OVERLAY=169, OVERWRITE=170, - PARTITION=171, PARTITIONED=172, PARTITIONS=173, PERCENTLIT=174, PIVOT=175, - PLACING=176, POSITION=177, PRECEDING=178, PRIMARY=179, PRINCIPALS=180, - PROPERTIES=181, PURGE=182, QUERY=183, RANGE=184, RECORDREADER=185, RECORDWRITER=186, - RECOVER=187, REDUCE=188, REFERENCES=189, REFRESH=190, RENAME=191, REPAIR=192, - REPLACE=193, RESET=194, RESPECT=195, RESTRICT=196, REVOKE=197, RIGHT=198, - RLIKE=199, ROLE=200, ROLES=201, ROLLBACK=202, ROLLUP=203, ROW=204, ROWS=205, - SECOND=206, SCHEMA=207, SELECT=208, SEMI=209, SEPARATED=210, SERDE=211, - SERDEPROPERTIES=212, SESSION_USER=213, SET=214, SETMINUS=215, SETS=216, - SHOW=217, SKEWED=218, SOME=219, SORT=220, SORTED=221, START=222, STATISTICS=223, - STORED=224, STRATIFY=225, STRUCT=226, SUBSTR=227, SUBSTRING=228, SYNC=229, - TABLE=230, TABLES=231, TABLESAMPLE=232, TBLPROPERTIES=233, TEMPORARY=234, - TERMINATED=235, THEN=236, TIME=237, TO=238, TOUCH=239, TRAILING=240, TRANSACTION=241, - TRANSACTIONS=242, TRANSFORM=243, TRIM=244, TRUE=245, TRUNCATE=246, TRY_CAST=247, - TYPE=248, UNARCHIVE=249, UNBOUNDED=250, UNCACHE=251, UNION=252, UNIQUE=253, - UNKNOWN=254, UNLOCK=255, UNSET=256, UPDATE=257, USE=258, USER=259, USING=260, - VALUES=261, VIEW=262, VIEWS=263, WHEN=264, WHERE=265, WINDOW=266, WITH=267, - YEAR=268, ZONE=269, KEY=270, EQ=271, NSEQ=272, NEQ=273, NEQJ=274, LT=275, - LTE=276, GT=277, GTE=278, PLUS=279, MINUS=280, ASTERISK=281, SLASH=282, - PERCENT=283, TILDE=284, AMPERSAND=285, PIPE=286, CONCAT_PIPE=287, HAT=288, - STRING=289, BIGINT_LITERAL=290, SMALLINT_LITERAL=291, TINYINT_LITERAL=292, - INTEGER_VALUE=293, EXPONENT_VALUE=294, DECIMAL_VALUE=295, FLOAT_LITERAL=296, - DOUBLE_LITERAL=297, BIGDECIMAL_LITERAL=298, IDENTIFIER=299, BACKQUOTED_IDENTIFIER=300, - SIMPLE_COMMENT=301, BRACKETED_COMMENT=302, WS=303, UNRECOGNIZED=304; - public static final int - RULE_extendStatement = 0, RULE_statement = 1, RULE_createTableHeader = 2, - RULE_colListAndPk = 3, RULE_primarySpec = 4, RULE_bucketSpec = 5, RULE_skewSpec = 6, - RULE_locationSpec = 7, RULE_commentSpec = 8, RULE_query = 9, RULE_ctes = 10, - RULE_namedQuery = 11, RULE_tableProvider = 12, RULE_createTableClauses = 13, - RULE_tablePropertyList = 14, RULE_tableProperty = 15, RULE_tablePropertyKey = 16, - RULE_tablePropertyValue = 17, RULE_constantList = 18, RULE_nestedConstantList = 19, - RULE_createFileFormat = 20, RULE_fileFormat = 21, RULE_storageHandler = 22, - RULE_queryOrganization = 23, RULE_queryTerm = 24, RULE_queryPrimary = 25, - RULE_sortItem = 26, RULE_fromStatement = 27, RULE_fromStatementBody = 28, - RULE_querySpecification = 29, RULE_transformClause = 30, RULE_selectClause = 31, - RULE_whereClause = 32, RULE_havingClause = 33, RULE_hint = 34, RULE_hintStatement = 35, - RULE_fromClause = 36, RULE_aggregationClause = 37, RULE_groupByClause = 38, - RULE_groupingAnalytics = 39, RULE_groupingElement = 40, RULE_groupingSet = 41, - RULE_pivotClause = 42, RULE_pivotColumn = 43, RULE_pivotValue = 44, RULE_lateralView = 45, - RULE_setQuantifier = 46, RULE_relation = 47, RULE_joinRelation = 48, RULE_joinType = 49, - RULE_joinCriteria = 50, RULE_sample = 51, RULE_sampleMethod = 52, RULE_identifierList = 53, - RULE_identifierSeq = 54, RULE_orderedIdentifierList = 55, RULE_orderedIdentifier = 56, - RULE_relationPrimary = 57, RULE_inlineTable = 58, RULE_functionTable = 59, - RULE_tableAlias = 60, RULE_rowFormat = 61, RULE_multipartIdentifier = 62, - RULE_namedExpression = 63, RULE_namedExpressionSeq = 64, RULE_partitionFieldList = 65, - RULE_partitionField = 66, RULE_transform = 67, RULE_transformArgument = 68, - RULE_expression = 69, RULE_expressionSeq = 70, RULE_booleanExpression = 71, - RULE_predicate = 72, RULE_valueExpression = 73, RULE_primaryExpression = 74, - RULE_constant = 75, RULE_comparisonOperator = 76, RULE_booleanValue = 77, - RULE_interval = 78, RULE_errorCapturingMultiUnitsInterval = 79, RULE_multiUnitsInterval = 80, - RULE_errorCapturingUnitToUnitInterval = 81, RULE_unitToUnitInterval = 82, - RULE_intervalValue = 83, RULE_colPosition = 84, RULE_dataType = 85, RULE_colTypeList = 86, - RULE_colType = 87, RULE_complexColTypeList = 88, RULE_complexColType = 89, - RULE_whenClause = 90, RULE_windowClause = 91, RULE_namedWindow = 92, RULE_windowSpec = 93, - RULE_windowFrame = 94, RULE_frameBound = 95, RULE_functionName = 96, RULE_qualifiedName = 97, - RULE_errorCapturingIdentifier = 98, RULE_errorCapturingIdentifierExtra = 99, - RULE_identifier = 100, RULE_strictIdentifier = 101, RULE_quotedIdentifier = 102, - RULE_number = 103, RULE_ansiNonReserved = 104, RULE_strictNonReserved = 105, - RULE_nonReserved = 106; - private static String[] makeRuleNames() { - return new String[] { - "extendStatement", "statement", "createTableHeader", "colListAndPk", - "primarySpec", "bucketSpec", "skewSpec", "locationSpec", "commentSpec", - "query", "ctes", "namedQuery", "tableProvider", "createTableClauses", - "tablePropertyList", "tableProperty", "tablePropertyKey", "tablePropertyValue", - "constantList", "nestedConstantList", "createFileFormat", "fileFormat", - "storageHandler", "queryOrganization", "queryTerm", "queryPrimary", "sortItem", - "fromStatement", "fromStatementBody", "querySpecification", "transformClause", - "selectClause", "whereClause", "havingClause", "hint", "hintStatement", - "fromClause", "aggregationClause", "groupByClause", "groupingAnalytics", - "groupingElement", "groupingSet", "pivotClause", "pivotColumn", "pivotValue", - "lateralView", "setQuantifier", "relation", "joinRelation", "joinType", - "joinCriteria", "sample", "sampleMethod", "identifierList", "identifierSeq", - "orderedIdentifierList", "orderedIdentifier", "relationPrimary", "inlineTable", - "functionTable", "tableAlias", "rowFormat", "multipartIdentifier", "namedExpression", - "namedExpressionSeq", "partitionFieldList", "partitionField", "transform", - "transformArgument", "expression", "expressionSeq", "booleanExpression", - "predicate", "valueExpression", "primaryExpression", "constant", "comparisonOperator", - "booleanValue", "interval", "errorCapturingMultiUnitsInterval", "multiUnitsInterval", - "errorCapturingUnitToUnitInterval", "unitToUnitInterval", "intervalValue", - "colPosition", "dataType", "colTypeList", "colType", "complexColTypeList", - "complexColType", "whenClause", "windowClause", "namedWindow", "windowSpec", - "windowFrame", "frameBound", "functionName", "qualifiedName", "errorCapturingIdentifier", - "errorCapturingIdentifierExtra", "identifier", "strictIdentifier", "quotedIdentifier", - "number", "ansiNonReserved", "strictNonReserved", "nonReserved" - }; - } - public static final String[] ruleNames = makeRuleNames(); - - private static String[] makeLiteralNames() { - return new String[] { - null, "';'", "'('", "','", "')'", "'.'", "'/*+'", "'*/'", "'->'", "'['", - "']'", "':'", "'ADD'", "'AFTER'", "'ALL'", "'ALTER'", "'ANALYZE'", "'AND'", - "'ANTI'", "'ANY'", "'ARCHIVE'", "'ARRAY'", "'AS'", "'ASC'", "'AT'", "'AUTHORIZATION'", - "'BETWEEN'", "'BOTH'", "'BUCKET'", "'BUCKETS'", "'BY'", "'CACHE'", "'CASCADE'", - "'CASE'", "'CAST'", "'CHANGE'", "'CHECK'", "'CLEAR'", "'CLUSTER'", "'CLUSTERED'", - "'CODEGEN'", "'COLLATE'", "'COLLECTION'", "'COLUMN'", "'COLUMNS'", "'COMMENT'", - "'COMMIT'", "'COMPACT'", "'COMPACTIONS'", "'COMPUTE'", "'CONCATENATE'", - "'CONSTRAINT'", "'COST'", "'CREATE'", "'CROSS'", "'CUBE'", "'CURRENT'", - "'CURRENT_DATE'", "'CURRENT_TIME'", "'CURRENT_TIMESTAMP'", "'CURRENT_USER'", - "'DAY'", "'DATA'", "'DATABASE'", null, "'DBPROPERTIES'", "'DEFINED'", - "'DELETE'", "'DELIMITED'", "'DESC'", "'DESCRIBE'", "'DFS'", "'DIRECTORIES'", - "'DIRECTORY'", "'DISTINCT'", "'DISTRIBUTE'", "'DIV'", "'DROP'", "'ELSE'", - "'END'", "'ESCAPE'", "'ESCAPED'", "'EXCEPT'", "'EXCHANGE'", "'EXISTS'", - "'EXPLAIN'", "'EXPORT'", "'EXTENDED'", "'EXTERNAL'", "'EXTRACT'", "'FALSE'", - "'FETCH'", "'FIELDS'", "'FILTER'", "'FILEFORMAT'", "'FIRST'", "'FOLLOWING'", - "'FOR'", "'FOREIGN'", "'FORMAT'", "'FORMATTED'", "'FROM'", "'FULL'", - "'FUNCTION'", "'FUNCTIONS'", "'GLOBAL'", "'GRANT'", "'GROUP'", "'GROUPING'", - "'HAVING'", "'HOUR'", "'IF'", "'IGNORE'", "'IMPORT'", "'IN'", "'INDEX'", - "'INDEXES'", "'INNER'", "'INPATH'", "'INPUTFORMAT'", "'INSERT'", "'INTERSECT'", - "'INTERVAL'", "'INTO'", "'IS'", "'ITEMS'", "'JOIN'", "'KEYS'", "'LAST'", - "'LATERAL'", "'LAZY'", "'LEADING'", "'LEFT'", "'LIKE'", "'LIMIT'", "'LINES'", - "'LIST'", "'LOAD'", "'LOCAL'", "'LOCATION'", "'LOCK'", "'LOCKS'", "'LOGICAL'", - "'MACRO'", "'MAP'", "'MATCHED'", "'MERGE'", "'MINUTE'", "'MONTH'", "'MSCK'", - "'NAMESPACE'", "'NAMESPACES'", "'NATURAL'", "'NO'", null, "'NULL'", "'NULLS'", - "'OF'", "'ON'", "'ONLY'", "'OPTION'", "'OPTIONS'", "'OR'", "'ORDER'", - "'OUT'", "'OUTER'", "'OUTPUTFORMAT'", "'OVER'", "'OVERLAPS'", "'OVERLAY'", - "'OVERWRITE'", "'PARTITION'", "'PARTITIONED'", "'PARTITIONS'", "'PERCENT'", - "'PIVOT'", "'PLACING'", "'POSITION'", "'PRECEDING'", "'PRIMARY'", "'PRINCIPALS'", - "'PROPERTIES'", "'PURGE'", "'QUERY'", "'RANGE'", "'RECORDREADER'", "'RECORDWRITER'", - "'RECOVER'", "'REDUCE'", "'REFERENCES'", "'REFRESH'", "'RENAME'", "'REPAIR'", - "'REPLACE'", "'RESET'", "'RESPECT'", "'RESTRICT'", "'REVOKE'", "'RIGHT'", - null, "'ROLE'", "'ROLES'", "'ROLLBACK'", "'ROLLUP'", "'ROW'", "'ROWS'", - "'SECOND'", "'SCHEMA'", "'SELECT'", "'SEMI'", "'SEPARATED'", "'SERDE'", - "'SERDEPROPERTIES'", "'SESSION_USER'", "'SET'", "'MINUS'", "'SETS'", - "'SHOW'", "'SKEWED'", "'SOME'", "'SORT'", "'SORTED'", "'START'", "'STATISTICS'", - "'STORED'", "'STRATIFY'", "'STRUCT'", "'SUBSTR'", "'SUBSTRING'", "'SYNC'", - "'TABLE'", "'TABLES'", "'TABLESAMPLE'", "'TBLPROPERTIES'", null, "'TERMINATED'", - "'THEN'", "'TIME'", "'TO'", "'TOUCH'", "'TRAILING'", "'TRANSACTION'", - "'TRANSACTIONS'", "'TRANSFORM'", "'TRIM'", "'TRUE'", "'TRUNCATE'", "'TRY_CAST'", - "'TYPE'", "'UNARCHIVE'", "'UNBOUNDED'", "'UNCACHE'", "'UNION'", "'UNIQUE'", - "'UNKNOWN'", "'UNLOCK'", "'UNSET'", "'UPDATE'", "'USE'", "'USER'", "'USING'", - "'VALUES'", "'VIEW'", "'VIEWS'", "'WHEN'", "'WHERE'", "'WINDOW'", "'WITH'", - "'YEAR'", "'ZONE'", "'KEY'", null, "'<=>'", "'<>'", "'!='", "'<'", null, - "'>'", null, "'+'", "'-'", "'*'", "'/'", "'%'", "'~'", "'&'", "'|'", - "'||'", "'^'" - }; - } - private static final String[] _LITERAL_NAMES = makeLiteralNames(); - private static String[] makeSymbolicNames() { - return new String[] { - null, null, null, null, null, null, null, null, null, null, null, null, - "ADD", "AFTER", "ALL", "ALTER", "ANALYZE", "AND", "ANTI", "ANY", "ARCHIVE", - "ARRAY", "AS", "ASC", "AT", "AUTHORIZATION", "BETWEEN", "BOTH", "BUCKET", - "BUCKETS", "BY", "CACHE", "CASCADE", "CASE", "CAST", "CHANGE", "CHECK", - "CLEAR", "CLUSTER", "CLUSTERED", "CODEGEN", "COLLATE", "COLLECTION", - "COLUMN", "COLUMNS", "COMMENT", "COMMIT", "COMPACT", "COMPACTIONS", "COMPUTE", - "CONCATENATE", "CONSTRAINT", "COST", "CREATE", "CROSS", "CUBE", "CURRENT", - "CURRENT_DATE", "CURRENT_TIME", "CURRENT_TIMESTAMP", "CURRENT_USER", - "DAY", "DATA", "DATABASE", "DATABASES", "DBPROPERTIES", "DEFINED", "DELETE", - "DELIMITED", "DESC", "DESCRIBE", "DFS", "DIRECTORIES", "DIRECTORY", "DISTINCT", - "DISTRIBUTE", "DIV", "DROP", "ELSE", "END", "ESCAPE", "ESCAPED", "EXCEPT", - "EXCHANGE", "EXISTS", "EXPLAIN", "EXPORT", "EXTENDED", "EXTERNAL", "EXTRACT", - "FALSE", "FETCH", "FIELDS", "FILTER", "FILEFORMAT", "FIRST", "FOLLOWING", - "FOR", "FOREIGN", "FORMAT", "FORMATTED", "FROM", "FULL", "FUNCTION", - "FUNCTIONS", "GLOBAL", "GRANT", "GROUP", "GROUPING", "HAVING", "HOUR", - "IF", "IGNORE", "IMPORT", "IN", "INDEX", "INDEXES", "INNER", "INPATH", - "INPUTFORMAT", "INSERT", "INTERSECT", "INTERVAL", "INTO", "IS", "ITEMS", - "JOIN", "KEYS", "LAST", "LATERAL", "LAZY", "LEADING", "LEFT", "LIKE", - "LIMIT", "LINES", "LIST", "LOAD", "LOCAL", "LOCATION", "LOCK", "LOCKS", - "LOGICAL", "MACRO", "MAP", "MATCHED", "MERGE", "MINUTE", "MONTH", "MSCK", - "NAMESPACE", "NAMESPACES", "NATURAL", "NO", "NOT", "NULL", "NULLS", "OF", - "ON", "ONLY", "OPTION", "OPTIONS", "OR", "ORDER", "OUT", "OUTER", "OUTPUTFORMAT", - "OVER", "OVERLAPS", "OVERLAY", "OVERWRITE", "PARTITION", "PARTITIONED", - "PARTITIONS", "PERCENTLIT", "PIVOT", "PLACING", "POSITION", "PRECEDING", - "PRIMARY", "PRINCIPALS", "PROPERTIES", "PURGE", "QUERY", "RANGE", "RECORDREADER", - "RECORDWRITER", "RECOVER", "REDUCE", "REFERENCES", "REFRESH", "RENAME", - "REPAIR", "REPLACE", "RESET", "RESPECT", "RESTRICT", "REVOKE", "RIGHT", - "RLIKE", "ROLE", "ROLES", "ROLLBACK", "ROLLUP", "ROW", "ROWS", "SECOND", - "SCHEMA", "SELECT", "SEMI", "SEPARATED", "SERDE", "SERDEPROPERTIES", - "SESSION_USER", "SET", "SETMINUS", "SETS", "SHOW", "SKEWED", "SOME", - "SORT", "SORTED", "START", "STATISTICS", "STORED", "STRATIFY", "STRUCT", - "SUBSTR", "SUBSTRING", "SYNC", "TABLE", "TABLES", "TABLESAMPLE", "TBLPROPERTIES", - "TEMPORARY", "TERMINATED", "THEN", "TIME", "TO", "TOUCH", "TRAILING", - "TRANSACTION", "TRANSACTIONS", "TRANSFORM", "TRIM", "TRUE", "TRUNCATE", - "TRY_CAST", "TYPE", "UNARCHIVE", "UNBOUNDED", "UNCACHE", "UNION", "UNIQUE", - "UNKNOWN", "UNLOCK", "UNSET", "UPDATE", "USE", "USER", "USING", "VALUES", - "VIEW", "VIEWS", "WHEN", "WHERE", "WINDOW", "WITH", "YEAR", "ZONE", "KEY", - "EQ", "NSEQ", "NEQ", "NEQJ", "LT", "LTE", "GT", "GTE", "PLUS", "MINUS", - "ASTERISK", "SLASH", "PERCENT", "TILDE", "AMPERSAND", "PIPE", "CONCAT_PIPE", - "HAT", "STRING", "BIGINT_LITERAL", "SMALLINT_LITERAL", "TINYINT_LITERAL", - "INTEGER_VALUE", "EXPONENT_VALUE", "DECIMAL_VALUE", "FLOAT_LITERAL", - "DOUBLE_LITERAL", "BIGDECIMAL_LITERAL", "IDENTIFIER", "BACKQUOTED_IDENTIFIER", - "SIMPLE_COMMENT", "BRACKETED_COMMENT", "WS", "UNRECOGNIZED" - }; - } - private static final String[] _SYMBOLIC_NAMES = makeSymbolicNames(); - public static final Vocabulary VOCABULARY = new VocabularyImpl(_LITERAL_NAMES, _SYMBOLIC_NAMES); - - /** - * @deprecated Use {@link #VOCABULARY} instead. - */ - @Deprecated - public static final String[] tokenNames; - static { - tokenNames = new String[_SYMBOLIC_NAMES.length]; - for (int i = 0; i < tokenNames.length; i++) { - tokenNames[i] = VOCABULARY.getLiteralName(i); - if (tokenNames[i] == null) { - tokenNames[i] = VOCABULARY.getSymbolicName(i); - } - - if (tokenNames[i] == null) { - tokenNames[i] = ""; - } - } - } - - @Override - @Deprecated - public String[] getTokenNames() { - return tokenNames; - } - - @Override - - public Vocabulary getVocabulary() { - return VOCABULARY; - } - - @Override - public String getGrammarFileName() { return "MixedFormatSqlExtend.g4"; } - - @Override - public String[] getRuleNames() { return ruleNames; } - - @Override - public String getSerializedATN() { return _serializedATN; } - - @Override - public ATN getATN() { return _ATN; } - - - /** - * When false, INTERSECT is given the greater precedence over the other set - * operations (UNION, EXCEPT and MINUS) as per the SQL standard. - */ - public boolean legacy_setops_precedence_enabled = false; - - /** - * When false, a literal with an exponent would be converted into - * double type rather than decimal type. - */ - public boolean legacy_exponent_literal_as_decimal_enabled = false; - - /** - * When true, the behavior of keywords follows ANSI SQL standard. - */ - public boolean SQL_standard_keyword_behavior = false; - - public MixedFormatSqlExtendParser(TokenStream input) { - super(input); - _interp = new ParserATNSimulator(this,_ATN,_decisionToDFA,_sharedContextCache); - } - - public static class ExtendStatementContext extends ParserRuleContext { - public StatementContext statement() { - return getRuleContext(StatementContext.class,0); - } - public TerminalNode EOF() { return getToken(MixedFormatSqlExtendParser.EOF, 0); } - public ExtendStatementContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_extendStatement; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterExtendStatement(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitExtendStatement(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitExtendStatement(this); - else return visitor.visitChildren(this); - } - } - - public final ExtendStatementContext extendStatement() throws RecognitionException { - ExtendStatementContext _localctx = new ExtendStatementContext(_ctx, getState()); - enterRule(_localctx, 0, RULE_extendStatement); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(214); - statement(); - setState(218); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__0) { - { - { - setState(215); - match(T__0); - } - } - setState(220); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(221); - match(EOF); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class StatementContext extends ParserRuleContext { - public StatementContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_statement; } - - public StatementContext() { } - public void copyFrom(StatementContext ctx) { - super.copyFrom(ctx); - } - } - public static class ExplainContext extends StatementContext { - public TerminalNode EXPLAIN() { return getToken(MixedFormatSqlExtendParser.EXPLAIN, 0); } - public StatementContext statement() { - return getRuleContext(StatementContext.class,0); - } - public TerminalNode LOGICAL() { return getToken(MixedFormatSqlExtendParser.LOGICAL, 0); } - public TerminalNode FORMATTED() { return getToken(MixedFormatSqlExtendParser.FORMATTED, 0); } - public TerminalNode EXTENDED() { return getToken(MixedFormatSqlExtendParser.EXTENDED, 0); } - public TerminalNode CODEGEN() { return getToken(MixedFormatSqlExtendParser.CODEGEN, 0); } - public TerminalNode COST() { return getToken(MixedFormatSqlExtendParser.COST, 0); } - public ExplainContext(StatementContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterExplain(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitExplain(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitExplain(this); - else return visitor.visitChildren(this); - } - } - public static class CreateTableWithPkContext extends StatementContext { - public CreateTableHeaderContext createTableHeader() { - return getRuleContext(CreateTableHeaderContext.class,0); - } - public ColListAndPkContext colListAndPk() { - return getRuleContext(ColListAndPkContext.class,0); - } - public CreateTableClausesContext createTableClauses() { - return getRuleContext(CreateTableClausesContext.class,0); - } - public TableProviderContext tableProvider() { - return getRuleContext(TableProviderContext.class,0); - } - public QueryContext query() { - return getRuleContext(QueryContext.class,0); - } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public CreateTableWithPkContext(StatementContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterCreateTableWithPk(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitCreateTableWithPk(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitCreateTableWithPk(this); - else return visitor.visitChildren(this); - } - } - - public final StatementContext statement() throws RecognitionException { - StatementContext _localctx = new StatementContext(_ctx, getState()); - enterRule(_localctx, 2, RULE_statement); - int _la; - try { - setState(240); - _errHandler.sync(this); - switch (_input.LA(1)) { - case CREATE: - _localctx = new CreateTableWithPkContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(223); - createTableHeader(); - setState(224); - colListAndPk(); - setState(226); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==USING) { - { - setState(225); - tableProvider(); - } - } - - setState(228); - createTableClauses(); - setState(233); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==T__1 || _la==AS || _la==FROM || _la==MAP || ((((_la - 188)) & ~0x3f) == 0 && ((1L << (_la - 188)) & ((1L << (REDUCE - 188)) | (1L << (SELECT - 188)) | (1L << (TABLE - 188)))) != 0) || _la==VALUES || _la==WITH) { - { - setState(230); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==AS) { - { - setState(229); - match(AS); - } - } - - setState(232); - query(); - } - } - - } - break; - case EXPLAIN: - _localctx = new ExplainContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(235); - match(EXPLAIN); - setState(237); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==CODEGEN || _la==COST || ((((_la - 87)) & ~0x3f) == 0 && ((1L << (_la - 87)) & ((1L << (EXTENDED - 87)) | (1L << (FORMATTED - 87)) | (1L << (LOGICAL - 87)))) != 0)) { - { - setState(236); - _la = _input.LA(1); - if ( !(_la==CODEGEN || _la==COST || ((((_la - 87)) & ~0x3f) == 0 && ((1L << (_la - 87)) & ((1L << (EXTENDED - 87)) | (1L << (FORMATTED - 87)) | (1L << (LOGICAL - 87)))) != 0)) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - } - - setState(239); - statement(); - } - break; - default: - throw new NoViableAltException(this); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class CreateTableHeaderContext extends ParserRuleContext { - public TerminalNode CREATE() { return getToken(MixedFormatSqlExtendParser.CREATE, 0); } - public TerminalNode TABLE() { return getToken(MixedFormatSqlExtendParser.TABLE, 0); } - public MultipartIdentifierContext multipartIdentifier() { - return getRuleContext(MultipartIdentifierContext.class,0); - } - public TerminalNode TEMPORARY() { return getToken(MixedFormatSqlExtendParser.TEMPORARY, 0); } - public TerminalNode EXTERNAL() { return getToken(MixedFormatSqlExtendParser.EXTERNAL, 0); } - public TerminalNode IF() { return getToken(MixedFormatSqlExtendParser.IF, 0); } - public TerminalNode NOT() { return getToken(MixedFormatSqlExtendParser.NOT, 0); } - public TerminalNode EXISTS() { return getToken(MixedFormatSqlExtendParser.EXISTS, 0); } - public CreateTableHeaderContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_createTableHeader; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterCreateTableHeader(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitCreateTableHeader(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitCreateTableHeader(this); - else return visitor.visitChildren(this); - } - } - - public final CreateTableHeaderContext createTableHeader() throws RecognitionException { - CreateTableHeaderContext _localctx = new CreateTableHeaderContext(_ctx, getState()); - enterRule(_localctx, 4, RULE_createTableHeader); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(242); - match(CREATE); - setState(244); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==TEMPORARY) { - { - setState(243); - match(TEMPORARY); - } - } - - setState(247); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==EXTERNAL) { - { - setState(246); - match(EXTERNAL); - } - } - - setState(249); - match(TABLE); - setState(253); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,8,_ctx) ) { - case 1: - { - setState(250); - match(IF); - setState(251); - match(NOT); - setState(252); - match(EXISTS); - } - break; - } - setState(255); - multipartIdentifier(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ColListAndPkContext extends ParserRuleContext { - public ColListAndPkContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_colListAndPk; } - - public ColListAndPkContext() { } - public void copyFrom(ColListAndPkContext ctx) { - super.copyFrom(ctx); - } - } - public static class ColListOnlyPkContext extends ColListAndPkContext { - public PrimarySpecContext primarySpec() { - return getRuleContext(PrimarySpecContext.class,0); - } - public ColListOnlyPkContext(ColListAndPkContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterColListOnlyPk(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitColListOnlyPk(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitColListOnlyPk(this); - else return visitor.visitChildren(this); - } - } - public static class ColListWithPkContext extends ColListAndPkContext { - public ColTypeListContext colTypeList() { - return getRuleContext(ColTypeListContext.class,0); - } - public PrimarySpecContext primarySpec() { - return getRuleContext(PrimarySpecContext.class,0); - } - public ColListWithPkContext(ColListAndPkContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterColListWithPk(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitColListWithPk(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitColListWithPk(this); - else return visitor.visitChildren(this); - } - } - - public final ColListAndPkContext colListAndPk() throws RecognitionException { - ColListAndPkContext _localctx = new ColListAndPkContext(_ctx, getState()); - enterRule(_localctx, 6, RULE_colListAndPk); - int _la; - try { - setState(266); - _errHandler.sync(this); - switch (_input.LA(1)) { - case T__1: - _localctx = new ColListWithPkContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(257); - match(T__1); - setState(258); - colTypeList(); - setState(261); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==T__2) { - { - setState(259); - match(T__2); - setState(260); - primarySpec(); - } - } - - setState(263); - match(T__3); - } - break; - case PRIMARY: - _localctx = new ColListOnlyPkContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(265); - primarySpec(); - } - break; - default: - throw new NoViableAltException(this); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class PrimarySpecContext extends ParserRuleContext { - public TerminalNode PRIMARY() { return getToken(MixedFormatSqlExtendParser.PRIMARY, 0); } - public TerminalNode KEY() { return getToken(MixedFormatSqlExtendParser.KEY, 0); } - public IdentifierListContext identifierList() { - return getRuleContext(IdentifierListContext.class,0); - } - public PrimarySpecContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_primarySpec; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPrimarySpec(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPrimarySpec(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPrimarySpec(this); - else return visitor.visitChildren(this); - } - } - - public final PrimarySpecContext primarySpec() throws RecognitionException { - PrimarySpecContext _localctx = new PrimarySpecContext(_ctx, getState()); - enterRule(_localctx, 8, RULE_primarySpec); - try { - enterOuterAlt(_localctx, 1); - { - setState(268); - match(PRIMARY); - setState(269); - match(KEY); - setState(270); - identifierList(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class BucketSpecContext extends ParserRuleContext { - public TerminalNode CLUSTERED() { return getToken(MixedFormatSqlExtendParser.CLUSTERED, 0); } - public List BY() { return getTokens(MixedFormatSqlExtendParser.BY); } - public TerminalNode BY(int i) { - return getToken(MixedFormatSqlExtendParser.BY, i); - } - public IdentifierListContext identifierList() { - return getRuleContext(IdentifierListContext.class,0); - } - public TerminalNode INTO() { return getToken(MixedFormatSqlExtendParser.INTO, 0); } - public TerminalNode INTEGER_VALUE() { return getToken(MixedFormatSqlExtendParser.INTEGER_VALUE, 0); } - public TerminalNode BUCKETS() { return getToken(MixedFormatSqlExtendParser.BUCKETS, 0); } - public TerminalNode SORTED() { return getToken(MixedFormatSqlExtendParser.SORTED, 0); } - public OrderedIdentifierListContext orderedIdentifierList() { - return getRuleContext(OrderedIdentifierListContext.class,0); - } - public BucketSpecContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_bucketSpec; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterBucketSpec(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitBucketSpec(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitBucketSpec(this); - else return visitor.visitChildren(this); - } - } - - public final BucketSpecContext bucketSpec() throws RecognitionException { - BucketSpecContext _localctx = new BucketSpecContext(_ctx, getState()); - enterRule(_localctx, 10, RULE_bucketSpec); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(272); - match(CLUSTERED); - setState(273); - match(BY); - setState(274); - identifierList(); - setState(278); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==SORTED) { - { - setState(275); - match(SORTED); - setState(276); - match(BY); - setState(277); - orderedIdentifierList(); - } - } - - setState(280); - match(INTO); - setState(281); - match(INTEGER_VALUE); - setState(282); - match(BUCKETS); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class SkewSpecContext extends ParserRuleContext { - public TerminalNode SKEWED() { return getToken(MixedFormatSqlExtendParser.SKEWED, 0); } - public TerminalNode BY() { return getToken(MixedFormatSqlExtendParser.BY, 0); } - public IdentifierListContext identifierList() { - return getRuleContext(IdentifierListContext.class,0); - } - public TerminalNode ON() { return getToken(MixedFormatSqlExtendParser.ON, 0); } - public ConstantListContext constantList() { - return getRuleContext(ConstantListContext.class,0); - } - public NestedConstantListContext nestedConstantList() { - return getRuleContext(NestedConstantListContext.class,0); - } - public TerminalNode STORED() { return getToken(MixedFormatSqlExtendParser.STORED, 0); } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public TerminalNode DIRECTORIES() { return getToken(MixedFormatSqlExtendParser.DIRECTORIES, 0); } - public SkewSpecContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_skewSpec; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSkewSpec(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSkewSpec(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSkewSpec(this); - else return visitor.visitChildren(this); - } - } - - public final SkewSpecContext skewSpec() throws RecognitionException { - SkewSpecContext _localctx = new SkewSpecContext(_ctx, getState()); - enterRule(_localctx, 12, RULE_skewSpec); - try { - enterOuterAlt(_localctx, 1); - { - setState(284); - match(SKEWED); - setState(285); - match(BY); - setState(286); - identifierList(); - setState(287); - match(ON); - setState(290); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,12,_ctx) ) { - case 1: - { - setState(288); - constantList(); - } - break; - case 2: - { - setState(289); - nestedConstantList(); - } - break; - } - setState(295); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,13,_ctx) ) { - case 1: - { - setState(292); - match(STORED); - setState(293); - match(AS); - setState(294); - match(DIRECTORIES); - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class LocationSpecContext extends ParserRuleContext { - public TerminalNode LOCATION() { return getToken(MixedFormatSqlExtendParser.LOCATION, 0); } - public TerminalNode STRING() { return getToken(MixedFormatSqlExtendParser.STRING, 0); } - public LocationSpecContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_locationSpec; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterLocationSpec(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitLocationSpec(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitLocationSpec(this); - else return visitor.visitChildren(this); - } - } - - public final LocationSpecContext locationSpec() throws RecognitionException { - LocationSpecContext _localctx = new LocationSpecContext(_ctx, getState()); - enterRule(_localctx, 14, RULE_locationSpec); - try { - enterOuterAlt(_localctx, 1); - { - setState(297); - match(LOCATION); - setState(298); - match(STRING); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class CommentSpecContext extends ParserRuleContext { - public TerminalNode COMMENT() { return getToken(MixedFormatSqlExtendParser.COMMENT, 0); } - public TerminalNode STRING() { return getToken(MixedFormatSqlExtendParser.STRING, 0); } - public CommentSpecContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_commentSpec; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterCommentSpec(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitCommentSpec(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitCommentSpec(this); - else return visitor.visitChildren(this); - } - } - - public final CommentSpecContext commentSpec() throws RecognitionException { - CommentSpecContext _localctx = new CommentSpecContext(_ctx, getState()); - enterRule(_localctx, 16, RULE_commentSpec); - try { - enterOuterAlt(_localctx, 1); - { - setState(300); - match(COMMENT); - setState(301); - match(STRING); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class QueryContext extends ParserRuleContext { - public QueryTermContext queryTerm() { - return getRuleContext(QueryTermContext.class,0); - } - public QueryOrganizationContext queryOrganization() { - return getRuleContext(QueryOrganizationContext.class,0); - } - public CtesContext ctes() { - return getRuleContext(CtesContext.class,0); - } - public QueryContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_query; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterQuery(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitQuery(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitQuery(this); - else return visitor.visitChildren(this); - } - } - - public final QueryContext query() throws RecognitionException { - QueryContext _localctx = new QueryContext(_ctx, getState()); - enterRule(_localctx, 18, RULE_query); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(304); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==WITH) { - { - setState(303); - ctes(); - } - } - - setState(306); - queryTerm(0); - setState(307); - queryOrganization(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class CtesContext extends ParserRuleContext { - public TerminalNode WITH() { return getToken(MixedFormatSqlExtendParser.WITH, 0); } - public List namedQuery() { - return getRuleContexts(NamedQueryContext.class); - } - public NamedQueryContext namedQuery(int i) { - return getRuleContext(NamedQueryContext.class,i); - } - public CtesContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_ctes; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterCtes(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitCtes(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitCtes(this); - else return visitor.visitChildren(this); - } - } - - public final CtesContext ctes() throws RecognitionException { - CtesContext _localctx = new CtesContext(_ctx, getState()); - enterRule(_localctx, 20, RULE_ctes); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(309); - match(WITH); - setState(310); - namedQuery(); - setState(315); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(311); - match(T__2); - setState(312); - namedQuery(); - } - } - setState(317); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class NamedQueryContext extends ParserRuleContext { - public ErrorCapturingIdentifierContext name; - public IdentifierListContext columnAliases; - public QueryContext query() { - return getRuleContext(QueryContext.class,0); - } - public ErrorCapturingIdentifierContext errorCapturingIdentifier() { - return getRuleContext(ErrorCapturingIdentifierContext.class,0); - } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public IdentifierListContext identifierList() { - return getRuleContext(IdentifierListContext.class,0); - } - public NamedQueryContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_namedQuery; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterNamedQuery(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitNamedQuery(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitNamedQuery(this); - else return visitor.visitChildren(this); - } - } - - public final NamedQueryContext namedQuery() throws RecognitionException { - NamedQueryContext _localctx = new NamedQueryContext(_ctx, getState()); - enterRule(_localctx, 22, RULE_namedQuery); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(318); - ((NamedQueryContext)_localctx).name = errorCapturingIdentifier(); - setState(320); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,16,_ctx) ) { - case 1: - { - setState(319); - ((NamedQueryContext)_localctx).columnAliases = identifierList(); - } - break; - } - setState(323); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==AS) { - { - setState(322); - match(AS); - } - } - - setState(325); - match(T__1); - setState(326); - query(); - setState(327); - match(T__3); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class TableProviderContext extends ParserRuleContext { - public TerminalNode USING() { return getToken(MixedFormatSqlExtendParser.USING, 0); } - public MultipartIdentifierContext multipartIdentifier() { - return getRuleContext(MultipartIdentifierContext.class,0); - } - public TableProviderContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_tableProvider; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTableProvider(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTableProvider(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTableProvider(this); - else return visitor.visitChildren(this); - } - } - - public final TableProviderContext tableProvider() throws RecognitionException { - TableProviderContext _localctx = new TableProviderContext(_ctx, getState()); - enterRule(_localctx, 24, RULE_tableProvider); - try { - enterOuterAlt(_localctx, 1); - { - setState(329); - match(USING); - setState(330); - multipartIdentifier(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class CreateTableClausesContext extends ParserRuleContext { - public TablePropertyListContext options; - public PartitionFieldListContext partitioning; - public TablePropertyListContext tableProps; - public List skewSpec() { - return getRuleContexts(SkewSpecContext.class); - } - public SkewSpecContext skewSpec(int i) { - return getRuleContext(SkewSpecContext.class,i); - } - public List bucketSpec() { - return getRuleContexts(BucketSpecContext.class); - } - public BucketSpecContext bucketSpec(int i) { - return getRuleContext(BucketSpecContext.class,i); - } - public List rowFormat() { - return getRuleContexts(RowFormatContext.class); - } - public RowFormatContext rowFormat(int i) { - return getRuleContext(RowFormatContext.class,i); - } - public List createFileFormat() { - return getRuleContexts(CreateFileFormatContext.class); - } - public CreateFileFormatContext createFileFormat(int i) { - return getRuleContext(CreateFileFormatContext.class,i); - } - public List locationSpec() { - return getRuleContexts(LocationSpecContext.class); - } - public LocationSpecContext locationSpec(int i) { - return getRuleContext(LocationSpecContext.class,i); - } - public List commentSpec() { - return getRuleContexts(CommentSpecContext.class); - } - public CommentSpecContext commentSpec(int i) { - return getRuleContext(CommentSpecContext.class,i); - } - public List OPTIONS() { return getTokens(MixedFormatSqlExtendParser.OPTIONS); } - public TerminalNode OPTIONS(int i) { - return getToken(MixedFormatSqlExtendParser.OPTIONS, i); - } - public List PARTITIONED() { return getTokens(MixedFormatSqlExtendParser.PARTITIONED); } - public TerminalNode PARTITIONED(int i) { - return getToken(MixedFormatSqlExtendParser.PARTITIONED, i); - } - public List BY() { return getTokens(MixedFormatSqlExtendParser.BY); } - public TerminalNode BY(int i) { - return getToken(MixedFormatSqlExtendParser.BY, i); - } - public List TBLPROPERTIES() { return getTokens(MixedFormatSqlExtendParser.TBLPROPERTIES); } - public TerminalNode TBLPROPERTIES(int i) { - return getToken(MixedFormatSqlExtendParser.TBLPROPERTIES, i); - } - public List tablePropertyList() { - return getRuleContexts(TablePropertyListContext.class); - } - public TablePropertyListContext tablePropertyList(int i) { - return getRuleContext(TablePropertyListContext.class,i); - } - public List partitionFieldList() { - return getRuleContexts(PartitionFieldListContext.class); - } - public PartitionFieldListContext partitionFieldList(int i) { - return getRuleContext(PartitionFieldListContext.class,i); - } - public CreateTableClausesContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_createTableClauses; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterCreateTableClauses(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitCreateTableClauses(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitCreateTableClauses(this); - else return visitor.visitChildren(this); - } - } - - public final CreateTableClausesContext createTableClauses() throws RecognitionException { - CreateTableClausesContext _localctx = new CreateTableClausesContext(_ctx, getState()); - enterRule(_localctx, 26, RULE_createTableClauses); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(347); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==CLUSTERED || _la==COMMENT || ((((_la - 139)) & ~0x3f) == 0 && ((1L << (_la - 139)) & ((1L << (LOCATION - 139)) | (1L << (OPTIONS - 139)) | (1L << (PARTITIONED - 139)))) != 0) || ((((_la - 204)) & ~0x3f) == 0 && ((1L << (_la - 204)) & ((1L << (ROW - 204)) | (1L << (SKEWED - 204)) | (1L << (STORED - 204)) | (1L << (TBLPROPERTIES - 204)))) != 0)) { - { - setState(345); - _errHandler.sync(this); - switch (_input.LA(1)) { - case OPTIONS: - { - { - setState(332); - match(OPTIONS); - setState(333); - ((CreateTableClausesContext)_localctx).options = tablePropertyList(); - } - } - break; - case PARTITIONED: - { - { - setState(334); - match(PARTITIONED); - setState(335); - match(BY); - setState(336); - ((CreateTableClausesContext)_localctx).partitioning = partitionFieldList(); - } - } - break; - case SKEWED: - { - setState(337); - skewSpec(); - } - break; - case CLUSTERED: - { - setState(338); - bucketSpec(); - } - break; - case ROW: - { - setState(339); - rowFormat(); - } - break; - case STORED: - { - setState(340); - createFileFormat(); - } - break; - case LOCATION: - { - setState(341); - locationSpec(); - } - break; - case COMMENT: - { - setState(342); - commentSpec(); - } - break; - case TBLPROPERTIES: - { - { - setState(343); - match(TBLPROPERTIES); - setState(344); - ((CreateTableClausesContext)_localctx).tableProps = tablePropertyList(); - } - } - break; - default: - throw new NoViableAltException(this); - } - } - setState(349); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class TablePropertyListContext extends ParserRuleContext { - public List tableProperty() { - return getRuleContexts(TablePropertyContext.class); - } - public TablePropertyContext tableProperty(int i) { - return getRuleContext(TablePropertyContext.class,i); - } - public TablePropertyListContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_tablePropertyList; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTablePropertyList(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTablePropertyList(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTablePropertyList(this); - else return visitor.visitChildren(this); - } - } - - public final TablePropertyListContext tablePropertyList() throws RecognitionException { - TablePropertyListContext _localctx = new TablePropertyListContext(_ctx, getState()); - enterRule(_localctx, 28, RULE_tablePropertyList); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(350); - match(T__1); - setState(351); - tableProperty(); - setState(356); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(352); - match(T__2); - setState(353); - tableProperty(); - } - } - setState(358); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(359); - match(T__3); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class TablePropertyContext extends ParserRuleContext { - public TablePropertyKeyContext key; - public TablePropertyValueContext value; - public TablePropertyKeyContext tablePropertyKey() { - return getRuleContext(TablePropertyKeyContext.class,0); - } - public TablePropertyValueContext tablePropertyValue() { - return getRuleContext(TablePropertyValueContext.class,0); - } - public TerminalNode EQ() { return getToken(MixedFormatSqlExtendParser.EQ, 0); } - public TablePropertyContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_tableProperty; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTableProperty(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTableProperty(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTableProperty(this); - else return visitor.visitChildren(this); - } - } - - public final TablePropertyContext tableProperty() throws RecognitionException { - TablePropertyContext _localctx = new TablePropertyContext(_ctx, getState()); - enterRule(_localctx, 30, RULE_tableProperty); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(361); - ((TablePropertyContext)_localctx).key = tablePropertyKey(); - setState(366); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==FALSE || ((((_la - 245)) & ~0x3f) == 0 && ((1L << (_la - 245)) & ((1L << (TRUE - 245)) | (1L << (EQ - 245)) | (1L << (STRING - 245)) | (1L << (INTEGER_VALUE - 245)) | (1L << (DECIMAL_VALUE - 245)))) != 0)) { - { - setState(363); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==EQ) { - { - setState(362); - match(EQ); - } - } - - setState(365); - ((TablePropertyContext)_localctx).value = tablePropertyValue(); - } - } - - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class TablePropertyKeyContext extends ParserRuleContext { - public List identifier() { - return getRuleContexts(IdentifierContext.class); - } - public IdentifierContext identifier(int i) { - return getRuleContext(IdentifierContext.class,i); - } - public TerminalNode STRING() { return getToken(MixedFormatSqlExtendParser.STRING, 0); } - public TablePropertyKeyContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_tablePropertyKey; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTablePropertyKey(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTablePropertyKey(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTablePropertyKey(this); - else return visitor.visitChildren(this); - } - } - - public final TablePropertyKeyContext tablePropertyKey() throws RecognitionException { - TablePropertyKeyContext _localctx = new TablePropertyKeyContext(_ctx, getState()); - enterRule(_localctx, 32, RULE_tablePropertyKey); - int _la; - try { - setState(377); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,24,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(368); - identifier(); - setState(373); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__4) { - { - { - setState(369); - match(T__4); - setState(370); - identifier(); - } - } - setState(375); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(376); - match(STRING); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class TablePropertyValueContext extends ParserRuleContext { - public TerminalNode INTEGER_VALUE() { return getToken(MixedFormatSqlExtendParser.INTEGER_VALUE, 0); } - public TerminalNode DECIMAL_VALUE() { return getToken(MixedFormatSqlExtendParser.DECIMAL_VALUE, 0); } - public BooleanValueContext booleanValue() { - return getRuleContext(BooleanValueContext.class,0); - } - public TerminalNode STRING() { return getToken(MixedFormatSqlExtendParser.STRING, 0); } - public TablePropertyValueContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_tablePropertyValue; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTablePropertyValue(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTablePropertyValue(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTablePropertyValue(this); - else return visitor.visitChildren(this); - } - } - - public final TablePropertyValueContext tablePropertyValue() throws RecognitionException { - TablePropertyValueContext _localctx = new TablePropertyValueContext(_ctx, getState()); - enterRule(_localctx, 34, RULE_tablePropertyValue); - try { - setState(383); - _errHandler.sync(this); - switch (_input.LA(1)) { - case INTEGER_VALUE: - enterOuterAlt(_localctx, 1); - { - setState(379); - match(INTEGER_VALUE); - } - break; - case DECIMAL_VALUE: - enterOuterAlt(_localctx, 2); - { - setState(380); - match(DECIMAL_VALUE); - } - break; - case FALSE: - case TRUE: - enterOuterAlt(_localctx, 3); - { - setState(381); - booleanValue(); - } - break; - case STRING: - enterOuterAlt(_localctx, 4); - { - setState(382); - match(STRING); - } - break; - default: - throw new NoViableAltException(this); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ConstantListContext extends ParserRuleContext { - public List constant() { - return getRuleContexts(ConstantContext.class); - } - public ConstantContext constant(int i) { - return getRuleContext(ConstantContext.class,i); - } - public ConstantListContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_constantList; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterConstantList(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitConstantList(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitConstantList(this); - else return visitor.visitChildren(this); - } - } - - public final ConstantListContext constantList() throws RecognitionException { - ConstantListContext _localctx = new ConstantListContext(_ctx, getState()); - enterRule(_localctx, 36, RULE_constantList); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(385); - match(T__1); - setState(386); - constant(); - setState(391); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(387); - match(T__2); - setState(388); - constant(); - } - } - setState(393); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(394); - match(T__3); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class NestedConstantListContext extends ParserRuleContext { - public List constantList() { - return getRuleContexts(ConstantListContext.class); - } - public ConstantListContext constantList(int i) { - return getRuleContext(ConstantListContext.class,i); - } - public NestedConstantListContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_nestedConstantList; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterNestedConstantList(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitNestedConstantList(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitNestedConstantList(this); - else return visitor.visitChildren(this); - } - } - - public final NestedConstantListContext nestedConstantList() throws RecognitionException { - NestedConstantListContext _localctx = new NestedConstantListContext(_ctx, getState()); - enterRule(_localctx, 38, RULE_nestedConstantList); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(396); - match(T__1); - setState(397); - constantList(); - setState(402); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(398); - match(T__2); - setState(399); - constantList(); - } - } - setState(404); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(405); - match(T__3); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class CreateFileFormatContext extends ParserRuleContext { - public TerminalNode STORED() { return getToken(MixedFormatSqlExtendParser.STORED, 0); } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public FileFormatContext fileFormat() { - return getRuleContext(FileFormatContext.class,0); - } - public TerminalNode BY() { return getToken(MixedFormatSqlExtendParser.BY, 0); } - public StorageHandlerContext storageHandler() { - return getRuleContext(StorageHandlerContext.class,0); - } - public CreateFileFormatContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_createFileFormat; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterCreateFileFormat(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitCreateFileFormat(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitCreateFileFormat(this); - else return visitor.visitChildren(this); - } - } - - public final CreateFileFormatContext createFileFormat() throws RecognitionException { - CreateFileFormatContext _localctx = new CreateFileFormatContext(_ctx, getState()); - enterRule(_localctx, 40, RULE_createFileFormat); - try { - setState(413); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,28,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(407); - match(STORED); - setState(408); - match(AS); - setState(409); - fileFormat(); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(410); - match(STORED); - setState(411); - match(BY); - setState(412); - storageHandler(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class FileFormatContext extends ParserRuleContext { - public FileFormatContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_fileFormat; } - - public FileFormatContext() { } - public void copyFrom(FileFormatContext ctx) { - super.copyFrom(ctx); - } - } - public static class TableFileFormatContext extends FileFormatContext { - public Token inFmt; - public Token outFmt; - public TerminalNode INPUTFORMAT() { return getToken(MixedFormatSqlExtendParser.INPUTFORMAT, 0); } - public TerminalNode OUTPUTFORMAT() { return getToken(MixedFormatSqlExtendParser.OUTPUTFORMAT, 0); } - public List STRING() { return getTokens(MixedFormatSqlExtendParser.STRING); } - public TerminalNode STRING(int i) { - return getToken(MixedFormatSqlExtendParser.STRING, i); - } - public TableFileFormatContext(FileFormatContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTableFileFormat(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTableFileFormat(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTableFileFormat(this); - else return visitor.visitChildren(this); - } - } - public static class GenericFileFormatContext extends FileFormatContext { - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public GenericFileFormatContext(FileFormatContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterGenericFileFormat(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitGenericFileFormat(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitGenericFileFormat(this); - else return visitor.visitChildren(this); - } - } - - public final FileFormatContext fileFormat() throws RecognitionException { - FileFormatContext _localctx = new FileFormatContext(_ctx, getState()); - enterRule(_localctx, 42, RULE_fileFormat); - try { - setState(420); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,29,_ctx) ) { - case 1: - _localctx = new TableFileFormatContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(415); - match(INPUTFORMAT); - setState(416); - ((TableFileFormatContext)_localctx).inFmt = match(STRING); - setState(417); - match(OUTPUTFORMAT); - setState(418); - ((TableFileFormatContext)_localctx).outFmt = match(STRING); - } - break; - case 2: - _localctx = new GenericFileFormatContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(419); - identifier(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class StorageHandlerContext extends ParserRuleContext { - public TerminalNode STRING() { return getToken(MixedFormatSqlExtendParser.STRING, 0); } - public TerminalNode WITH() { return getToken(MixedFormatSqlExtendParser.WITH, 0); } - public TerminalNode SERDEPROPERTIES() { return getToken(MixedFormatSqlExtendParser.SERDEPROPERTIES, 0); } - public TablePropertyListContext tablePropertyList() { - return getRuleContext(TablePropertyListContext.class,0); - } - public StorageHandlerContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_storageHandler; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterStorageHandler(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitStorageHandler(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitStorageHandler(this); - else return visitor.visitChildren(this); - } - } - - public final StorageHandlerContext storageHandler() throws RecognitionException { - StorageHandlerContext _localctx = new StorageHandlerContext(_ctx, getState()); - enterRule(_localctx, 44, RULE_storageHandler); - try { - enterOuterAlt(_localctx, 1); - { - setState(422); - match(STRING); - setState(426); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,30,_ctx) ) { - case 1: - { - setState(423); - match(WITH); - setState(424); - match(SERDEPROPERTIES); - setState(425); - tablePropertyList(); - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class QueryOrganizationContext extends ParserRuleContext { - public SortItemContext sortItem; - public List order = new ArrayList(); - public ExpressionContext expression; - public List clusterBy = new ArrayList(); - public List distributeBy = new ArrayList(); - public List sort = new ArrayList(); - public ExpressionContext limit; - public TerminalNode ORDER() { return getToken(MixedFormatSqlExtendParser.ORDER, 0); } - public List BY() { return getTokens(MixedFormatSqlExtendParser.BY); } - public TerminalNode BY(int i) { - return getToken(MixedFormatSqlExtendParser.BY, i); - } - public TerminalNode CLUSTER() { return getToken(MixedFormatSqlExtendParser.CLUSTER, 0); } - public TerminalNode DISTRIBUTE() { return getToken(MixedFormatSqlExtendParser.DISTRIBUTE, 0); } - public TerminalNode SORT() { return getToken(MixedFormatSqlExtendParser.SORT, 0); } - public WindowClauseContext windowClause() { - return getRuleContext(WindowClauseContext.class,0); - } - public TerminalNode LIMIT() { return getToken(MixedFormatSqlExtendParser.LIMIT, 0); } - public List sortItem() { - return getRuleContexts(SortItemContext.class); - } - public SortItemContext sortItem(int i) { - return getRuleContext(SortItemContext.class,i); - } - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public TerminalNode ALL() { return getToken(MixedFormatSqlExtendParser.ALL, 0); } - public QueryOrganizationContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_queryOrganization; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterQueryOrganization(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitQueryOrganization(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitQueryOrganization(this); - else return visitor.visitChildren(this); - } - } - - public final QueryOrganizationContext queryOrganization() throws RecognitionException { - QueryOrganizationContext _localctx = new QueryOrganizationContext(_ctx, getState()); - enterRule(_localctx, 46, RULE_queryOrganization); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(438); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,32,_ctx) ) { - case 1: - { - setState(428); - match(ORDER); - setState(429); - match(BY); - setState(430); - ((QueryOrganizationContext)_localctx).sortItem = sortItem(); - ((QueryOrganizationContext)_localctx).order.add(((QueryOrganizationContext)_localctx).sortItem); - setState(435); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,31,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(431); - match(T__2); - setState(432); - ((QueryOrganizationContext)_localctx).sortItem = sortItem(); - ((QueryOrganizationContext)_localctx).order.add(((QueryOrganizationContext)_localctx).sortItem); - } - } - } - setState(437); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,31,_ctx); - } - } - break; - } - setState(450); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,34,_ctx) ) { - case 1: - { - setState(440); - match(CLUSTER); - setState(441); - match(BY); - setState(442); - ((QueryOrganizationContext)_localctx).expression = expression(); - ((QueryOrganizationContext)_localctx).clusterBy.add(((QueryOrganizationContext)_localctx).expression); - setState(447); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,33,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(443); - match(T__2); - setState(444); - ((QueryOrganizationContext)_localctx).expression = expression(); - ((QueryOrganizationContext)_localctx).clusterBy.add(((QueryOrganizationContext)_localctx).expression); - } - } - } - setState(449); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,33,_ctx); - } - } - break; - } - setState(462); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,36,_ctx) ) { - case 1: - { - setState(452); - match(DISTRIBUTE); - setState(453); - match(BY); - setState(454); - ((QueryOrganizationContext)_localctx).expression = expression(); - ((QueryOrganizationContext)_localctx).distributeBy.add(((QueryOrganizationContext)_localctx).expression); - setState(459); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,35,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(455); - match(T__2); - setState(456); - ((QueryOrganizationContext)_localctx).expression = expression(); - ((QueryOrganizationContext)_localctx).distributeBy.add(((QueryOrganizationContext)_localctx).expression); - } - } - } - setState(461); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,35,_ctx); - } - } - break; - } - setState(474); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,38,_ctx) ) { - case 1: - { - setState(464); - match(SORT); - setState(465); - match(BY); - setState(466); - ((QueryOrganizationContext)_localctx).sortItem = sortItem(); - ((QueryOrganizationContext)_localctx).sort.add(((QueryOrganizationContext)_localctx).sortItem); - setState(471); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,37,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(467); - match(T__2); - setState(468); - ((QueryOrganizationContext)_localctx).sortItem = sortItem(); - ((QueryOrganizationContext)_localctx).sort.add(((QueryOrganizationContext)_localctx).sortItem); - } - } - } - setState(473); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,37,_ctx); - } - } - break; - } - setState(477); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,39,_ctx) ) { - case 1: - { - setState(476); - windowClause(); - } - break; - } - setState(484); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,41,_ctx) ) { - case 1: - { - setState(479); - match(LIMIT); - setState(482); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,40,_ctx) ) { - case 1: - { - setState(480); - match(ALL); - } - break; - case 2: - { - setState(481); - ((QueryOrganizationContext)_localctx).limit = expression(); - } - break; - } - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class QueryTermContext extends ParserRuleContext { - public QueryTermContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_queryTerm; } - - public QueryTermContext() { } - public void copyFrom(QueryTermContext ctx) { - super.copyFrom(ctx); - } - } - public static class QueryTermDefaultContext extends QueryTermContext { - public QueryPrimaryContext queryPrimary() { - return getRuleContext(QueryPrimaryContext.class,0); - } - public QueryTermDefaultContext(QueryTermContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterQueryTermDefault(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitQueryTermDefault(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitQueryTermDefault(this); - else return visitor.visitChildren(this); - } - } - public static class SetOperationContext extends QueryTermContext { - public QueryTermContext left; - public Token operator; - public QueryTermContext right; - public List queryTerm() { - return getRuleContexts(QueryTermContext.class); - } - public QueryTermContext queryTerm(int i) { - return getRuleContext(QueryTermContext.class,i); - } - public TerminalNode INTERSECT() { return getToken(MixedFormatSqlExtendParser.INTERSECT, 0); } - public TerminalNode UNION() { return getToken(MixedFormatSqlExtendParser.UNION, 0); } - public TerminalNode EXCEPT() { return getToken(MixedFormatSqlExtendParser.EXCEPT, 0); } - public TerminalNode SETMINUS() { return getToken(MixedFormatSqlExtendParser.SETMINUS, 0); } - public SetQuantifierContext setQuantifier() { - return getRuleContext(SetQuantifierContext.class,0); - } - public SetOperationContext(QueryTermContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSetOperation(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSetOperation(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSetOperation(this); - else return visitor.visitChildren(this); - } - } - - public final QueryTermContext queryTerm() throws RecognitionException { - return queryTerm(0); - } - - private QueryTermContext queryTerm(int _p) throws RecognitionException { - ParserRuleContext _parentctx = _ctx; - int _parentState = getState(); - QueryTermContext _localctx = new QueryTermContext(_ctx, _parentState); - QueryTermContext _prevctx = _localctx; - int _startState = 48; - enterRecursionRule(_localctx, 48, RULE_queryTerm, _p); - int _la; - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - { - _localctx = new QueryTermDefaultContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - - setState(487); - queryPrimary(); - } - _ctx.stop = _input.LT(-1); - setState(512); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,46,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - if ( _parseListeners!=null ) triggerExitRuleEvent(); - _prevctx = _localctx; - { - setState(510); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,45,_ctx) ) { - case 1: - { - _localctx = new SetOperationContext(new QueryTermContext(_parentctx, _parentState)); - ((SetOperationContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_queryTerm); - setState(489); - if (!(precpred(_ctx, 3))) throw new FailedPredicateException(this, "precpred(_ctx, 3)"); - setState(490); - if (!(legacy_setops_precedence_enabled)) throw new FailedPredicateException(this, "legacy_setops_precedence_enabled"); - setState(491); - ((SetOperationContext)_localctx).operator = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==EXCEPT || _la==INTERSECT || _la==SETMINUS || _la==UNION) ) { - ((SetOperationContext)_localctx).operator = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(493); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==ALL || _la==DISTINCT) { - { - setState(492); - setQuantifier(); - } - } - - setState(495); - ((SetOperationContext)_localctx).right = queryTerm(4); - } - break; - case 2: - { - _localctx = new SetOperationContext(new QueryTermContext(_parentctx, _parentState)); - ((SetOperationContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_queryTerm); - setState(496); - if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)"); - setState(497); - if (!(!legacy_setops_precedence_enabled)) throw new FailedPredicateException(this, "!legacy_setops_precedence_enabled"); - setState(498); - ((SetOperationContext)_localctx).operator = match(INTERSECT); - setState(500); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==ALL || _la==DISTINCT) { - { - setState(499); - setQuantifier(); - } - } - - setState(502); - ((SetOperationContext)_localctx).right = queryTerm(3); - } - break; - case 3: - { - _localctx = new SetOperationContext(new QueryTermContext(_parentctx, _parentState)); - ((SetOperationContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_queryTerm); - setState(503); - if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)"); - setState(504); - if (!(!legacy_setops_precedence_enabled)) throw new FailedPredicateException(this, "!legacy_setops_precedence_enabled"); - setState(505); - ((SetOperationContext)_localctx).operator = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==EXCEPT || _la==SETMINUS || _la==UNION) ) { - ((SetOperationContext)_localctx).operator = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(507); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==ALL || _la==DISTINCT) { - { - setState(506); - setQuantifier(); - } - } - - setState(509); - ((SetOperationContext)_localctx).right = queryTerm(2); - } - break; - } - } - } - setState(514); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,46,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - unrollRecursionContexts(_parentctx); - } - return _localctx; - } - - public static class QueryPrimaryContext extends ParserRuleContext { - public QueryPrimaryContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_queryPrimary; } - - public QueryPrimaryContext() { } - public void copyFrom(QueryPrimaryContext ctx) { - super.copyFrom(ctx); - } - } - public static class SubqueryContext extends QueryPrimaryContext { - public QueryContext query() { - return getRuleContext(QueryContext.class,0); - } - public SubqueryContext(QueryPrimaryContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSubquery(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSubquery(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSubquery(this); - else return visitor.visitChildren(this); - } - } - public static class QueryPrimaryDefaultContext extends QueryPrimaryContext { - public QuerySpecificationContext querySpecification() { - return getRuleContext(QuerySpecificationContext.class,0); - } - public QueryPrimaryDefaultContext(QueryPrimaryContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterQueryPrimaryDefault(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitQueryPrimaryDefault(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitQueryPrimaryDefault(this); - else return visitor.visitChildren(this); - } - } - public static class InlineTableDefault1Context extends QueryPrimaryContext { - public InlineTableContext inlineTable() { - return getRuleContext(InlineTableContext.class,0); - } - public InlineTableDefault1Context(QueryPrimaryContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterInlineTableDefault1(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitInlineTableDefault1(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitInlineTableDefault1(this); - else return visitor.visitChildren(this); - } - } - public static class FromStmtContext extends QueryPrimaryContext { - public FromStatementContext fromStatement() { - return getRuleContext(FromStatementContext.class,0); - } - public FromStmtContext(QueryPrimaryContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterFromStmt(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitFromStmt(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitFromStmt(this); - else return visitor.visitChildren(this); - } - } - public static class TableContext extends QueryPrimaryContext { - public TerminalNode TABLE() { return getToken(MixedFormatSqlExtendParser.TABLE, 0); } - public MultipartIdentifierContext multipartIdentifier() { - return getRuleContext(MultipartIdentifierContext.class,0); - } - public TableContext(QueryPrimaryContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTable(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTable(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTable(this); - else return visitor.visitChildren(this); - } - } - - public final QueryPrimaryContext queryPrimary() throws RecognitionException { - QueryPrimaryContext _localctx = new QueryPrimaryContext(_ctx, getState()); - enterRule(_localctx, 50, RULE_queryPrimary); - try { - setState(524); - _errHandler.sync(this); - switch (_input.LA(1)) { - case MAP: - case REDUCE: - case SELECT: - _localctx = new QueryPrimaryDefaultContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(515); - querySpecification(); - } - break; - case FROM: - _localctx = new FromStmtContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(516); - fromStatement(); - } - break; - case TABLE: - _localctx = new TableContext(_localctx); - enterOuterAlt(_localctx, 3); - { - setState(517); - match(TABLE); - setState(518); - multipartIdentifier(); - } - break; - case VALUES: - _localctx = new InlineTableDefault1Context(_localctx); - enterOuterAlt(_localctx, 4); - { - setState(519); - inlineTable(); - } - break; - case T__1: - _localctx = new SubqueryContext(_localctx); - enterOuterAlt(_localctx, 5); - { - setState(520); - match(T__1); - setState(521); - query(); - setState(522); - match(T__3); - } - break; - default: - throw new NoViableAltException(this); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class SortItemContext extends ParserRuleContext { - public Token ordering; - public Token nullOrder; - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public TerminalNode NULLS() { return getToken(MixedFormatSqlExtendParser.NULLS, 0); } - public TerminalNode ASC() { return getToken(MixedFormatSqlExtendParser.ASC, 0); } - public TerminalNode DESC() { return getToken(MixedFormatSqlExtendParser.DESC, 0); } - public TerminalNode LAST() { return getToken(MixedFormatSqlExtendParser.LAST, 0); } - public TerminalNode FIRST() { return getToken(MixedFormatSqlExtendParser.FIRST, 0); } - public SortItemContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_sortItem; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSortItem(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSortItem(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSortItem(this); - else return visitor.visitChildren(this); - } - } - - public final SortItemContext sortItem() throws RecognitionException { - SortItemContext _localctx = new SortItemContext(_ctx, getState()); - enterRule(_localctx, 52, RULE_sortItem); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(526); - expression(); - setState(528); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,48,_ctx) ) { - case 1: - { - setState(527); - ((SortItemContext)_localctx).ordering = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==ASC || _la==DESC) ) { - ((SortItemContext)_localctx).ordering = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - break; - } - setState(532); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,49,_ctx) ) { - case 1: - { - setState(530); - match(NULLS); - setState(531); - ((SortItemContext)_localctx).nullOrder = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==FIRST || _la==LAST) ) { - ((SortItemContext)_localctx).nullOrder = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class FromStatementContext extends ParserRuleContext { - public FromClauseContext fromClause() { - return getRuleContext(FromClauseContext.class,0); - } - public List fromStatementBody() { - return getRuleContexts(FromStatementBodyContext.class); - } - public FromStatementBodyContext fromStatementBody(int i) { - return getRuleContext(FromStatementBodyContext.class,i); - } - public FromStatementContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_fromStatement; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterFromStatement(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitFromStatement(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitFromStatement(this); - else return visitor.visitChildren(this); - } - } - - public final FromStatementContext fromStatement() throws RecognitionException { - FromStatementContext _localctx = new FromStatementContext(_ctx, getState()); - enterRule(_localctx, 54, RULE_fromStatement); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(534); - fromClause(); - setState(536); - _errHandler.sync(this); - _alt = 1; - do { - switch (_alt) { - case 1: - { - { - setState(535); - fromStatementBody(); - } - } - break; - default: - throw new NoViableAltException(this); - } - setState(538); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,50,_ctx); - } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class FromStatementBodyContext extends ParserRuleContext { - public TransformClauseContext transformClause() { - return getRuleContext(TransformClauseContext.class,0); - } - public QueryOrganizationContext queryOrganization() { - return getRuleContext(QueryOrganizationContext.class,0); - } - public WhereClauseContext whereClause() { - return getRuleContext(WhereClauseContext.class,0); - } - public SelectClauseContext selectClause() { - return getRuleContext(SelectClauseContext.class,0); - } - public List lateralView() { - return getRuleContexts(LateralViewContext.class); - } - public LateralViewContext lateralView(int i) { - return getRuleContext(LateralViewContext.class,i); - } - public AggregationClauseContext aggregationClause() { - return getRuleContext(AggregationClauseContext.class,0); - } - public HavingClauseContext havingClause() { - return getRuleContext(HavingClauseContext.class,0); - } - public WindowClauseContext windowClause() { - return getRuleContext(WindowClauseContext.class,0); - } - public FromStatementBodyContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_fromStatementBody; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterFromStatementBody(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitFromStatementBody(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitFromStatementBody(this); - else return visitor.visitChildren(this); - } - } - - public final FromStatementBodyContext fromStatementBody() throws RecognitionException { - FromStatementBodyContext _localctx = new FromStatementBodyContext(_ctx, getState()); - enterRule(_localctx, 56, RULE_fromStatementBody); - try { - int _alt; - setState(567); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,57,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(540); - transformClause(); - setState(542); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,51,_ctx) ) { - case 1: - { - setState(541); - whereClause(); - } - break; - } - setState(544); - queryOrganization(); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(546); - selectClause(); - setState(550); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,52,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(547); - lateralView(); - } - } - } - setState(552); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,52,_ctx); - } - setState(554); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,53,_ctx) ) { - case 1: - { - setState(553); - whereClause(); - } - break; - } - setState(557); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,54,_ctx) ) { - case 1: - { - setState(556); - aggregationClause(); - } - break; - } - setState(560); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,55,_ctx) ) { - case 1: - { - setState(559); - havingClause(); - } - break; - } - setState(563); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,56,_ctx) ) { - case 1: - { - setState(562); - windowClause(); - } - break; - } - setState(565); - queryOrganization(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class QuerySpecificationContext extends ParserRuleContext { - public QuerySpecificationContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_querySpecification; } - - public QuerySpecificationContext() { } - public void copyFrom(QuerySpecificationContext ctx) { - super.copyFrom(ctx); - } - } - public static class RegularQuerySpecificationContext extends QuerySpecificationContext { - public SelectClauseContext selectClause() { - return getRuleContext(SelectClauseContext.class,0); - } - public FromClauseContext fromClause() { - return getRuleContext(FromClauseContext.class,0); - } - public List lateralView() { - return getRuleContexts(LateralViewContext.class); - } - public LateralViewContext lateralView(int i) { - return getRuleContext(LateralViewContext.class,i); - } - public WhereClauseContext whereClause() { - return getRuleContext(WhereClauseContext.class,0); - } - public AggregationClauseContext aggregationClause() { - return getRuleContext(AggregationClauseContext.class,0); - } - public HavingClauseContext havingClause() { - return getRuleContext(HavingClauseContext.class,0); - } - public WindowClauseContext windowClause() { - return getRuleContext(WindowClauseContext.class,0); - } - public RegularQuerySpecificationContext(QuerySpecificationContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterRegularQuerySpecification(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitRegularQuerySpecification(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitRegularQuerySpecification(this); - else return visitor.visitChildren(this); - } - } - public static class TransformQuerySpecificationContext extends QuerySpecificationContext { - public TransformClauseContext transformClause() { - return getRuleContext(TransformClauseContext.class,0); - } - public FromClauseContext fromClause() { - return getRuleContext(FromClauseContext.class,0); - } - public List lateralView() { - return getRuleContexts(LateralViewContext.class); - } - public LateralViewContext lateralView(int i) { - return getRuleContext(LateralViewContext.class,i); - } - public WhereClauseContext whereClause() { - return getRuleContext(WhereClauseContext.class,0); - } - public AggregationClauseContext aggregationClause() { - return getRuleContext(AggregationClauseContext.class,0); - } - public HavingClauseContext havingClause() { - return getRuleContext(HavingClauseContext.class,0); - } - public WindowClauseContext windowClause() { - return getRuleContext(WindowClauseContext.class,0); - } - public TransformQuerySpecificationContext(QuerySpecificationContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTransformQuerySpecification(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTransformQuerySpecification(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTransformQuerySpecification(this); - else return visitor.visitChildren(this); - } - } - - public final QuerySpecificationContext querySpecification() throws RecognitionException { - QuerySpecificationContext _localctx = new QuerySpecificationContext(_ctx, getState()); - enterRule(_localctx, 58, RULE_querySpecification); - try { - int _alt; - setState(613); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,70,_ctx) ) { - case 1: - _localctx = new TransformQuerySpecificationContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(569); - transformClause(); - setState(571); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,58,_ctx) ) { - case 1: - { - setState(570); - fromClause(); - } - break; - } - setState(576); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,59,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(573); - lateralView(); - } - } - } - setState(578); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,59,_ctx); - } - setState(580); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,60,_ctx) ) { - case 1: - { - setState(579); - whereClause(); - } - break; - } - setState(583); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,61,_ctx) ) { - case 1: - { - setState(582); - aggregationClause(); - } - break; - } - setState(586); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,62,_ctx) ) { - case 1: - { - setState(585); - havingClause(); - } - break; - } - setState(589); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,63,_ctx) ) { - case 1: - { - setState(588); - windowClause(); - } - break; - } - } - break; - case 2: - _localctx = new RegularQuerySpecificationContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(591); - selectClause(); - setState(593); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,64,_ctx) ) { - case 1: - { - setState(592); - fromClause(); - } - break; - } - setState(598); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,65,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(595); - lateralView(); - } - } - } - setState(600); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,65,_ctx); - } - setState(602); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,66,_ctx) ) { - case 1: - { - setState(601); - whereClause(); - } - break; - } - setState(605); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,67,_ctx) ) { - case 1: - { - setState(604); - aggregationClause(); - } - break; - } - setState(608); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,68,_ctx) ) { - case 1: - { - setState(607); - havingClause(); - } - break; - } - setState(611); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,69,_ctx) ) { - case 1: - { - setState(610); - windowClause(); - } - break; - } - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class TransformClauseContext extends ParserRuleContext { - public Token kind; - public RowFormatContext inRowFormat; - public Token recordWriter; - public Token script; - public RowFormatContext outRowFormat; - public Token recordReader; - public TerminalNode USING() { return getToken(MixedFormatSqlExtendParser.USING, 0); } - public List STRING() { return getTokens(MixedFormatSqlExtendParser.STRING); } - public TerminalNode STRING(int i) { - return getToken(MixedFormatSqlExtendParser.STRING, i); - } - public TerminalNode SELECT() { return getToken(MixedFormatSqlExtendParser.SELECT, 0); } - public ExpressionSeqContext expressionSeq() { - return getRuleContext(ExpressionSeqContext.class,0); - } - public TerminalNode TRANSFORM() { return getToken(MixedFormatSqlExtendParser.TRANSFORM, 0); } - public TerminalNode MAP() { return getToken(MixedFormatSqlExtendParser.MAP, 0); } - public TerminalNode REDUCE() { return getToken(MixedFormatSqlExtendParser.REDUCE, 0); } - public TerminalNode RECORDWRITER() { return getToken(MixedFormatSqlExtendParser.RECORDWRITER, 0); } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public TerminalNode RECORDREADER() { return getToken(MixedFormatSqlExtendParser.RECORDREADER, 0); } - public List rowFormat() { - return getRuleContexts(RowFormatContext.class); - } - public RowFormatContext rowFormat(int i) { - return getRuleContext(RowFormatContext.class,i); - } - public SetQuantifierContext setQuantifier() { - return getRuleContext(SetQuantifierContext.class,0); - } - public IdentifierSeqContext identifierSeq() { - return getRuleContext(IdentifierSeqContext.class,0); - } - public ColTypeListContext colTypeList() { - return getRuleContext(ColTypeListContext.class,0); - } - public TransformClauseContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_transformClause; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTransformClause(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTransformClause(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTransformClause(this); - else return visitor.visitChildren(this); - } - } - - public final TransformClauseContext transformClause() throws RecognitionException { - TransformClauseContext _localctx = new TransformClauseContext(_ctx, getState()); - enterRule(_localctx, 60, RULE_transformClause); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(634); - _errHandler.sync(this); - switch (_input.LA(1)) { - case SELECT: - { - setState(615); - match(SELECT); - setState(616); - ((TransformClauseContext)_localctx).kind = match(TRANSFORM); - setState(617); - match(T__1); - setState(619); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,71,_ctx) ) { - case 1: - { - setState(618); - setQuantifier(); - } - break; - } - setState(621); - expressionSeq(); - setState(622); - match(T__3); - } - break; - case MAP: - { - setState(624); - ((TransformClauseContext)_localctx).kind = match(MAP); - setState(626); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,72,_ctx) ) { - case 1: - { - setState(625); - setQuantifier(); - } - break; - } - setState(628); - expressionSeq(); - } - break; - case REDUCE: - { - setState(629); - ((TransformClauseContext)_localctx).kind = match(REDUCE); - setState(631); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,73,_ctx) ) { - case 1: - { - setState(630); - setQuantifier(); - } - break; - } - setState(633); - expressionSeq(); - } - break; - default: - throw new NoViableAltException(this); - } - setState(637); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==ROW) { - { - setState(636); - ((TransformClauseContext)_localctx).inRowFormat = rowFormat(); - } - } - - setState(641); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==RECORDWRITER) { - { - setState(639); - match(RECORDWRITER); - setState(640); - ((TransformClauseContext)_localctx).recordWriter = match(STRING); - } - } - - setState(643); - match(USING); - setState(644); - ((TransformClauseContext)_localctx).script = match(STRING); - setState(657); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,79,_ctx) ) { - case 1: - { - setState(645); - match(AS); - setState(655); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,78,_ctx) ) { - case 1: - { - setState(646); - identifierSeq(); - } - break; - case 2: - { - setState(647); - colTypeList(); - } - break; - case 3: - { - { - setState(648); - match(T__1); - setState(651); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,77,_ctx) ) { - case 1: - { - setState(649); - identifierSeq(); - } - break; - case 2: - { - setState(650); - colTypeList(); - } - break; - } - setState(653); - match(T__3); - } - } - break; - } - } - break; - } - setState(660); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,80,_ctx) ) { - case 1: - { - setState(659); - ((TransformClauseContext)_localctx).outRowFormat = rowFormat(); - } - break; - } - setState(664); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,81,_ctx) ) { - case 1: - { - setState(662); - match(RECORDREADER); - setState(663); - ((TransformClauseContext)_localctx).recordReader = match(STRING); - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class SelectClauseContext extends ParserRuleContext { - public HintContext hint; - public List hints = new ArrayList(); - public TerminalNode SELECT() { return getToken(MixedFormatSqlExtendParser.SELECT, 0); } - public NamedExpressionSeqContext namedExpressionSeq() { - return getRuleContext(NamedExpressionSeqContext.class,0); - } - public SetQuantifierContext setQuantifier() { - return getRuleContext(SetQuantifierContext.class,0); - } - public List hint() { - return getRuleContexts(HintContext.class); - } - public HintContext hint(int i) { - return getRuleContext(HintContext.class,i); - } - public SelectClauseContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_selectClause; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSelectClause(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSelectClause(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSelectClause(this); - else return visitor.visitChildren(this); - } - } - - public final SelectClauseContext selectClause() throws RecognitionException { - SelectClauseContext _localctx = new SelectClauseContext(_ctx, getState()); - enterRule(_localctx, 62, RULE_selectClause); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(666); - match(SELECT); - setState(670); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,82,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(667); - ((SelectClauseContext)_localctx).hint = hint(); - ((SelectClauseContext)_localctx).hints.add(((SelectClauseContext)_localctx).hint); - } - } - } - setState(672); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,82,_ctx); - } - setState(674); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,83,_ctx) ) { - case 1: - { - setState(673); - setQuantifier(); - } - break; - } - setState(676); - namedExpressionSeq(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class WhereClauseContext extends ParserRuleContext { - public TerminalNode WHERE() { return getToken(MixedFormatSqlExtendParser.WHERE, 0); } - public BooleanExpressionContext booleanExpression() { - return getRuleContext(BooleanExpressionContext.class,0); - } - public WhereClauseContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_whereClause; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterWhereClause(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitWhereClause(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitWhereClause(this); - else return visitor.visitChildren(this); - } - } - - public final WhereClauseContext whereClause() throws RecognitionException { - WhereClauseContext _localctx = new WhereClauseContext(_ctx, getState()); - enterRule(_localctx, 64, RULE_whereClause); - try { - enterOuterAlt(_localctx, 1); - { - setState(678); - match(WHERE); - setState(679); - booleanExpression(0); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class HavingClauseContext extends ParserRuleContext { - public TerminalNode HAVING() { return getToken(MixedFormatSqlExtendParser.HAVING, 0); } - public BooleanExpressionContext booleanExpression() { - return getRuleContext(BooleanExpressionContext.class,0); - } - public HavingClauseContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_havingClause; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterHavingClause(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitHavingClause(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitHavingClause(this); - else return visitor.visitChildren(this); - } - } - - public final HavingClauseContext havingClause() throws RecognitionException { - HavingClauseContext _localctx = new HavingClauseContext(_ctx, getState()); - enterRule(_localctx, 66, RULE_havingClause); - try { - enterOuterAlt(_localctx, 1); - { - setState(681); - match(HAVING); - setState(682); - booleanExpression(0); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class HintContext extends ParserRuleContext { - public HintStatementContext hintStatement; - public List hintStatements = new ArrayList(); - public List hintStatement() { - return getRuleContexts(HintStatementContext.class); - } - public HintStatementContext hintStatement(int i) { - return getRuleContext(HintStatementContext.class,i); - } - public HintContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_hint; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterHint(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitHint(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitHint(this); - else return visitor.visitChildren(this); - } - } - - public final HintContext hint() throws RecognitionException { - HintContext _localctx = new HintContext(_ctx, getState()); - enterRule(_localctx, 68, RULE_hint); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(684); - match(T__5); - setState(685); - ((HintContext)_localctx).hintStatement = hintStatement(); - ((HintContext)_localctx).hintStatements.add(((HintContext)_localctx).hintStatement); - setState(692); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,85,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(687); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,84,_ctx) ) { - case 1: - { - setState(686); - match(T__2); - } - break; - } - setState(689); - ((HintContext)_localctx).hintStatement = hintStatement(); - ((HintContext)_localctx).hintStatements.add(((HintContext)_localctx).hintStatement); - } - } - } - setState(694); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,85,_ctx); - } - setState(695); - match(T__6); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class HintStatementContext extends ParserRuleContext { - public IdentifierContext hintName; - public PrimaryExpressionContext primaryExpression; - public List parameters = new ArrayList(); - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public List primaryExpression() { - return getRuleContexts(PrimaryExpressionContext.class); - } - public PrimaryExpressionContext primaryExpression(int i) { - return getRuleContext(PrimaryExpressionContext.class,i); - } - public HintStatementContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_hintStatement; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterHintStatement(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitHintStatement(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitHintStatement(this); - else return visitor.visitChildren(this); - } - } - - public final HintStatementContext hintStatement() throws RecognitionException { - HintStatementContext _localctx = new HintStatementContext(_ctx, getState()); - enterRule(_localctx, 70, RULE_hintStatement); - int _la; - try { - setState(710); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,87,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(697); - ((HintStatementContext)_localctx).hintName = identifier(); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(698); - ((HintStatementContext)_localctx).hintName = identifier(); - setState(699); - match(T__1); - setState(700); - ((HintStatementContext)_localctx).primaryExpression = primaryExpression(0); - ((HintStatementContext)_localctx).parameters.add(((HintStatementContext)_localctx).primaryExpression); - setState(705); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(701); - match(T__2); - setState(702); - ((HintStatementContext)_localctx).primaryExpression = primaryExpression(0); - ((HintStatementContext)_localctx).parameters.add(((HintStatementContext)_localctx).primaryExpression); - } - } - setState(707); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(708); - match(T__3); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class FromClauseContext extends ParserRuleContext { - public TerminalNode FROM() { return getToken(MixedFormatSqlExtendParser.FROM, 0); } - public List relation() { - return getRuleContexts(RelationContext.class); - } - public RelationContext relation(int i) { - return getRuleContext(RelationContext.class,i); - } - public List lateralView() { - return getRuleContexts(LateralViewContext.class); - } - public LateralViewContext lateralView(int i) { - return getRuleContext(LateralViewContext.class,i); - } - public PivotClauseContext pivotClause() { - return getRuleContext(PivotClauseContext.class,0); - } - public FromClauseContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_fromClause; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterFromClause(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitFromClause(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitFromClause(this); - else return visitor.visitChildren(this); - } - } - - public final FromClauseContext fromClause() throws RecognitionException { - FromClauseContext _localctx = new FromClauseContext(_ctx, getState()); - enterRule(_localctx, 72, RULE_fromClause); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(712); - match(FROM); - setState(713); - relation(); - setState(718); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,88,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(714); - match(T__2); - setState(715); - relation(); - } - } - } - setState(720); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,88,_ctx); - } - setState(724); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,89,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(721); - lateralView(); - } - } - } - setState(726); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,89,_ctx); - } - setState(728); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,90,_ctx) ) { - case 1: - { - setState(727); - pivotClause(); - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class AggregationClauseContext extends ParserRuleContext { - public GroupByClauseContext groupByClause; - public List groupingExpressionsWithGroupingAnalytics = new ArrayList(); - public ExpressionContext expression; - public List groupingExpressions = new ArrayList(); - public Token kind; - public TerminalNode GROUP() { return getToken(MixedFormatSqlExtendParser.GROUP, 0); } - public TerminalNode BY() { return getToken(MixedFormatSqlExtendParser.BY, 0); } - public List groupByClause() { - return getRuleContexts(GroupByClauseContext.class); - } - public GroupByClauseContext groupByClause(int i) { - return getRuleContext(GroupByClauseContext.class,i); - } - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public TerminalNode WITH() { return getToken(MixedFormatSqlExtendParser.WITH, 0); } - public TerminalNode SETS() { return getToken(MixedFormatSqlExtendParser.SETS, 0); } - public List groupingSet() { - return getRuleContexts(GroupingSetContext.class); - } - public GroupingSetContext groupingSet(int i) { - return getRuleContext(GroupingSetContext.class,i); - } - public TerminalNode ROLLUP() { return getToken(MixedFormatSqlExtendParser.ROLLUP, 0); } - public TerminalNode CUBE() { return getToken(MixedFormatSqlExtendParser.CUBE, 0); } - public TerminalNode GROUPING() { return getToken(MixedFormatSqlExtendParser.GROUPING, 0); } - public AggregationClauseContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_aggregationClause; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterAggregationClause(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitAggregationClause(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitAggregationClause(this); - else return visitor.visitChildren(this); - } - } - - public final AggregationClauseContext aggregationClause() throws RecognitionException { - AggregationClauseContext _localctx = new AggregationClauseContext(_ctx, getState()); - enterRule(_localctx, 74, RULE_aggregationClause); - int _la; - try { - int _alt; - setState(769); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,95,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(730); - match(GROUP); - setState(731); - match(BY); - setState(732); - ((AggregationClauseContext)_localctx).groupByClause = groupByClause(); - ((AggregationClauseContext)_localctx).groupingExpressionsWithGroupingAnalytics.add(((AggregationClauseContext)_localctx).groupByClause); - setState(737); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,91,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(733); - match(T__2); - setState(734); - ((AggregationClauseContext)_localctx).groupByClause = groupByClause(); - ((AggregationClauseContext)_localctx).groupingExpressionsWithGroupingAnalytics.add(((AggregationClauseContext)_localctx).groupByClause); - } - } - } - setState(739); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,91,_ctx); - } - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(740); - match(GROUP); - setState(741); - match(BY); - setState(742); - ((AggregationClauseContext)_localctx).expression = expression(); - ((AggregationClauseContext)_localctx).groupingExpressions.add(((AggregationClauseContext)_localctx).expression); - setState(747); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,92,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(743); - match(T__2); - setState(744); - ((AggregationClauseContext)_localctx).expression = expression(); - ((AggregationClauseContext)_localctx).groupingExpressions.add(((AggregationClauseContext)_localctx).expression); - } - } - } - setState(749); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,92,_ctx); - } - setState(767); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,94,_ctx) ) { - case 1: - { - setState(750); - match(WITH); - setState(751); - ((AggregationClauseContext)_localctx).kind = match(ROLLUP); - } - break; - case 2: - { - setState(752); - match(WITH); - setState(753); - ((AggregationClauseContext)_localctx).kind = match(CUBE); - } - break; - case 3: - { - setState(754); - ((AggregationClauseContext)_localctx).kind = match(GROUPING); - setState(755); - match(SETS); - setState(756); - match(T__1); - setState(757); - groupingSet(); - setState(762); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(758); - match(T__2); - setState(759); - groupingSet(); - } - } - setState(764); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(765); - match(T__3); - } - break; - } - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class GroupByClauseContext extends ParserRuleContext { - public GroupingAnalyticsContext groupingAnalytics() { - return getRuleContext(GroupingAnalyticsContext.class,0); - } - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public GroupByClauseContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_groupByClause; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterGroupByClause(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitGroupByClause(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitGroupByClause(this); - else return visitor.visitChildren(this); - } - } - - public final GroupByClauseContext groupByClause() throws RecognitionException { - GroupByClauseContext _localctx = new GroupByClauseContext(_ctx, getState()); - enterRule(_localctx, 76, RULE_groupByClause); - try { - setState(773); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,96,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(771); - groupingAnalytics(); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(772); - expression(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class GroupingAnalyticsContext extends ParserRuleContext { - public List groupingSet() { - return getRuleContexts(GroupingSetContext.class); - } - public GroupingSetContext groupingSet(int i) { - return getRuleContext(GroupingSetContext.class,i); - } - public TerminalNode ROLLUP() { return getToken(MixedFormatSqlExtendParser.ROLLUP, 0); } - public TerminalNode CUBE() { return getToken(MixedFormatSqlExtendParser.CUBE, 0); } - public TerminalNode GROUPING() { return getToken(MixedFormatSqlExtendParser.GROUPING, 0); } - public TerminalNode SETS() { return getToken(MixedFormatSqlExtendParser.SETS, 0); } - public List groupingElement() { - return getRuleContexts(GroupingElementContext.class); - } - public GroupingElementContext groupingElement(int i) { - return getRuleContext(GroupingElementContext.class,i); - } - public GroupingAnalyticsContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_groupingAnalytics; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterGroupingAnalytics(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitGroupingAnalytics(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitGroupingAnalytics(this); - else return visitor.visitChildren(this); - } - } - - public final GroupingAnalyticsContext groupingAnalytics() throws RecognitionException { - GroupingAnalyticsContext _localctx = new GroupingAnalyticsContext(_ctx, getState()); - enterRule(_localctx, 78, RULE_groupingAnalytics); - int _la; - try { - setState(800); - _errHandler.sync(this); - switch (_input.LA(1)) { - case CUBE: - case ROLLUP: - enterOuterAlt(_localctx, 1); - { - setState(775); - _la = _input.LA(1); - if ( !(_la==CUBE || _la==ROLLUP) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(776); - match(T__1); - setState(777); - groupingSet(); - setState(782); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(778); - match(T__2); - setState(779); - groupingSet(); - } - } - setState(784); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(785); - match(T__3); - } - break; - case GROUPING: - enterOuterAlt(_localctx, 2); - { - setState(787); - match(GROUPING); - setState(788); - match(SETS); - setState(789); - match(T__1); - setState(790); - groupingElement(); - setState(795); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(791); - match(T__2); - setState(792); - groupingElement(); - } - } - setState(797); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(798); - match(T__3); - } - break; - default: - throw new NoViableAltException(this); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class GroupingElementContext extends ParserRuleContext { - public GroupingAnalyticsContext groupingAnalytics() { - return getRuleContext(GroupingAnalyticsContext.class,0); - } - public GroupingSetContext groupingSet() { - return getRuleContext(GroupingSetContext.class,0); - } - public GroupingElementContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_groupingElement; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterGroupingElement(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitGroupingElement(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitGroupingElement(this); - else return visitor.visitChildren(this); - } - } - - public final GroupingElementContext groupingElement() throws RecognitionException { - GroupingElementContext _localctx = new GroupingElementContext(_ctx, getState()); - enterRule(_localctx, 80, RULE_groupingElement); - try { - setState(804); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,100,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(802); - groupingAnalytics(); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(803); - groupingSet(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class GroupingSetContext extends ParserRuleContext { - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public GroupingSetContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_groupingSet; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterGroupingSet(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitGroupingSet(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitGroupingSet(this); - else return visitor.visitChildren(this); - } - } - - public final GroupingSetContext groupingSet() throws RecognitionException { - GroupingSetContext _localctx = new GroupingSetContext(_ctx, getState()); - enterRule(_localctx, 82, RULE_groupingSet); - int _la; - try { - setState(819); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,103,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(806); - match(T__1); - setState(815); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,102,_ctx) ) { - case 1: - { - setState(807); - expression(); - setState(812); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(808); - match(T__2); - setState(809); - expression(); - } - } - setState(814); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - break; - } - setState(817); - match(T__3); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(818); - expression(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class PivotClauseContext extends ParserRuleContext { - public NamedExpressionSeqContext aggregates; - public PivotValueContext pivotValue; - public List pivotValues = new ArrayList(); - public TerminalNode PIVOT() { return getToken(MixedFormatSqlExtendParser.PIVOT, 0); } - public TerminalNode FOR() { return getToken(MixedFormatSqlExtendParser.FOR, 0); } - public PivotColumnContext pivotColumn() { - return getRuleContext(PivotColumnContext.class,0); - } - public TerminalNode IN() { return getToken(MixedFormatSqlExtendParser.IN, 0); } - public NamedExpressionSeqContext namedExpressionSeq() { - return getRuleContext(NamedExpressionSeqContext.class,0); - } - public List pivotValue() { - return getRuleContexts(PivotValueContext.class); - } - public PivotValueContext pivotValue(int i) { - return getRuleContext(PivotValueContext.class,i); - } - public PivotClauseContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_pivotClause; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPivotClause(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPivotClause(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPivotClause(this); - else return visitor.visitChildren(this); - } - } - - public final PivotClauseContext pivotClause() throws RecognitionException { - PivotClauseContext _localctx = new PivotClauseContext(_ctx, getState()); - enterRule(_localctx, 84, RULE_pivotClause); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(821); - match(PIVOT); - setState(822); - match(T__1); - setState(823); - ((PivotClauseContext)_localctx).aggregates = namedExpressionSeq(); - setState(824); - match(FOR); - setState(825); - pivotColumn(); - setState(826); - match(IN); - setState(827); - match(T__1); - setState(828); - ((PivotClauseContext)_localctx).pivotValue = pivotValue(); - ((PivotClauseContext)_localctx).pivotValues.add(((PivotClauseContext)_localctx).pivotValue); - setState(833); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(829); - match(T__2); - setState(830); - ((PivotClauseContext)_localctx).pivotValue = pivotValue(); - ((PivotClauseContext)_localctx).pivotValues.add(((PivotClauseContext)_localctx).pivotValue); - } - } - setState(835); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(836); - match(T__3); - setState(837); - match(T__3); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class PivotColumnContext extends ParserRuleContext { - public IdentifierContext identifier; - public List identifiers = new ArrayList(); - public List identifier() { - return getRuleContexts(IdentifierContext.class); - } - public IdentifierContext identifier(int i) { - return getRuleContext(IdentifierContext.class,i); - } - public PivotColumnContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_pivotColumn; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPivotColumn(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPivotColumn(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPivotColumn(this); - else return visitor.visitChildren(this); - } - } - - public final PivotColumnContext pivotColumn() throws RecognitionException { - PivotColumnContext _localctx = new PivotColumnContext(_ctx, getState()); - enterRule(_localctx, 86, RULE_pivotColumn); - int _la; - try { - setState(851); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,106,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(839); - ((PivotColumnContext)_localctx).identifier = identifier(); - ((PivotColumnContext)_localctx).identifiers.add(((PivotColumnContext)_localctx).identifier); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(840); - match(T__1); - setState(841); - ((PivotColumnContext)_localctx).identifier = identifier(); - ((PivotColumnContext)_localctx).identifiers.add(((PivotColumnContext)_localctx).identifier); - setState(846); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(842); - match(T__2); - setState(843); - ((PivotColumnContext)_localctx).identifier = identifier(); - ((PivotColumnContext)_localctx).identifiers.add(((PivotColumnContext)_localctx).identifier); - } - } - setState(848); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(849); - match(T__3); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class PivotValueContext extends ParserRuleContext { - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public PivotValueContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_pivotValue; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPivotValue(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPivotValue(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPivotValue(this); - else return visitor.visitChildren(this); - } - } - - public final PivotValueContext pivotValue() throws RecognitionException { - PivotValueContext _localctx = new PivotValueContext(_ctx, getState()); - enterRule(_localctx, 88, RULE_pivotValue); - try { - enterOuterAlt(_localctx, 1); - { - setState(853); - expression(); - setState(858); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,108,_ctx) ) { - case 1: - { - setState(855); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,107,_ctx) ) { - case 1: - { - setState(854); - match(AS); - } - break; - } - setState(857); - identifier(); - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class LateralViewContext extends ParserRuleContext { - public IdentifierContext tblName; - public IdentifierContext identifier; - public List colName = new ArrayList(); - public TerminalNode LATERAL() { return getToken(MixedFormatSqlExtendParser.LATERAL, 0); } - public TerminalNode VIEW() { return getToken(MixedFormatSqlExtendParser.VIEW, 0); } - public QualifiedNameContext qualifiedName() { - return getRuleContext(QualifiedNameContext.class,0); - } - public List identifier() { - return getRuleContexts(IdentifierContext.class); - } - public IdentifierContext identifier(int i) { - return getRuleContext(IdentifierContext.class,i); - } - public TerminalNode OUTER() { return getToken(MixedFormatSqlExtendParser.OUTER, 0); } - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public LateralViewContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_lateralView; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterLateralView(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitLateralView(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitLateralView(this); - else return visitor.visitChildren(this); - } - } - - public final LateralViewContext lateralView() throws RecognitionException { - LateralViewContext _localctx = new LateralViewContext(_ctx, getState()); - enterRule(_localctx, 90, RULE_lateralView); - int _la; - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(860); - match(LATERAL); - setState(861); - match(VIEW); - setState(863); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,109,_ctx) ) { - case 1: - { - setState(862); - match(OUTER); - } - break; - } - setState(865); - qualifiedName(); - setState(866); - match(T__1); - setState(875); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,111,_ctx) ) { - case 1: - { - setState(867); - expression(); - setState(872); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(868); - match(T__2); - setState(869); - expression(); - } - } - setState(874); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - break; - } - setState(877); - match(T__3); - setState(878); - ((LateralViewContext)_localctx).tblName = identifier(); - setState(890); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,114,_ctx) ) { - case 1: - { - setState(880); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,112,_ctx) ) { - case 1: - { - setState(879); - match(AS); - } - break; - } - setState(882); - ((LateralViewContext)_localctx).identifier = identifier(); - ((LateralViewContext)_localctx).colName.add(((LateralViewContext)_localctx).identifier); - setState(887); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,113,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(883); - match(T__2); - setState(884); - ((LateralViewContext)_localctx).identifier = identifier(); - ((LateralViewContext)_localctx).colName.add(((LateralViewContext)_localctx).identifier); - } - } - } - setState(889); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,113,_ctx); - } - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class SetQuantifierContext extends ParserRuleContext { - public TerminalNode DISTINCT() { return getToken(MixedFormatSqlExtendParser.DISTINCT, 0); } - public TerminalNode ALL() { return getToken(MixedFormatSqlExtendParser.ALL, 0); } - public SetQuantifierContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_setQuantifier; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSetQuantifier(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSetQuantifier(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSetQuantifier(this); - else return visitor.visitChildren(this); - } - } - - public final SetQuantifierContext setQuantifier() throws RecognitionException { - SetQuantifierContext _localctx = new SetQuantifierContext(_ctx, getState()); - enterRule(_localctx, 92, RULE_setQuantifier); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(892); - _la = _input.LA(1); - if ( !(_la==ALL || _la==DISTINCT) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class RelationContext extends ParserRuleContext { - public RelationPrimaryContext relationPrimary() { - return getRuleContext(RelationPrimaryContext.class,0); - } - public TerminalNode LATERAL() { return getToken(MixedFormatSqlExtendParser.LATERAL, 0); } - public List joinRelation() { - return getRuleContexts(JoinRelationContext.class); - } - public JoinRelationContext joinRelation(int i) { - return getRuleContext(JoinRelationContext.class,i); - } - public RelationContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_relation; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterRelation(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitRelation(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitRelation(this); - else return visitor.visitChildren(this); - } - } - - public final RelationContext relation() throws RecognitionException { - RelationContext _localctx = new RelationContext(_ctx, getState()); - enterRule(_localctx, 94, RULE_relation); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(895); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,115,_ctx) ) { - case 1: - { - setState(894); - match(LATERAL); - } - break; - } - setState(897); - relationPrimary(); - setState(901); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,116,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(898); - joinRelation(); - } - } - } - setState(903); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,116,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class JoinRelationContext extends ParserRuleContext { - public RelationPrimaryContext right; - public TerminalNode JOIN() { return getToken(MixedFormatSqlExtendParser.JOIN, 0); } - public RelationPrimaryContext relationPrimary() { - return getRuleContext(RelationPrimaryContext.class,0); - } - public JoinTypeContext joinType() { - return getRuleContext(JoinTypeContext.class,0); - } - public TerminalNode LATERAL() { return getToken(MixedFormatSqlExtendParser.LATERAL, 0); } - public JoinCriteriaContext joinCriteria() { - return getRuleContext(JoinCriteriaContext.class,0); - } - public TerminalNode NATURAL() { return getToken(MixedFormatSqlExtendParser.NATURAL, 0); } - public JoinRelationContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_joinRelation; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterJoinRelation(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitJoinRelation(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitJoinRelation(this); - else return visitor.visitChildren(this); - } - } - - public final JoinRelationContext joinRelation() throws RecognitionException { - JoinRelationContext _localctx = new JoinRelationContext(_ctx, getState()); - enterRule(_localctx, 96, RULE_joinRelation); - try { - setState(921); - _errHandler.sync(this); - switch (_input.LA(1)) { - case ANTI: - case CROSS: - case FULL: - case INNER: - case JOIN: - case LEFT: - case RIGHT: - case SEMI: - enterOuterAlt(_localctx, 1); - { - { - setState(904); - joinType(); - } - setState(905); - match(JOIN); - setState(907); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,117,_ctx) ) { - case 1: - { - setState(906); - match(LATERAL); - } - break; - } - setState(909); - ((JoinRelationContext)_localctx).right = relationPrimary(); - setState(911); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,118,_ctx) ) { - case 1: - { - setState(910); - joinCriteria(); - } - break; - } - } - break; - case NATURAL: - enterOuterAlt(_localctx, 2); - { - setState(913); - match(NATURAL); - setState(914); - joinType(); - setState(915); - match(JOIN); - setState(917); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,119,_ctx) ) { - case 1: - { - setState(916); - match(LATERAL); - } - break; - } - setState(919); - ((JoinRelationContext)_localctx).right = relationPrimary(); - } - break; - default: - throw new NoViableAltException(this); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class JoinTypeContext extends ParserRuleContext { - public TerminalNode INNER() { return getToken(MixedFormatSqlExtendParser.INNER, 0); } - public TerminalNode CROSS() { return getToken(MixedFormatSqlExtendParser.CROSS, 0); } - public TerminalNode LEFT() { return getToken(MixedFormatSqlExtendParser.LEFT, 0); } - public TerminalNode OUTER() { return getToken(MixedFormatSqlExtendParser.OUTER, 0); } - public TerminalNode SEMI() { return getToken(MixedFormatSqlExtendParser.SEMI, 0); } - public TerminalNode RIGHT() { return getToken(MixedFormatSqlExtendParser.RIGHT, 0); } - public TerminalNode FULL() { return getToken(MixedFormatSqlExtendParser.FULL, 0); } - public TerminalNode ANTI() { return getToken(MixedFormatSqlExtendParser.ANTI, 0); } - public JoinTypeContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_joinType; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterJoinType(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitJoinType(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitJoinType(this); - else return visitor.visitChildren(this); - } - } - - public final JoinTypeContext joinType() throws RecognitionException { - JoinTypeContext _localctx = new JoinTypeContext(_ctx, getState()); - enterRule(_localctx, 98, RULE_joinType); - int _la; - try { - setState(947); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,127,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(924); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==INNER) { - { - setState(923); - match(INNER); - } - } - - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(926); - match(CROSS); - } - break; - case 3: - enterOuterAlt(_localctx, 3); - { - setState(927); - match(LEFT); - setState(929); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==OUTER) { - { - setState(928); - match(OUTER); - } - } - - } - break; - case 4: - enterOuterAlt(_localctx, 4); - { - setState(932); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==LEFT) { - { - setState(931); - match(LEFT); - } - } - - setState(934); - match(SEMI); - } - break; - case 5: - enterOuterAlt(_localctx, 5); - { - setState(935); - match(RIGHT); - setState(937); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==OUTER) { - { - setState(936); - match(OUTER); - } - } - - } - break; - case 6: - enterOuterAlt(_localctx, 6); - { - setState(939); - match(FULL); - setState(941); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==OUTER) { - { - setState(940); - match(OUTER); - } - } - - } - break; - case 7: - enterOuterAlt(_localctx, 7); - { - setState(944); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==LEFT) { - { - setState(943); - match(LEFT); - } - } - - setState(946); - match(ANTI); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class JoinCriteriaContext extends ParserRuleContext { - public TerminalNode ON() { return getToken(MixedFormatSqlExtendParser.ON, 0); } - public BooleanExpressionContext booleanExpression() { - return getRuleContext(BooleanExpressionContext.class,0); - } - public TerminalNode USING() { return getToken(MixedFormatSqlExtendParser.USING, 0); } - public IdentifierListContext identifierList() { - return getRuleContext(IdentifierListContext.class,0); - } - public JoinCriteriaContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_joinCriteria; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterJoinCriteria(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitJoinCriteria(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitJoinCriteria(this); - else return visitor.visitChildren(this); - } - } - - public final JoinCriteriaContext joinCriteria() throws RecognitionException { - JoinCriteriaContext _localctx = new JoinCriteriaContext(_ctx, getState()); - enterRule(_localctx, 100, RULE_joinCriteria); - try { - setState(953); - _errHandler.sync(this); - switch (_input.LA(1)) { - case ON: - enterOuterAlt(_localctx, 1); - { - setState(949); - match(ON); - setState(950); - booleanExpression(0); - } - break; - case USING: - enterOuterAlt(_localctx, 2); - { - setState(951); - match(USING); - setState(952); - identifierList(); - } - break; - default: - throw new NoViableAltException(this); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class SampleContext extends ParserRuleContext { - public TerminalNode TABLESAMPLE() { return getToken(MixedFormatSqlExtendParser.TABLESAMPLE, 0); } - public SampleMethodContext sampleMethod() { - return getRuleContext(SampleMethodContext.class,0); - } - public SampleContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_sample; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSample(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSample(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSample(this); - else return visitor.visitChildren(this); - } - } - - public final SampleContext sample() throws RecognitionException { - SampleContext _localctx = new SampleContext(_ctx, getState()); - enterRule(_localctx, 102, RULE_sample); - try { - enterOuterAlt(_localctx, 1); - { - setState(955); - match(TABLESAMPLE); - setState(956); - match(T__1); - setState(958); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,129,_ctx) ) { - case 1: - { - setState(957); - sampleMethod(); - } - break; - } - setState(960); - match(T__3); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class SampleMethodContext extends ParserRuleContext { - public SampleMethodContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_sampleMethod; } - - public SampleMethodContext() { } - public void copyFrom(SampleMethodContext ctx) { - super.copyFrom(ctx); - } - } - public static class SampleByRowsContext extends SampleMethodContext { - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public TerminalNode ROWS() { return getToken(MixedFormatSqlExtendParser.ROWS, 0); } - public SampleByRowsContext(SampleMethodContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSampleByRows(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSampleByRows(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSampleByRows(this); - else return visitor.visitChildren(this); - } - } - public static class SampleByPercentileContext extends SampleMethodContext { - public Token negativeSign; - public Token percentage; - public TerminalNode PERCENTLIT() { return getToken(MixedFormatSqlExtendParser.PERCENTLIT, 0); } - public TerminalNode INTEGER_VALUE() { return getToken(MixedFormatSqlExtendParser.INTEGER_VALUE, 0); } - public TerminalNode DECIMAL_VALUE() { return getToken(MixedFormatSqlExtendParser.DECIMAL_VALUE, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public SampleByPercentileContext(SampleMethodContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSampleByPercentile(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSampleByPercentile(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSampleByPercentile(this); - else return visitor.visitChildren(this); - } - } - public static class SampleByBucketContext extends SampleMethodContext { - public Token sampleType; - public Token numerator; - public Token denominator; - public TerminalNode OUT() { return getToken(MixedFormatSqlExtendParser.OUT, 0); } - public TerminalNode OF() { return getToken(MixedFormatSqlExtendParser.OF, 0); } - public TerminalNode BUCKET() { return getToken(MixedFormatSqlExtendParser.BUCKET, 0); } - public List INTEGER_VALUE() { return getTokens(MixedFormatSqlExtendParser.INTEGER_VALUE); } - public TerminalNode INTEGER_VALUE(int i) { - return getToken(MixedFormatSqlExtendParser.INTEGER_VALUE, i); - } - public TerminalNode ON() { return getToken(MixedFormatSqlExtendParser.ON, 0); } - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public QualifiedNameContext qualifiedName() { - return getRuleContext(QualifiedNameContext.class,0); - } - public SampleByBucketContext(SampleMethodContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSampleByBucket(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSampleByBucket(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSampleByBucket(this); - else return visitor.visitChildren(this); - } - } - public static class SampleByBytesContext extends SampleMethodContext { - public ExpressionContext bytes; - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public SampleByBytesContext(SampleMethodContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSampleByBytes(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSampleByBytes(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSampleByBytes(this); - else return visitor.visitChildren(this); - } - } - - public final SampleMethodContext sampleMethod() throws RecognitionException { - SampleMethodContext _localctx = new SampleMethodContext(_ctx, getState()); - enterRule(_localctx, 104, RULE_sampleMethod); - int _la; - try { - setState(986); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,133,_ctx) ) { - case 1: - _localctx = new SampleByPercentileContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(963); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(962); - ((SampleByPercentileContext)_localctx).negativeSign = match(MINUS); - } - } - - setState(965); - ((SampleByPercentileContext)_localctx).percentage = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==INTEGER_VALUE || _la==DECIMAL_VALUE) ) { - ((SampleByPercentileContext)_localctx).percentage = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(966); - match(PERCENTLIT); - } - break; - case 2: - _localctx = new SampleByRowsContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(967); - expression(); - setState(968); - match(ROWS); - } - break; - case 3: - _localctx = new SampleByBucketContext(_localctx); - enterOuterAlt(_localctx, 3); - { - setState(970); - ((SampleByBucketContext)_localctx).sampleType = match(BUCKET); - setState(971); - ((SampleByBucketContext)_localctx).numerator = match(INTEGER_VALUE); - setState(972); - match(OUT); - setState(973); - match(OF); - setState(974); - ((SampleByBucketContext)_localctx).denominator = match(INTEGER_VALUE); - setState(983); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==ON) { - { - setState(975); - match(ON); - setState(981); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,131,_ctx) ) { - case 1: - { - setState(976); - identifier(); - } - break; - case 2: - { - setState(977); - qualifiedName(); - setState(978); - match(T__1); - setState(979); - match(T__3); - } - break; - } - } - } - - } - break; - case 4: - _localctx = new SampleByBytesContext(_localctx); - enterOuterAlt(_localctx, 4); - { - setState(985); - ((SampleByBytesContext)_localctx).bytes = expression(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class IdentifierListContext extends ParserRuleContext { - public IdentifierSeqContext identifierSeq() { - return getRuleContext(IdentifierSeqContext.class,0); - } - public IdentifierListContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_identifierList; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterIdentifierList(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitIdentifierList(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitIdentifierList(this); - else return visitor.visitChildren(this); - } - } - - public final IdentifierListContext identifierList() throws RecognitionException { - IdentifierListContext _localctx = new IdentifierListContext(_ctx, getState()); - enterRule(_localctx, 106, RULE_identifierList); - try { - enterOuterAlt(_localctx, 1); - { - setState(988); - match(T__1); - setState(989); - identifierSeq(); - setState(990); - match(T__3); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class IdentifierSeqContext extends ParserRuleContext { - public ErrorCapturingIdentifierContext errorCapturingIdentifier; - public List ident = new ArrayList(); - public List errorCapturingIdentifier() { - return getRuleContexts(ErrorCapturingIdentifierContext.class); - } - public ErrorCapturingIdentifierContext errorCapturingIdentifier(int i) { - return getRuleContext(ErrorCapturingIdentifierContext.class,i); - } - public IdentifierSeqContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_identifierSeq; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterIdentifierSeq(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitIdentifierSeq(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitIdentifierSeq(this); - else return visitor.visitChildren(this); - } - } - - public final IdentifierSeqContext identifierSeq() throws RecognitionException { - IdentifierSeqContext _localctx = new IdentifierSeqContext(_ctx, getState()); - enterRule(_localctx, 108, RULE_identifierSeq); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(992); - ((IdentifierSeqContext)_localctx).errorCapturingIdentifier = errorCapturingIdentifier(); - ((IdentifierSeqContext)_localctx).ident.add(((IdentifierSeqContext)_localctx).errorCapturingIdentifier); - setState(997); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,134,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(993); - match(T__2); - setState(994); - ((IdentifierSeqContext)_localctx).errorCapturingIdentifier = errorCapturingIdentifier(); - ((IdentifierSeqContext)_localctx).ident.add(((IdentifierSeqContext)_localctx).errorCapturingIdentifier); - } - } - } - setState(999); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,134,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class OrderedIdentifierListContext extends ParserRuleContext { - public List orderedIdentifier() { - return getRuleContexts(OrderedIdentifierContext.class); - } - public OrderedIdentifierContext orderedIdentifier(int i) { - return getRuleContext(OrderedIdentifierContext.class,i); - } - public OrderedIdentifierListContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_orderedIdentifierList; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterOrderedIdentifierList(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitOrderedIdentifierList(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitOrderedIdentifierList(this); - else return visitor.visitChildren(this); - } - } - - public final OrderedIdentifierListContext orderedIdentifierList() throws RecognitionException { - OrderedIdentifierListContext _localctx = new OrderedIdentifierListContext(_ctx, getState()); - enterRule(_localctx, 110, RULE_orderedIdentifierList); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1000); - match(T__1); - setState(1001); - orderedIdentifier(); - setState(1006); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1002); - match(T__2); - setState(1003); - orderedIdentifier(); - } - } - setState(1008); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(1009); - match(T__3); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class OrderedIdentifierContext extends ParserRuleContext { - public ErrorCapturingIdentifierContext ident; - public Token ordering; - public ErrorCapturingIdentifierContext errorCapturingIdentifier() { - return getRuleContext(ErrorCapturingIdentifierContext.class,0); - } - public TerminalNode ASC() { return getToken(MixedFormatSqlExtendParser.ASC, 0); } - public TerminalNode DESC() { return getToken(MixedFormatSqlExtendParser.DESC, 0); } - public OrderedIdentifierContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_orderedIdentifier; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterOrderedIdentifier(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitOrderedIdentifier(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitOrderedIdentifier(this); - else return visitor.visitChildren(this); - } - } - - public final OrderedIdentifierContext orderedIdentifier() throws RecognitionException { - OrderedIdentifierContext _localctx = new OrderedIdentifierContext(_ctx, getState()); - enterRule(_localctx, 112, RULE_orderedIdentifier); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1011); - ((OrderedIdentifierContext)_localctx).ident = errorCapturingIdentifier(); - setState(1013); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==ASC || _la==DESC) { - { - setState(1012); - ((OrderedIdentifierContext)_localctx).ordering = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==ASC || _la==DESC) ) { - ((OrderedIdentifierContext)_localctx).ordering = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - } - - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class RelationPrimaryContext extends ParserRuleContext { - public RelationPrimaryContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_relationPrimary; } - - public RelationPrimaryContext() { } - public void copyFrom(RelationPrimaryContext ctx) { - super.copyFrom(ctx); - } - } - public static class TableValuedFunctionContext extends RelationPrimaryContext { - public FunctionTableContext functionTable() { - return getRuleContext(FunctionTableContext.class,0); - } - public TableValuedFunctionContext(RelationPrimaryContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTableValuedFunction(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTableValuedFunction(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTableValuedFunction(this); - else return visitor.visitChildren(this); - } - } - public static class InlineTableDefault2Context extends RelationPrimaryContext { - public InlineTableContext inlineTable() { - return getRuleContext(InlineTableContext.class,0); - } - public InlineTableDefault2Context(RelationPrimaryContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterInlineTableDefault2(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitInlineTableDefault2(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitInlineTableDefault2(this); - else return visitor.visitChildren(this); - } - } - public static class AliasedRelationContext extends RelationPrimaryContext { - public RelationContext relation() { - return getRuleContext(RelationContext.class,0); - } - public TableAliasContext tableAlias() { - return getRuleContext(TableAliasContext.class,0); - } - public SampleContext sample() { - return getRuleContext(SampleContext.class,0); - } - public AliasedRelationContext(RelationPrimaryContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterAliasedRelation(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitAliasedRelation(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitAliasedRelation(this); - else return visitor.visitChildren(this); - } - } - public static class AliasedQueryContext extends RelationPrimaryContext { - public QueryContext query() { - return getRuleContext(QueryContext.class,0); - } - public TableAliasContext tableAlias() { - return getRuleContext(TableAliasContext.class,0); - } - public SampleContext sample() { - return getRuleContext(SampleContext.class,0); - } - public AliasedQueryContext(RelationPrimaryContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterAliasedQuery(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitAliasedQuery(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitAliasedQuery(this); - else return visitor.visitChildren(this); - } - } - public static class TableNameContext extends RelationPrimaryContext { - public MultipartIdentifierContext multipartIdentifier() { - return getRuleContext(MultipartIdentifierContext.class,0); - } - public TableAliasContext tableAlias() { - return getRuleContext(TableAliasContext.class,0); - } - public SampleContext sample() { - return getRuleContext(SampleContext.class,0); - } - public TableNameContext(RelationPrimaryContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTableName(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTableName(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTableName(this); - else return visitor.visitChildren(this); - } - } - - public final RelationPrimaryContext relationPrimary() throws RecognitionException { - RelationPrimaryContext _localctx = new RelationPrimaryContext(_ctx, getState()); - enterRule(_localctx, 114, RULE_relationPrimary); - try { - setState(1039); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,140,_ctx) ) { - case 1: - _localctx = new TableNameContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(1015); - multipartIdentifier(); - setState(1017); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,137,_ctx) ) { - case 1: - { - setState(1016); - sample(); - } - break; - } - setState(1019); - tableAlias(); - } - break; - case 2: - _localctx = new AliasedQueryContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(1021); - match(T__1); - setState(1022); - query(); - setState(1023); - match(T__3); - setState(1025); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,138,_ctx) ) { - case 1: - { - setState(1024); - sample(); - } - break; - } - setState(1027); - tableAlias(); - } - break; - case 3: - _localctx = new AliasedRelationContext(_localctx); - enterOuterAlt(_localctx, 3); - { - setState(1029); - match(T__1); - setState(1030); - relation(); - setState(1031); - match(T__3); - setState(1033); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,139,_ctx) ) { - case 1: - { - setState(1032); - sample(); - } - break; - } - setState(1035); - tableAlias(); - } - break; - case 4: - _localctx = new InlineTableDefault2Context(_localctx); - enterOuterAlt(_localctx, 4); - { - setState(1037); - inlineTable(); - } - break; - case 5: - _localctx = new TableValuedFunctionContext(_localctx); - enterOuterAlt(_localctx, 5); - { - setState(1038); - functionTable(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class InlineTableContext extends ParserRuleContext { - public TerminalNode VALUES() { return getToken(MixedFormatSqlExtendParser.VALUES, 0); } - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public TableAliasContext tableAlias() { - return getRuleContext(TableAliasContext.class,0); - } - public InlineTableContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_inlineTable; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterInlineTable(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitInlineTable(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitInlineTable(this); - else return visitor.visitChildren(this); - } - } - - public final InlineTableContext inlineTable() throws RecognitionException { - InlineTableContext _localctx = new InlineTableContext(_ctx, getState()); - enterRule(_localctx, 116, RULE_inlineTable); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(1041); - match(VALUES); - setState(1042); - expression(); - setState(1047); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,141,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(1043); - match(T__2); - setState(1044); - expression(); - } - } - } - setState(1049); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,141,_ctx); - } - setState(1050); - tableAlias(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class FunctionTableContext extends ParserRuleContext { - public FunctionNameContext funcName; - public TableAliasContext tableAlias() { - return getRuleContext(TableAliasContext.class,0); - } - public FunctionNameContext functionName() { - return getRuleContext(FunctionNameContext.class,0); - } - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public FunctionTableContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_functionTable; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterFunctionTable(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitFunctionTable(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitFunctionTable(this); - else return visitor.visitChildren(this); - } - } - - public final FunctionTableContext functionTable() throws RecognitionException { - FunctionTableContext _localctx = new FunctionTableContext(_ctx, getState()); - enterRule(_localctx, 118, RULE_functionTable); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1052); - ((FunctionTableContext)_localctx).funcName = functionName(); - setState(1053); - match(T__1); - setState(1062); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,143,_ctx) ) { - case 1: - { - setState(1054); - expression(); - setState(1059); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1055); - match(T__2); - setState(1056); - expression(); - } - } - setState(1061); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - break; - } - setState(1064); - match(T__3); - setState(1065); - tableAlias(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class TableAliasContext extends ParserRuleContext { - public StrictIdentifierContext strictIdentifier() { - return getRuleContext(StrictIdentifierContext.class,0); - } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public IdentifierListContext identifierList() { - return getRuleContext(IdentifierListContext.class,0); - } - public TableAliasContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_tableAlias; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTableAlias(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTableAlias(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTableAlias(this); - else return visitor.visitChildren(this); - } - } - - public final TableAliasContext tableAlias() throws RecognitionException { - TableAliasContext _localctx = new TableAliasContext(_ctx, getState()); - enterRule(_localctx, 120, RULE_tableAlias); - try { - enterOuterAlt(_localctx, 1); - { - setState(1074); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,146,_ctx) ) { - case 1: - { - setState(1068); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,144,_ctx) ) { - case 1: - { - setState(1067); - match(AS); - } - break; - } - setState(1070); - strictIdentifier(); - setState(1072); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,145,_ctx) ) { - case 1: - { - setState(1071); - identifierList(); - } - break; - } - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class RowFormatContext extends ParserRuleContext { - public RowFormatContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_rowFormat; } - - public RowFormatContext() { } - public void copyFrom(RowFormatContext ctx) { - super.copyFrom(ctx); - } - } - public static class RowFormatSerdeContext extends RowFormatContext { - public Token name; - public TablePropertyListContext props; - public TerminalNode ROW() { return getToken(MixedFormatSqlExtendParser.ROW, 0); } - public TerminalNode FORMAT() { return getToken(MixedFormatSqlExtendParser.FORMAT, 0); } - public TerminalNode SERDE() { return getToken(MixedFormatSqlExtendParser.SERDE, 0); } - public TerminalNode STRING() { return getToken(MixedFormatSqlExtendParser.STRING, 0); } - public TerminalNode WITH() { return getToken(MixedFormatSqlExtendParser.WITH, 0); } - public TerminalNode SERDEPROPERTIES() { return getToken(MixedFormatSqlExtendParser.SERDEPROPERTIES, 0); } - public TablePropertyListContext tablePropertyList() { - return getRuleContext(TablePropertyListContext.class,0); - } - public RowFormatSerdeContext(RowFormatContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterRowFormatSerde(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitRowFormatSerde(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitRowFormatSerde(this); - else return visitor.visitChildren(this); - } - } - public static class RowFormatDelimitedContext extends RowFormatContext { - public Token fieldsTerminatedBy; - public Token escapedBy; - public Token collectionItemsTerminatedBy; - public Token keysTerminatedBy; - public Token linesSeparatedBy; - public Token nullDefinedAs; - public TerminalNode ROW() { return getToken(MixedFormatSqlExtendParser.ROW, 0); } - public TerminalNode FORMAT() { return getToken(MixedFormatSqlExtendParser.FORMAT, 0); } - public TerminalNode DELIMITED() { return getToken(MixedFormatSqlExtendParser.DELIMITED, 0); } - public TerminalNode FIELDS() { return getToken(MixedFormatSqlExtendParser.FIELDS, 0); } - public List TERMINATED() { return getTokens(MixedFormatSqlExtendParser.TERMINATED); } - public TerminalNode TERMINATED(int i) { - return getToken(MixedFormatSqlExtendParser.TERMINATED, i); - } - public List BY() { return getTokens(MixedFormatSqlExtendParser.BY); } - public TerminalNode BY(int i) { - return getToken(MixedFormatSqlExtendParser.BY, i); - } - public TerminalNode COLLECTION() { return getToken(MixedFormatSqlExtendParser.COLLECTION, 0); } - public TerminalNode ITEMS() { return getToken(MixedFormatSqlExtendParser.ITEMS, 0); } - public TerminalNode MAP() { return getToken(MixedFormatSqlExtendParser.MAP, 0); } - public TerminalNode KEYS() { return getToken(MixedFormatSqlExtendParser.KEYS, 0); } - public TerminalNode LINES() { return getToken(MixedFormatSqlExtendParser.LINES, 0); } - public TerminalNode NULL() { return getToken(MixedFormatSqlExtendParser.NULL, 0); } - public TerminalNode DEFINED() { return getToken(MixedFormatSqlExtendParser.DEFINED, 0); } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public List STRING() { return getTokens(MixedFormatSqlExtendParser.STRING); } - public TerminalNode STRING(int i) { - return getToken(MixedFormatSqlExtendParser.STRING, i); - } - public TerminalNode ESCAPED() { return getToken(MixedFormatSqlExtendParser.ESCAPED, 0); } - public RowFormatDelimitedContext(RowFormatContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterRowFormatDelimited(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitRowFormatDelimited(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitRowFormatDelimited(this); - else return visitor.visitChildren(this); - } - } - - public final RowFormatContext rowFormat() throws RecognitionException { - RowFormatContext _localctx = new RowFormatContext(_ctx, getState()); - enterRule(_localctx, 122, RULE_rowFormat); - try { - setState(1125); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,154,_ctx) ) { - case 1: - _localctx = new RowFormatSerdeContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(1076); - match(ROW); - setState(1077); - match(FORMAT); - setState(1078); - match(SERDE); - setState(1079); - ((RowFormatSerdeContext)_localctx).name = match(STRING); - setState(1083); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,147,_ctx) ) { - case 1: - { - setState(1080); - match(WITH); - setState(1081); - match(SERDEPROPERTIES); - setState(1082); - ((RowFormatSerdeContext)_localctx).props = tablePropertyList(); - } - break; - } - } - break; - case 2: - _localctx = new RowFormatDelimitedContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(1085); - match(ROW); - setState(1086); - match(FORMAT); - setState(1087); - match(DELIMITED); - setState(1097); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,149,_ctx) ) { - case 1: - { - setState(1088); - match(FIELDS); - setState(1089); - match(TERMINATED); - setState(1090); - match(BY); - setState(1091); - ((RowFormatDelimitedContext)_localctx).fieldsTerminatedBy = match(STRING); - setState(1095); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,148,_ctx) ) { - case 1: - { - setState(1092); - match(ESCAPED); - setState(1093); - match(BY); - setState(1094); - ((RowFormatDelimitedContext)_localctx).escapedBy = match(STRING); - } - break; - } - } - break; - } - setState(1104); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,150,_ctx) ) { - case 1: - { - setState(1099); - match(COLLECTION); - setState(1100); - match(ITEMS); - setState(1101); - match(TERMINATED); - setState(1102); - match(BY); - setState(1103); - ((RowFormatDelimitedContext)_localctx).collectionItemsTerminatedBy = match(STRING); - } - break; - } - setState(1111); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,151,_ctx) ) { - case 1: - { - setState(1106); - match(MAP); - setState(1107); - match(KEYS); - setState(1108); - match(TERMINATED); - setState(1109); - match(BY); - setState(1110); - ((RowFormatDelimitedContext)_localctx).keysTerminatedBy = match(STRING); - } - break; - } - setState(1117); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,152,_ctx) ) { - case 1: - { - setState(1113); - match(LINES); - setState(1114); - match(TERMINATED); - setState(1115); - match(BY); - setState(1116); - ((RowFormatDelimitedContext)_localctx).linesSeparatedBy = match(STRING); - } - break; - } - setState(1123); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,153,_ctx) ) { - case 1: - { - setState(1119); - match(NULL); - setState(1120); - match(DEFINED); - setState(1121); - match(AS); - setState(1122); - ((RowFormatDelimitedContext)_localctx).nullDefinedAs = match(STRING); - } - break; - } - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class MultipartIdentifierContext extends ParserRuleContext { - public ErrorCapturingIdentifierContext errorCapturingIdentifier; - public List parts = new ArrayList(); - public List errorCapturingIdentifier() { - return getRuleContexts(ErrorCapturingIdentifierContext.class); - } - public ErrorCapturingIdentifierContext errorCapturingIdentifier(int i) { - return getRuleContext(ErrorCapturingIdentifierContext.class,i); - } - public MultipartIdentifierContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_multipartIdentifier; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterMultipartIdentifier(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitMultipartIdentifier(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitMultipartIdentifier(this); - else return visitor.visitChildren(this); - } - } - - public final MultipartIdentifierContext multipartIdentifier() throws RecognitionException { - MultipartIdentifierContext _localctx = new MultipartIdentifierContext(_ctx, getState()); - enterRule(_localctx, 124, RULE_multipartIdentifier); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(1127); - ((MultipartIdentifierContext)_localctx).errorCapturingIdentifier = errorCapturingIdentifier(); - ((MultipartIdentifierContext)_localctx).parts.add(((MultipartIdentifierContext)_localctx).errorCapturingIdentifier); - setState(1132); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,155,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(1128); - match(T__4); - setState(1129); - ((MultipartIdentifierContext)_localctx).errorCapturingIdentifier = errorCapturingIdentifier(); - ((MultipartIdentifierContext)_localctx).parts.add(((MultipartIdentifierContext)_localctx).errorCapturingIdentifier); - } - } - } - setState(1134); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,155,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class NamedExpressionContext extends ParserRuleContext { - public ErrorCapturingIdentifierContext name; - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public IdentifierListContext identifierList() { - return getRuleContext(IdentifierListContext.class,0); - } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public ErrorCapturingIdentifierContext errorCapturingIdentifier() { - return getRuleContext(ErrorCapturingIdentifierContext.class,0); - } - public NamedExpressionContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_namedExpression; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterNamedExpression(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitNamedExpression(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitNamedExpression(this); - else return visitor.visitChildren(this); - } - } - - public final NamedExpressionContext namedExpression() throws RecognitionException { - NamedExpressionContext _localctx = new NamedExpressionContext(_ctx, getState()); - enterRule(_localctx, 126, RULE_namedExpression); - try { - enterOuterAlt(_localctx, 1); - { - setState(1135); - expression(); - setState(1143); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,158,_ctx) ) { - case 1: - { - setState(1137); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,156,_ctx) ) { - case 1: - { - setState(1136); - match(AS); - } - break; - } - setState(1141); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,157,_ctx) ) { - case 1: - { - setState(1139); - ((NamedExpressionContext)_localctx).name = errorCapturingIdentifier(); - } - break; - case 2: - { - setState(1140); - identifierList(); - } - break; - } - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class NamedExpressionSeqContext extends ParserRuleContext { - public List namedExpression() { - return getRuleContexts(NamedExpressionContext.class); - } - public NamedExpressionContext namedExpression(int i) { - return getRuleContext(NamedExpressionContext.class,i); - } - public NamedExpressionSeqContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_namedExpressionSeq; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterNamedExpressionSeq(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitNamedExpressionSeq(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitNamedExpressionSeq(this); - else return visitor.visitChildren(this); - } - } - - public final NamedExpressionSeqContext namedExpressionSeq() throws RecognitionException { - NamedExpressionSeqContext _localctx = new NamedExpressionSeqContext(_ctx, getState()); - enterRule(_localctx, 128, RULE_namedExpressionSeq); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(1145); - namedExpression(); - setState(1150); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,159,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(1146); - match(T__2); - setState(1147); - namedExpression(); - } - } - } - setState(1152); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,159,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class PartitionFieldListContext extends ParserRuleContext { - public PartitionFieldContext partitionField; - public List fields = new ArrayList(); - public List partitionField() { - return getRuleContexts(PartitionFieldContext.class); - } - public PartitionFieldContext partitionField(int i) { - return getRuleContext(PartitionFieldContext.class,i); - } - public PartitionFieldListContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_partitionFieldList; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPartitionFieldList(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPartitionFieldList(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPartitionFieldList(this); - else return visitor.visitChildren(this); - } - } - - public final PartitionFieldListContext partitionFieldList() throws RecognitionException { - PartitionFieldListContext _localctx = new PartitionFieldListContext(_ctx, getState()); - enterRule(_localctx, 130, RULE_partitionFieldList); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1153); - match(T__1); - setState(1154); - ((PartitionFieldListContext)_localctx).partitionField = partitionField(); - ((PartitionFieldListContext)_localctx).fields.add(((PartitionFieldListContext)_localctx).partitionField); - setState(1159); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1155); - match(T__2); - setState(1156); - ((PartitionFieldListContext)_localctx).partitionField = partitionField(); - ((PartitionFieldListContext)_localctx).fields.add(((PartitionFieldListContext)_localctx).partitionField); - } - } - setState(1161); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(1162); - match(T__3); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class PartitionFieldContext extends ParserRuleContext { - public PartitionFieldContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_partitionField; } - - public PartitionFieldContext() { } - public void copyFrom(PartitionFieldContext ctx) { - super.copyFrom(ctx); - } - } - public static class PartitionColumnContext extends PartitionFieldContext { - public ColTypeContext colType() { - return getRuleContext(ColTypeContext.class,0); - } - public PartitionColumnContext(PartitionFieldContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPartitionColumn(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPartitionColumn(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPartitionColumn(this); - else return visitor.visitChildren(this); - } - } - public static class PartitionTransformContext extends PartitionFieldContext { - public TransformContext transform() { - return getRuleContext(TransformContext.class,0); - } - public PartitionTransformContext(PartitionFieldContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPartitionTransform(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPartitionTransform(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPartitionTransform(this); - else return visitor.visitChildren(this); - } - } - - public final PartitionFieldContext partitionField() throws RecognitionException { - PartitionFieldContext _localctx = new PartitionFieldContext(_ctx, getState()); - enterRule(_localctx, 132, RULE_partitionField); - try { - setState(1166); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,161,_ctx) ) { - case 1: - _localctx = new PartitionTransformContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(1164); - transform(); - } - break; - case 2: - _localctx = new PartitionColumnContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(1165); - colType(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class TransformContext extends ParserRuleContext { - public TransformContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_transform; } - - public TransformContext() { } - public void copyFrom(TransformContext ctx) { - super.copyFrom(ctx); - } - } - public static class IdentityTransformContext extends TransformContext { - public QualifiedNameContext qualifiedName() { - return getRuleContext(QualifiedNameContext.class,0); - } - public IdentityTransformContext(TransformContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterIdentityTransform(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitIdentityTransform(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitIdentityTransform(this); - else return visitor.visitChildren(this); - } - } - public static class ApplyTransformContext extends TransformContext { - public IdentifierContext transformName; - public TransformArgumentContext transformArgument; - public List argument = new ArrayList(); - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public List transformArgument() { - return getRuleContexts(TransformArgumentContext.class); - } - public TransformArgumentContext transformArgument(int i) { - return getRuleContext(TransformArgumentContext.class,i); - } - public ApplyTransformContext(TransformContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterApplyTransform(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitApplyTransform(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitApplyTransform(this); - else return visitor.visitChildren(this); - } - } - - public final TransformContext transform() throws RecognitionException { - TransformContext _localctx = new TransformContext(_ctx, getState()); - enterRule(_localctx, 134, RULE_transform); - int _la; - try { - setState(1181); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,163,_ctx) ) { - case 1: - _localctx = new IdentityTransformContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(1168); - qualifiedName(); - } - break; - case 2: - _localctx = new ApplyTransformContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(1169); - ((ApplyTransformContext)_localctx).transformName = identifier(); - setState(1170); - match(T__1); - setState(1171); - ((ApplyTransformContext)_localctx).transformArgument = transformArgument(); - ((ApplyTransformContext)_localctx).argument.add(((ApplyTransformContext)_localctx).transformArgument); - setState(1176); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1172); - match(T__2); - setState(1173); - ((ApplyTransformContext)_localctx).transformArgument = transformArgument(); - ((ApplyTransformContext)_localctx).argument.add(((ApplyTransformContext)_localctx).transformArgument); - } - } - setState(1178); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(1179); - match(T__3); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class TransformArgumentContext extends ParserRuleContext { - public QualifiedNameContext qualifiedName() { - return getRuleContext(QualifiedNameContext.class,0); - } - public ConstantContext constant() { - return getRuleContext(ConstantContext.class,0); - } - public TransformArgumentContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_transformArgument; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTransformArgument(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTransformArgument(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTransformArgument(this); - else return visitor.visitChildren(this); - } - } - - public final TransformArgumentContext transformArgument() throws RecognitionException { - TransformArgumentContext _localctx = new TransformArgumentContext(_ctx, getState()); - enterRule(_localctx, 136, RULE_transformArgument); - try { - setState(1185); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,164,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(1183); - qualifiedName(); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(1184); - constant(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ExpressionContext extends ParserRuleContext { - public BooleanExpressionContext booleanExpression() { - return getRuleContext(BooleanExpressionContext.class,0); - } - public ExpressionContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_expression; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterExpression(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitExpression(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitExpression(this); - else return visitor.visitChildren(this); - } - } - - public final ExpressionContext expression() throws RecognitionException { - ExpressionContext _localctx = new ExpressionContext(_ctx, getState()); - enterRule(_localctx, 138, RULE_expression); - try { - enterOuterAlt(_localctx, 1); - { - setState(1187); - booleanExpression(0); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ExpressionSeqContext extends ParserRuleContext { - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public ExpressionSeqContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_expressionSeq; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterExpressionSeq(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitExpressionSeq(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitExpressionSeq(this); - else return visitor.visitChildren(this); - } - } - - public final ExpressionSeqContext expressionSeq() throws RecognitionException { - ExpressionSeqContext _localctx = new ExpressionSeqContext(_ctx, getState()); - enterRule(_localctx, 140, RULE_expressionSeq); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1189); - expression(); - setState(1194); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1190); - match(T__2); - setState(1191); - expression(); - } - } - setState(1196); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class BooleanExpressionContext extends ParserRuleContext { - public BooleanExpressionContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_booleanExpression; } - - public BooleanExpressionContext() { } - public void copyFrom(BooleanExpressionContext ctx) { - super.copyFrom(ctx); - } - } - public static class LogicalNotContext extends BooleanExpressionContext { - public TerminalNode NOT() { return getToken(MixedFormatSqlExtendParser.NOT, 0); } - public BooleanExpressionContext booleanExpression() { - return getRuleContext(BooleanExpressionContext.class,0); - } - public LogicalNotContext(BooleanExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterLogicalNot(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitLogicalNot(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitLogicalNot(this); - else return visitor.visitChildren(this); - } - } - public static class PredicatedContext extends BooleanExpressionContext { - public ValueExpressionContext valueExpression() { - return getRuleContext(ValueExpressionContext.class,0); - } - public PredicateContext predicate() { - return getRuleContext(PredicateContext.class,0); - } - public PredicatedContext(BooleanExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPredicated(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPredicated(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPredicated(this); - else return visitor.visitChildren(this); - } - } - public static class ExistsContext extends BooleanExpressionContext { - public TerminalNode EXISTS() { return getToken(MixedFormatSqlExtendParser.EXISTS, 0); } - public QueryContext query() { - return getRuleContext(QueryContext.class,0); - } - public ExistsContext(BooleanExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterExists(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitExists(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitExists(this); - else return visitor.visitChildren(this); - } - } - public static class LogicalBinaryContext extends BooleanExpressionContext { - public BooleanExpressionContext left; - public Token operator; - public BooleanExpressionContext right; - public List booleanExpression() { - return getRuleContexts(BooleanExpressionContext.class); - } - public BooleanExpressionContext booleanExpression(int i) { - return getRuleContext(BooleanExpressionContext.class,i); - } - public TerminalNode AND() { return getToken(MixedFormatSqlExtendParser.AND, 0); } - public TerminalNode OR() { return getToken(MixedFormatSqlExtendParser.OR, 0); } - public LogicalBinaryContext(BooleanExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterLogicalBinary(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitLogicalBinary(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitLogicalBinary(this); - else return visitor.visitChildren(this); - } - } - - public final BooleanExpressionContext booleanExpression() throws RecognitionException { - return booleanExpression(0); - } - - private BooleanExpressionContext booleanExpression(int _p) throws RecognitionException { - ParserRuleContext _parentctx = _ctx; - int _parentState = getState(); - BooleanExpressionContext _localctx = new BooleanExpressionContext(_ctx, _parentState); - BooleanExpressionContext _prevctx = _localctx; - int _startState = 142; - enterRecursionRule(_localctx, 142, RULE_booleanExpression, _p); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(1209); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,167,_ctx) ) { - case 1: - { - _localctx = new LogicalNotContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - - setState(1198); - match(NOT); - setState(1199); - booleanExpression(5); - } - break; - case 2: - { - _localctx = new ExistsContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1200); - match(EXISTS); - setState(1201); - match(T__1); - setState(1202); - query(); - setState(1203); - match(T__3); - } - break; - case 3: - { - _localctx = new PredicatedContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1205); - valueExpression(0); - setState(1207); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,166,_ctx) ) { - case 1: - { - setState(1206); - predicate(); - } - break; - } - } - break; - } - _ctx.stop = _input.LT(-1); - setState(1219); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,169,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - if ( _parseListeners!=null ) triggerExitRuleEvent(); - _prevctx = _localctx; - { - setState(1217); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,168,_ctx) ) { - case 1: - { - _localctx = new LogicalBinaryContext(new BooleanExpressionContext(_parentctx, _parentState)); - ((LogicalBinaryContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_booleanExpression); - setState(1211); - if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)"); - setState(1212); - ((LogicalBinaryContext)_localctx).operator = match(AND); - setState(1213); - ((LogicalBinaryContext)_localctx).right = booleanExpression(3); - } - break; - case 2: - { - _localctx = new LogicalBinaryContext(new BooleanExpressionContext(_parentctx, _parentState)); - ((LogicalBinaryContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_booleanExpression); - setState(1214); - if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)"); - setState(1215); - ((LogicalBinaryContext)_localctx).operator = match(OR); - setState(1216); - ((LogicalBinaryContext)_localctx).right = booleanExpression(2); - } - break; - } - } - } - setState(1221); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,169,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - unrollRecursionContexts(_parentctx); - } - return _localctx; - } - - public static class PredicateContext extends ParserRuleContext { - public Token kind; - public ValueExpressionContext lower; - public ValueExpressionContext upper; - public ValueExpressionContext pattern; - public Token quantifier; - public Token escapeChar; - public ValueExpressionContext right; - public TerminalNode AND() { return getToken(MixedFormatSqlExtendParser.AND, 0); } - public TerminalNode BETWEEN() { return getToken(MixedFormatSqlExtendParser.BETWEEN, 0); } - public List valueExpression() { - return getRuleContexts(ValueExpressionContext.class); - } - public ValueExpressionContext valueExpression(int i) { - return getRuleContext(ValueExpressionContext.class,i); - } - public TerminalNode NOT() { return getToken(MixedFormatSqlExtendParser.NOT, 0); } - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public TerminalNode IN() { return getToken(MixedFormatSqlExtendParser.IN, 0); } - public QueryContext query() { - return getRuleContext(QueryContext.class,0); - } - public TerminalNode RLIKE() { return getToken(MixedFormatSqlExtendParser.RLIKE, 0); } - public TerminalNode LIKE() { return getToken(MixedFormatSqlExtendParser.LIKE, 0); } - public TerminalNode ANY() { return getToken(MixedFormatSqlExtendParser.ANY, 0); } - public TerminalNode SOME() { return getToken(MixedFormatSqlExtendParser.SOME, 0); } - public TerminalNode ALL() { return getToken(MixedFormatSqlExtendParser.ALL, 0); } - public TerminalNode ESCAPE() { return getToken(MixedFormatSqlExtendParser.ESCAPE, 0); } - public TerminalNode STRING() { return getToken(MixedFormatSqlExtendParser.STRING, 0); } - public TerminalNode IS() { return getToken(MixedFormatSqlExtendParser.IS, 0); } - public TerminalNode NULL() { return getToken(MixedFormatSqlExtendParser.NULL, 0); } - public TerminalNode TRUE() { return getToken(MixedFormatSqlExtendParser.TRUE, 0); } - public TerminalNode FALSE() { return getToken(MixedFormatSqlExtendParser.FALSE, 0); } - public TerminalNode UNKNOWN() { return getToken(MixedFormatSqlExtendParser.UNKNOWN, 0); } - public TerminalNode FROM() { return getToken(MixedFormatSqlExtendParser.FROM, 0); } - public TerminalNode DISTINCT() { return getToken(MixedFormatSqlExtendParser.DISTINCT, 0); } - public PredicateContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_predicate; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPredicate(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPredicate(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPredicate(this); - else return visitor.visitChildren(this); - } - } - - public final PredicateContext predicate() throws RecognitionException { - PredicateContext _localctx = new PredicateContext(_ctx, getState()); - enterRule(_localctx, 144, RULE_predicate); - int _la; - try { - setState(1304); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,183,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(1223); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==NOT) { - { - setState(1222); - match(NOT); - } - } - - setState(1225); - ((PredicateContext)_localctx).kind = match(BETWEEN); - setState(1226); - ((PredicateContext)_localctx).lower = valueExpression(0); - setState(1227); - match(AND); - setState(1228); - ((PredicateContext)_localctx).upper = valueExpression(0); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(1231); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==NOT) { - { - setState(1230); - match(NOT); - } - } - - setState(1233); - ((PredicateContext)_localctx).kind = match(IN); - setState(1234); - match(T__1); - setState(1235); - expression(); - setState(1240); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1236); - match(T__2); - setState(1237); - expression(); - } - } - setState(1242); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(1243); - match(T__3); - } - break; - case 3: - enterOuterAlt(_localctx, 3); - { - setState(1246); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==NOT) { - { - setState(1245); - match(NOT); - } - } - - setState(1248); - ((PredicateContext)_localctx).kind = match(IN); - setState(1249); - match(T__1); - setState(1250); - query(); - setState(1251); - match(T__3); - } - break; - case 4: - enterOuterAlt(_localctx, 4); - { - setState(1254); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==NOT) { - { - setState(1253); - match(NOT); - } - } - - setState(1256); - ((PredicateContext)_localctx).kind = match(RLIKE); - setState(1257); - ((PredicateContext)_localctx).pattern = valueExpression(0); - } - break; - case 5: - enterOuterAlt(_localctx, 5); - { - setState(1259); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==NOT) { - { - setState(1258); - match(NOT); - } - } - - setState(1261); - ((PredicateContext)_localctx).kind = match(LIKE); - setState(1262); - ((PredicateContext)_localctx).quantifier = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==ALL || _la==ANY || _la==SOME) ) { - ((PredicateContext)_localctx).quantifier = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1276); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,177,_ctx) ) { - case 1: - { - setState(1263); - match(T__1); - setState(1264); - match(T__3); - } - break; - case 2: - { - setState(1265); - match(T__1); - setState(1266); - expression(); - setState(1271); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1267); - match(T__2); - setState(1268); - expression(); - } - } - setState(1273); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(1274); - match(T__3); - } - break; - } - } - break; - case 6: - enterOuterAlt(_localctx, 6); - { - setState(1279); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==NOT) { - { - setState(1278); - match(NOT); - } - } - - setState(1281); - ((PredicateContext)_localctx).kind = match(LIKE); - setState(1282); - ((PredicateContext)_localctx).pattern = valueExpression(0); - setState(1285); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,179,_ctx) ) { - case 1: - { - setState(1283); - match(ESCAPE); - setState(1284); - ((PredicateContext)_localctx).escapeChar = match(STRING); - } - break; - } - } - break; - case 7: - enterOuterAlt(_localctx, 7); - { - setState(1287); - match(IS); - setState(1289); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==NOT) { - { - setState(1288); - match(NOT); - } - } - - setState(1291); - ((PredicateContext)_localctx).kind = match(NULL); - } - break; - case 8: - enterOuterAlt(_localctx, 8); - { - setState(1292); - match(IS); - setState(1294); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==NOT) { - { - setState(1293); - match(NOT); - } - } - - setState(1296); - ((PredicateContext)_localctx).kind = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==FALSE || _la==TRUE || _la==UNKNOWN) ) { - ((PredicateContext)_localctx).kind = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - break; - case 9: - enterOuterAlt(_localctx, 9); - { - setState(1297); - match(IS); - setState(1299); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==NOT) { - { - setState(1298); - match(NOT); - } - } - - setState(1301); - ((PredicateContext)_localctx).kind = match(DISTINCT); - setState(1302); - match(FROM); - setState(1303); - ((PredicateContext)_localctx).right = valueExpression(0); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ValueExpressionContext extends ParserRuleContext { - public ValueExpressionContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_valueExpression; } - - public ValueExpressionContext() { } - public void copyFrom(ValueExpressionContext ctx) { - super.copyFrom(ctx); - } - } - public static class ValueExpressionDefaultContext extends ValueExpressionContext { - public PrimaryExpressionContext primaryExpression() { - return getRuleContext(PrimaryExpressionContext.class,0); - } - public ValueExpressionDefaultContext(ValueExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterValueExpressionDefault(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitValueExpressionDefault(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitValueExpressionDefault(this); - else return visitor.visitChildren(this); - } - } - public static class ComparisonContext extends ValueExpressionContext { - public ValueExpressionContext left; - public ValueExpressionContext right; - public ComparisonOperatorContext comparisonOperator() { - return getRuleContext(ComparisonOperatorContext.class,0); - } - public List valueExpression() { - return getRuleContexts(ValueExpressionContext.class); - } - public ValueExpressionContext valueExpression(int i) { - return getRuleContext(ValueExpressionContext.class,i); - } - public ComparisonContext(ValueExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterComparison(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitComparison(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitComparison(this); - else return visitor.visitChildren(this); - } - } - public static class ArithmeticBinaryContext extends ValueExpressionContext { - public ValueExpressionContext left; - public Token operator; - public ValueExpressionContext right; - public List valueExpression() { - return getRuleContexts(ValueExpressionContext.class); - } - public ValueExpressionContext valueExpression(int i) { - return getRuleContext(ValueExpressionContext.class,i); - } - public TerminalNode ASTERISK() { return getToken(MixedFormatSqlExtendParser.ASTERISK, 0); } - public TerminalNode SLASH() { return getToken(MixedFormatSqlExtendParser.SLASH, 0); } - public TerminalNode PERCENT() { return getToken(MixedFormatSqlExtendParser.PERCENT, 0); } - public TerminalNode DIV() { return getToken(MixedFormatSqlExtendParser.DIV, 0); } - public TerminalNode PLUS() { return getToken(MixedFormatSqlExtendParser.PLUS, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public TerminalNode CONCAT_PIPE() { return getToken(MixedFormatSqlExtendParser.CONCAT_PIPE, 0); } - public TerminalNode AMPERSAND() { return getToken(MixedFormatSqlExtendParser.AMPERSAND, 0); } - public TerminalNode HAT() { return getToken(MixedFormatSqlExtendParser.HAT, 0); } - public TerminalNode PIPE() { return getToken(MixedFormatSqlExtendParser.PIPE, 0); } - public ArithmeticBinaryContext(ValueExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterArithmeticBinary(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitArithmeticBinary(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitArithmeticBinary(this); - else return visitor.visitChildren(this); - } - } - public static class ArithmeticUnaryContext extends ValueExpressionContext { - public Token operator; - public ValueExpressionContext valueExpression() { - return getRuleContext(ValueExpressionContext.class,0); - } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public TerminalNode PLUS() { return getToken(MixedFormatSqlExtendParser.PLUS, 0); } - public TerminalNode TILDE() { return getToken(MixedFormatSqlExtendParser.TILDE, 0); } - public ArithmeticUnaryContext(ValueExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterArithmeticUnary(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitArithmeticUnary(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitArithmeticUnary(this); - else return visitor.visitChildren(this); - } - } - - public final ValueExpressionContext valueExpression() throws RecognitionException { - return valueExpression(0); - } - - private ValueExpressionContext valueExpression(int _p) throws RecognitionException { - ParserRuleContext _parentctx = _ctx; - int _parentState = getState(); - ValueExpressionContext _localctx = new ValueExpressionContext(_ctx, _parentState); - ValueExpressionContext _prevctx = _localctx; - int _startState = 146; - enterRecursionRule(_localctx, 146, RULE_valueExpression, _p); - int _la; - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(1310); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,184,_ctx) ) { - case 1: - { - _localctx = new ValueExpressionDefaultContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - - setState(1307); - primaryExpression(0); - } - break; - case 2: - { - _localctx = new ArithmeticUnaryContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1308); - ((ArithmeticUnaryContext)_localctx).operator = _input.LT(1); - _la = _input.LA(1); - if ( !(((((_la - 279)) & ~0x3f) == 0 && ((1L << (_la - 279)) & ((1L << (PLUS - 279)) | (1L << (MINUS - 279)) | (1L << (TILDE - 279)))) != 0)) ) { - ((ArithmeticUnaryContext)_localctx).operator = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1309); - valueExpression(7); - } - break; - } - _ctx.stop = _input.LT(-1); - setState(1333); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,186,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - if ( _parseListeners!=null ) triggerExitRuleEvent(); - _prevctx = _localctx; - { - setState(1331); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,185,_ctx) ) { - case 1: - { - _localctx = new ArithmeticBinaryContext(new ValueExpressionContext(_parentctx, _parentState)); - ((ArithmeticBinaryContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_valueExpression); - setState(1312); - if (!(precpred(_ctx, 6))) throw new FailedPredicateException(this, "precpred(_ctx, 6)"); - setState(1313); - ((ArithmeticBinaryContext)_localctx).operator = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==DIV || ((((_la - 281)) & ~0x3f) == 0 && ((1L << (_la - 281)) & ((1L << (ASTERISK - 281)) | (1L << (SLASH - 281)) | (1L << (PERCENT - 281)))) != 0)) ) { - ((ArithmeticBinaryContext)_localctx).operator = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1314); - ((ArithmeticBinaryContext)_localctx).right = valueExpression(7); - } - break; - case 2: - { - _localctx = new ArithmeticBinaryContext(new ValueExpressionContext(_parentctx, _parentState)); - ((ArithmeticBinaryContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_valueExpression); - setState(1315); - if (!(precpred(_ctx, 5))) throw new FailedPredicateException(this, "precpred(_ctx, 5)"); - setState(1316); - ((ArithmeticBinaryContext)_localctx).operator = _input.LT(1); - _la = _input.LA(1); - if ( !(((((_la - 279)) & ~0x3f) == 0 && ((1L << (_la - 279)) & ((1L << (PLUS - 279)) | (1L << (MINUS - 279)) | (1L << (CONCAT_PIPE - 279)))) != 0)) ) { - ((ArithmeticBinaryContext)_localctx).operator = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1317); - ((ArithmeticBinaryContext)_localctx).right = valueExpression(6); - } - break; - case 3: - { - _localctx = new ArithmeticBinaryContext(new ValueExpressionContext(_parentctx, _parentState)); - ((ArithmeticBinaryContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_valueExpression); - setState(1318); - if (!(precpred(_ctx, 4))) throw new FailedPredicateException(this, "precpred(_ctx, 4)"); - setState(1319); - ((ArithmeticBinaryContext)_localctx).operator = match(AMPERSAND); - setState(1320); - ((ArithmeticBinaryContext)_localctx).right = valueExpression(5); - } - break; - case 4: - { - _localctx = new ArithmeticBinaryContext(new ValueExpressionContext(_parentctx, _parentState)); - ((ArithmeticBinaryContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_valueExpression); - setState(1321); - if (!(precpred(_ctx, 3))) throw new FailedPredicateException(this, "precpred(_ctx, 3)"); - setState(1322); - ((ArithmeticBinaryContext)_localctx).operator = match(HAT); - setState(1323); - ((ArithmeticBinaryContext)_localctx).right = valueExpression(4); - } - break; - case 5: - { - _localctx = new ArithmeticBinaryContext(new ValueExpressionContext(_parentctx, _parentState)); - ((ArithmeticBinaryContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_valueExpression); - setState(1324); - if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)"); - setState(1325); - ((ArithmeticBinaryContext)_localctx).operator = match(PIPE); - setState(1326); - ((ArithmeticBinaryContext)_localctx).right = valueExpression(3); - } - break; - case 6: - { - _localctx = new ComparisonContext(new ValueExpressionContext(_parentctx, _parentState)); - ((ComparisonContext)_localctx).left = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_valueExpression); - setState(1327); - if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)"); - setState(1328); - comparisonOperator(); - setState(1329); - ((ComparisonContext)_localctx).right = valueExpression(2); - } - break; - } - } - } - setState(1335); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,186,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - unrollRecursionContexts(_parentctx); - } - return _localctx; - } - - public static class PrimaryExpressionContext extends ParserRuleContext { - public PrimaryExpressionContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_primaryExpression; } - - public PrimaryExpressionContext() { } - public void copyFrom(PrimaryExpressionContext ctx) { - super.copyFrom(ctx); - } - } - public static class StructContext extends PrimaryExpressionContext { - public NamedExpressionContext namedExpression; - public List argument = new ArrayList(); - public TerminalNode STRUCT() { return getToken(MixedFormatSqlExtendParser.STRUCT, 0); } - public List namedExpression() { - return getRuleContexts(NamedExpressionContext.class); - } - public NamedExpressionContext namedExpression(int i) { - return getRuleContext(NamedExpressionContext.class,i); - } - public StructContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterStruct(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitStruct(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitStruct(this); - else return visitor.visitChildren(this); - } - } - public static class DereferenceContext extends PrimaryExpressionContext { - public PrimaryExpressionContext base; - public IdentifierContext fieldName; - public PrimaryExpressionContext primaryExpression() { - return getRuleContext(PrimaryExpressionContext.class,0); - } - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public DereferenceContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterDereference(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitDereference(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitDereference(this); - else return visitor.visitChildren(this); - } - } - public static class SimpleCaseContext extends PrimaryExpressionContext { - public ExpressionContext value; - public ExpressionContext elseExpression; - public TerminalNode CASE() { return getToken(MixedFormatSqlExtendParser.CASE, 0); } - public TerminalNode END() { return getToken(MixedFormatSqlExtendParser.END, 0); } - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public List whenClause() { - return getRuleContexts(WhenClauseContext.class); - } - public WhenClauseContext whenClause(int i) { - return getRuleContext(WhenClauseContext.class,i); - } - public TerminalNode ELSE() { return getToken(MixedFormatSqlExtendParser.ELSE, 0); } - public SimpleCaseContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSimpleCase(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSimpleCase(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSimpleCase(this); - else return visitor.visitChildren(this); - } - } - public static class CurrentLikeContext extends PrimaryExpressionContext { - public Token name; - public TerminalNode CURRENT_DATE() { return getToken(MixedFormatSqlExtendParser.CURRENT_DATE, 0); } - public TerminalNode CURRENT_TIMESTAMP() { return getToken(MixedFormatSqlExtendParser.CURRENT_TIMESTAMP, 0); } - public TerminalNode CURRENT_USER() { return getToken(MixedFormatSqlExtendParser.CURRENT_USER, 0); } - public CurrentLikeContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterCurrentLike(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitCurrentLike(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitCurrentLike(this); - else return visitor.visitChildren(this); - } - } - public static class ColumnReferenceContext extends PrimaryExpressionContext { - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public ColumnReferenceContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterColumnReference(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitColumnReference(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitColumnReference(this); - else return visitor.visitChildren(this); - } - } - public static class RowConstructorContext extends PrimaryExpressionContext { - public List namedExpression() { - return getRuleContexts(NamedExpressionContext.class); - } - public NamedExpressionContext namedExpression(int i) { - return getRuleContext(NamedExpressionContext.class,i); - } - public RowConstructorContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterRowConstructor(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitRowConstructor(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitRowConstructor(this); - else return visitor.visitChildren(this); - } - } - public static class LastContext extends PrimaryExpressionContext { - public TerminalNode LAST() { return getToken(MixedFormatSqlExtendParser.LAST, 0); } - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public TerminalNode IGNORE() { return getToken(MixedFormatSqlExtendParser.IGNORE, 0); } - public TerminalNode NULLS() { return getToken(MixedFormatSqlExtendParser.NULLS, 0); } - public LastContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterLast(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitLast(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitLast(this); - else return visitor.visitChildren(this); - } - } - public static class StarContext extends PrimaryExpressionContext { - public TerminalNode ASTERISK() { return getToken(MixedFormatSqlExtendParser.ASTERISK, 0); } - public QualifiedNameContext qualifiedName() { - return getRuleContext(QualifiedNameContext.class,0); - } - public StarContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterStar(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitStar(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitStar(this); - else return visitor.visitChildren(this); - } - } - public static class OverlayContext extends PrimaryExpressionContext { - public ValueExpressionContext input; - public ValueExpressionContext replace; - public ValueExpressionContext position; - public ValueExpressionContext length; - public TerminalNode OVERLAY() { return getToken(MixedFormatSqlExtendParser.OVERLAY, 0); } - public TerminalNode PLACING() { return getToken(MixedFormatSqlExtendParser.PLACING, 0); } - public TerminalNode FROM() { return getToken(MixedFormatSqlExtendParser.FROM, 0); } - public List valueExpression() { - return getRuleContexts(ValueExpressionContext.class); - } - public ValueExpressionContext valueExpression(int i) { - return getRuleContext(ValueExpressionContext.class,i); - } - public TerminalNode FOR() { return getToken(MixedFormatSqlExtendParser.FOR, 0); } - public OverlayContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterOverlay(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitOverlay(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitOverlay(this); - else return visitor.visitChildren(this); - } - } - public static class SubscriptContext extends PrimaryExpressionContext { - public PrimaryExpressionContext value; - public ValueExpressionContext index; - public PrimaryExpressionContext primaryExpression() { - return getRuleContext(PrimaryExpressionContext.class,0); - } - public ValueExpressionContext valueExpression() { - return getRuleContext(ValueExpressionContext.class,0); - } - public SubscriptContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSubscript(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSubscript(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSubscript(this); - else return visitor.visitChildren(this); - } - } - public static class SubqueryExpressionContext extends PrimaryExpressionContext { - public QueryContext query() { - return getRuleContext(QueryContext.class,0); - } - public SubqueryExpressionContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSubqueryExpression(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSubqueryExpression(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSubqueryExpression(this); - else return visitor.visitChildren(this); - } - } - public static class SubstringContext extends PrimaryExpressionContext { - public ValueExpressionContext str; - public ValueExpressionContext pos; - public ValueExpressionContext len; - public TerminalNode SUBSTR() { return getToken(MixedFormatSqlExtendParser.SUBSTR, 0); } - public TerminalNode SUBSTRING() { return getToken(MixedFormatSqlExtendParser.SUBSTRING, 0); } - public List valueExpression() { - return getRuleContexts(ValueExpressionContext.class); - } - public ValueExpressionContext valueExpression(int i) { - return getRuleContext(ValueExpressionContext.class,i); - } - public TerminalNode FROM() { return getToken(MixedFormatSqlExtendParser.FROM, 0); } - public TerminalNode FOR() { return getToken(MixedFormatSqlExtendParser.FOR, 0); } - public SubstringContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSubstring(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSubstring(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSubstring(this); - else return visitor.visitChildren(this); - } - } - public static class CastContext extends PrimaryExpressionContext { - public Token name; - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public DataTypeContext dataType() { - return getRuleContext(DataTypeContext.class,0); - } - public TerminalNode CAST() { return getToken(MixedFormatSqlExtendParser.CAST, 0); } - public TerminalNode TRY_CAST() { return getToken(MixedFormatSqlExtendParser.TRY_CAST, 0); } - public CastContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterCast(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitCast(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitCast(this); - else return visitor.visitChildren(this); - } - } - public static class ConstantDefaultContext extends PrimaryExpressionContext { - public ConstantContext constant() { - return getRuleContext(ConstantContext.class,0); - } - public ConstantDefaultContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterConstantDefault(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitConstantDefault(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitConstantDefault(this); - else return visitor.visitChildren(this); - } - } - public static class LambdaContext extends PrimaryExpressionContext { - public List identifier() { - return getRuleContexts(IdentifierContext.class); - } - public IdentifierContext identifier(int i) { - return getRuleContext(IdentifierContext.class,i); - } - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public LambdaContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterLambda(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitLambda(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitLambda(this); - else return visitor.visitChildren(this); - } - } - public static class ParenthesizedExpressionContext extends PrimaryExpressionContext { - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public ParenthesizedExpressionContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterParenthesizedExpression(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitParenthesizedExpression(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitParenthesizedExpression(this); - else return visitor.visitChildren(this); - } - } - public static class ExtractContext extends PrimaryExpressionContext { - public IdentifierContext field; - public ValueExpressionContext source; - public TerminalNode EXTRACT() { return getToken(MixedFormatSqlExtendParser.EXTRACT, 0); } - public TerminalNode FROM() { return getToken(MixedFormatSqlExtendParser.FROM, 0); } - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public ValueExpressionContext valueExpression() { - return getRuleContext(ValueExpressionContext.class,0); - } - public ExtractContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterExtract(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitExtract(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitExtract(this); - else return visitor.visitChildren(this); - } - } - public static class TrimContext extends PrimaryExpressionContext { - public Token trimOption; - public ValueExpressionContext trimStr; - public ValueExpressionContext srcStr; - public TerminalNode TRIM() { return getToken(MixedFormatSqlExtendParser.TRIM, 0); } - public TerminalNode FROM() { return getToken(MixedFormatSqlExtendParser.FROM, 0); } - public List valueExpression() { - return getRuleContexts(ValueExpressionContext.class); - } - public ValueExpressionContext valueExpression(int i) { - return getRuleContext(ValueExpressionContext.class,i); - } - public TerminalNode BOTH() { return getToken(MixedFormatSqlExtendParser.BOTH, 0); } - public TerminalNode LEADING() { return getToken(MixedFormatSqlExtendParser.LEADING, 0); } - public TerminalNode TRAILING() { return getToken(MixedFormatSqlExtendParser.TRAILING, 0); } - public TrimContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTrim(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTrim(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTrim(this); - else return visitor.visitChildren(this); - } - } - public static class FunctionCallContext extends PrimaryExpressionContext { - public ExpressionContext expression; - public List argument = new ArrayList(); - public BooleanExpressionContext where; - public Token nullsOption; - public FunctionNameContext functionName() { - return getRuleContext(FunctionNameContext.class,0); - } - public TerminalNode FILTER() { return getToken(MixedFormatSqlExtendParser.FILTER, 0); } - public TerminalNode WHERE() { return getToken(MixedFormatSqlExtendParser.WHERE, 0); } - public TerminalNode NULLS() { return getToken(MixedFormatSqlExtendParser.NULLS, 0); } - public TerminalNode OVER() { return getToken(MixedFormatSqlExtendParser.OVER, 0); } - public WindowSpecContext windowSpec() { - return getRuleContext(WindowSpecContext.class,0); - } - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public BooleanExpressionContext booleanExpression() { - return getRuleContext(BooleanExpressionContext.class,0); - } - public TerminalNode IGNORE() { return getToken(MixedFormatSqlExtendParser.IGNORE, 0); } - public TerminalNode RESPECT() { return getToken(MixedFormatSqlExtendParser.RESPECT, 0); } - public SetQuantifierContext setQuantifier() { - return getRuleContext(SetQuantifierContext.class,0); - } - public FunctionCallContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterFunctionCall(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitFunctionCall(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitFunctionCall(this); - else return visitor.visitChildren(this); - } - } - public static class SearchedCaseContext extends PrimaryExpressionContext { - public ExpressionContext elseExpression; - public TerminalNode CASE() { return getToken(MixedFormatSqlExtendParser.CASE, 0); } - public TerminalNode END() { return getToken(MixedFormatSqlExtendParser.END, 0); } - public List whenClause() { - return getRuleContexts(WhenClauseContext.class); - } - public WhenClauseContext whenClause(int i) { - return getRuleContext(WhenClauseContext.class,i); - } - public TerminalNode ELSE() { return getToken(MixedFormatSqlExtendParser.ELSE, 0); } - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public SearchedCaseContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSearchedCase(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSearchedCase(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSearchedCase(this); - else return visitor.visitChildren(this); - } - } - public static class PositionContext extends PrimaryExpressionContext { - public ValueExpressionContext substr; - public ValueExpressionContext str; - public TerminalNode POSITION() { return getToken(MixedFormatSqlExtendParser.POSITION, 0); } - public TerminalNode IN() { return getToken(MixedFormatSqlExtendParser.IN, 0); } - public List valueExpression() { - return getRuleContexts(ValueExpressionContext.class); - } - public ValueExpressionContext valueExpression(int i) { - return getRuleContext(ValueExpressionContext.class,i); - } - public PositionContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPosition(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPosition(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPosition(this); - else return visitor.visitChildren(this); - } - } - public static class FirstContext extends PrimaryExpressionContext { - public TerminalNode FIRST() { return getToken(MixedFormatSqlExtendParser.FIRST, 0); } - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public TerminalNode IGNORE() { return getToken(MixedFormatSqlExtendParser.IGNORE, 0); } - public TerminalNode NULLS() { return getToken(MixedFormatSqlExtendParser.NULLS, 0); } - public FirstContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterFirst(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitFirst(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitFirst(this); - else return visitor.visitChildren(this); - } - } - - public final PrimaryExpressionContext primaryExpression() throws RecognitionException { - return primaryExpression(0); - } - - private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionException { - ParserRuleContext _parentctx = _ctx; - int _parentState = getState(); - PrimaryExpressionContext _localctx = new PrimaryExpressionContext(_ctx, _parentState); - PrimaryExpressionContext _prevctx = _localctx; - int _startState = 148; - enterRecursionRule(_localctx, 148, RULE_primaryExpression, _p); - int _la; - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(1524); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,207,_ctx) ) { - case 1: - { - _localctx = new CurrentLikeContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - - setState(1337); - ((CurrentLikeContext)_localctx).name = _input.LT(1); - _la = _input.LA(1); - if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << CURRENT_DATE) | (1L << CURRENT_TIMESTAMP) | (1L << CURRENT_USER))) != 0)) ) { - ((CurrentLikeContext)_localctx).name = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - break; - case 2: - { - _localctx = new SearchedCaseContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1338); - match(CASE); - setState(1340); - _errHandler.sync(this); - _la = _input.LA(1); - do { - { - { - setState(1339); - whenClause(); - } - } - setState(1342); - _errHandler.sync(this); - _la = _input.LA(1); - } while ( _la==WHEN ); - setState(1346); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==ELSE) { - { - setState(1344); - match(ELSE); - setState(1345); - ((SearchedCaseContext)_localctx).elseExpression = expression(); - } - } - - setState(1348); - match(END); - } - break; - case 3: - { - _localctx = new SimpleCaseContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1350); - match(CASE); - setState(1351); - ((SimpleCaseContext)_localctx).value = expression(); - setState(1353); - _errHandler.sync(this); - _la = _input.LA(1); - do { - { - { - setState(1352); - whenClause(); - } - } - setState(1355); - _errHandler.sync(this); - _la = _input.LA(1); - } while ( _la==WHEN ); - setState(1359); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==ELSE) { - { - setState(1357); - match(ELSE); - setState(1358); - ((SimpleCaseContext)_localctx).elseExpression = expression(); - } - } - - setState(1361); - match(END); - } - break; - case 4: - { - _localctx = new CastContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1363); - ((CastContext)_localctx).name = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==CAST || _la==TRY_CAST) ) { - ((CastContext)_localctx).name = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1364); - match(T__1); - setState(1365); - expression(); - setState(1366); - match(AS); - setState(1367); - dataType(); - setState(1368); - match(T__3); - } - break; - case 5: - { - _localctx = new StructContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1370); - match(STRUCT); - setState(1371); - match(T__1); - setState(1380); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,192,_ctx) ) { - case 1: - { - setState(1372); - ((StructContext)_localctx).namedExpression = namedExpression(); - ((StructContext)_localctx).argument.add(((StructContext)_localctx).namedExpression); - setState(1377); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1373); - match(T__2); - setState(1374); - ((StructContext)_localctx).namedExpression = namedExpression(); - ((StructContext)_localctx).argument.add(((StructContext)_localctx).namedExpression); - } - } - setState(1379); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - break; - } - setState(1382); - match(T__3); - } - break; - case 6: - { - _localctx = new FirstContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1383); - match(FIRST); - setState(1384); - match(T__1); - setState(1385); - expression(); - setState(1388); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==IGNORE) { - { - setState(1386); - match(IGNORE); - setState(1387); - match(NULLS); - } - } - - setState(1390); - match(T__3); - } - break; - case 7: - { - _localctx = new LastContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1392); - match(LAST); - setState(1393); - match(T__1); - setState(1394); - expression(); - setState(1397); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==IGNORE) { - { - setState(1395); - match(IGNORE); - setState(1396); - match(NULLS); - } - } - - setState(1399); - match(T__3); - } - break; - case 8: - { - _localctx = new PositionContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1401); - match(POSITION); - setState(1402); - match(T__1); - setState(1403); - ((PositionContext)_localctx).substr = valueExpression(0); - setState(1404); - match(IN); - setState(1405); - ((PositionContext)_localctx).str = valueExpression(0); - setState(1406); - match(T__3); - } - break; - case 9: - { - _localctx = new ConstantDefaultContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1408); - constant(); - } - break; - case 10: - { - _localctx = new StarContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1409); - match(ASTERISK); - } - break; - case 11: - { - _localctx = new StarContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1410); - qualifiedName(); - setState(1411); - match(T__4); - setState(1412); - match(ASTERISK); - } - break; - case 12: - { - _localctx = new RowConstructorContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1414); - match(T__1); - setState(1415); - namedExpression(); - setState(1418); - _errHandler.sync(this); - _la = _input.LA(1); - do { - { - { - setState(1416); - match(T__2); - setState(1417); - namedExpression(); - } - } - setState(1420); - _errHandler.sync(this); - _la = _input.LA(1); - } while ( _la==T__2 ); - setState(1422); - match(T__3); - } - break; - case 13: - { - _localctx = new SubqueryExpressionContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1424); - match(T__1); - setState(1425); - query(); - setState(1426); - match(T__3); - } - break; - case 14: - { - _localctx = new FunctionCallContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1428); - functionName(); - setState(1429); - match(T__1); - setState(1441); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,198,_ctx) ) { - case 1: - { - setState(1431); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,196,_ctx) ) { - case 1: - { - setState(1430); - setQuantifier(); - } - break; - } - setState(1433); - ((FunctionCallContext)_localctx).expression = expression(); - ((FunctionCallContext)_localctx).argument.add(((FunctionCallContext)_localctx).expression); - setState(1438); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1434); - match(T__2); - setState(1435); - ((FunctionCallContext)_localctx).expression = expression(); - ((FunctionCallContext)_localctx).argument.add(((FunctionCallContext)_localctx).expression); - } - } - setState(1440); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - break; - } - setState(1443); - match(T__3); - setState(1450); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,199,_ctx) ) { - case 1: - { - setState(1444); - match(FILTER); - setState(1445); - match(T__1); - setState(1446); - match(WHERE); - setState(1447); - ((FunctionCallContext)_localctx).where = booleanExpression(0); - setState(1448); - match(T__3); - } - break; - } - setState(1454); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,200,_ctx) ) { - case 1: - { - setState(1452); - ((FunctionCallContext)_localctx).nullsOption = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==IGNORE || _la==RESPECT) ) { - ((FunctionCallContext)_localctx).nullsOption = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1453); - match(NULLS); - } - break; - } - setState(1458); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,201,_ctx) ) { - case 1: - { - setState(1456); - match(OVER); - setState(1457); - windowSpec(); - } - break; - } - } - break; - case 15: - { - _localctx = new LambdaContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1460); - identifier(); - setState(1461); - match(T__7); - setState(1462); - expression(); - } - break; - case 16: - { - _localctx = new LambdaContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1464); - match(T__1); - setState(1465); - identifier(); - setState(1468); - _errHandler.sync(this); - _la = _input.LA(1); - do { - { - { - setState(1466); - match(T__2); - setState(1467); - identifier(); - } - } - setState(1470); - _errHandler.sync(this); - _la = _input.LA(1); - } while ( _la==T__2 ); - setState(1472); - match(T__3); - setState(1473); - match(T__7); - setState(1474); - expression(); - } - break; - case 17: - { - _localctx = new ColumnReferenceContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1476); - identifier(); - } - break; - case 18: - { - _localctx = new ParenthesizedExpressionContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1477); - match(T__1); - setState(1478); - expression(); - setState(1479); - match(T__3); - } - break; - case 19: - { - _localctx = new ExtractContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1481); - match(EXTRACT); - setState(1482); - match(T__1); - setState(1483); - ((ExtractContext)_localctx).field = identifier(); - setState(1484); - match(FROM); - setState(1485); - ((ExtractContext)_localctx).source = valueExpression(0); - setState(1486); - match(T__3); - } - break; - case 20: - { - _localctx = new SubstringContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1488); - _la = _input.LA(1); - if ( !(_la==SUBSTR || _la==SUBSTRING) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1489); - match(T__1); - setState(1490); - ((SubstringContext)_localctx).str = valueExpression(0); - setState(1491); - _la = _input.LA(1); - if ( !(_la==T__2 || _la==FROM) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1492); - ((SubstringContext)_localctx).pos = valueExpression(0); - setState(1495); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==T__2 || _la==FOR) { - { - setState(1493); - _la = _input.LA(1); - if ( !(_la==T__2 || _la==FOR) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1494); - ((SubstringContext)_localctx).len = valueExpression(0); - } - } - - setState(1497); - match(T__3); - } - break; - case 21: - { - _localctx = new TrimContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1499); - match(TRIM); - setState(1500); - match(T__1); - setState(1502); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,204,_ctx) ) { - case 1: - { - setState(1501); - ((TrimContext)_localctx).trimOption = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==BOTH || _la==LEADING || _la==TRAILING) ) { - ((TrimContext)_localctx).trimOption = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - break; - } - setState(1505); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,205,_ctx) ) { - case 1: - { - setState(1504); - ((TrimContext)_localctx).trimStr = valueExpression(0); - } - break; - } - setState(1507); - match(FROM); - setState(1508); - ((TrimContext)_localctx).srcStr = valueExpression(0); - setState(1509); - match(T__3); - } - break; - case 22: - { - _localctx = new OverlayContext(_localctx); - _ctx = _localctx; - _prevctx = _localctx; - setState(1511); - match(OVERLAY); - setState(1512); - match(T__1); - setState(1513); - ((OverlayContext)_localctx).input = valueExpression(0); - setState(1514); - match(PLACING); - setState(1515); - ((OverlayContext)_localctx).replace = valueExpression(0); - setState(1516); - match(FROM); - setState(1517); - ((OverlayContext)_localctx).position = valueExpression(0); - setState(1520); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==FOR) { - { - setState(1518); - match(FOR); - setState(1519); - ((OverlayContext)_localctx).length = valueExpression(0); - } - } - - setState(1522); - match(T__3); - } - break; - } - _ctx.stop = _input.LT(-1); - setState(1536); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,209,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - if ( _parseListeners!=null ) triggerExitRuleEvent(); - _prevctx = _localctx; - { - setState(1534); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,208,_ctx) ) { - case 1: - { - _localctx = new SubscriptContext(new PrimaryExpressionContext(_parentctx, _parentState)); - ((SubscriptContext)_localctx).value = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_primaryExpression); - setState(1526); - if (!(precpred(_ctx, 8))) throw new FailedPredicateException(this, "precpred(_ctx, 8)"); - setState(1527); - match(T__8); - setState(1528); - ((SubscriptContext)_localctx).index = valueExpression(0); - setState(1529); - match(T__9); - } - break; - case 2: - { - _localctx = new DereferenceContext(new PrimaryExpressionContext(_parentctx, _parentState)); - ((DereferenceContext)_localctx).base = _prevctx; - pushNewRecursionContext(_localctx, _startState, RULE_primaryExpression); - setState(1531); - if (!(precpred(_ctx, 6))) throw new FailedPredicateException(this, "precpred(_ctx, 6)"); - setState(1532); - match(T__4); - setState(1533); - ((DereferenceContext)_localctx).fieldName = identifier(); - } - break; - } - } - } - setState(1538); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,209,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - unrollRecursionContexts(_parentctx); - } - return _localctx; - } - - public static class ConstantContext extends ParserRuleContext { - public ConstantContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_constant; } - - public ConstantContext() { } - public void copyFrom(ConstantContext ctx) { - super.copyFrom(ctx); - } - } - public static class NullLiteralContext extends ConstantContext { - public TerminalNode NULL() { return getToken(MixedFormatSqlExtendParser.NULL, 0); } - public NullLiteralContext(ConstantContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterNullLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitNullLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitNullLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class StringLiteralContext extends ConstantContext { - public List STRING() { return getTokens(MixedFormatSqlExtendParser.STRING); } - public TerminalNode STRING(int i) { - return getToken(MixedFormatSqlExtendParser.STRING, i); - } - public StringLiteralContext(ConstantContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterStringLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitStringLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitStringLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class TypeConstructorContext extends ConstantContext { - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public TerminalNode STRING() { return getToken(MixedFormatSqlExtendParser.STRING, 0); } - public TypeConstructorContext(ConstantContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTypeConstructor(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTypeConstructor(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTypeConstructor(this); - else return visitor.visitChildren(this); - } - } - public static class IntervalLiteralContext extends ConstantContext { - public IntervalContext interval() { - return getRuleContext(IntervalContext.class,0); - } - public IntervalLiteralContext(ConstantContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterIntervalLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitIntervalLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitIntervalLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class NumericLiteralContext extends ConstantContext { - public NumberContext number() { - return getRuleContext(NumberContext.class,0); - } - public NumericLiteralContext(ConstantContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterNumericLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitNumericLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitNumericLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class BooleanLiteralContext extends ConstantContext { - public BooleanValueContext booleanValue() { - return getRuleContext(BooleanValueContext.class,0); - } - public BooleanLiteralContext(ConstantContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterBooleanLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitBooleanLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitBooleanLiteral(this); - else return visitor.visitChildren(this); - } - } - - public final ConstantContext constant() throws RecognitionException { - ConstantContext _localctx = new ConstantContext(_ctx, getState()); - enterRule(_localctx, 150, RULE_constant); - try { - int _alt; - setState(1551); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,211,_ctx) ) { - case 1: - _localctx = new NullLiteralContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(1539); - match(NULL); - } - break; - case 2: - _localctx = new IntervalLiteralContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(1540); - interval(); - } - break; - case 3: - _localctx = new TypeConstructorContext(_localctx); - enterOuterAlt(_localctx, 3); - { - setState(1541); - identifier(); - setState(1542); - match(STRING); - } - break; - case 4: - _localctx = new NumericLiteralContext(_localctx); - enterOuterAlt(_localctx, 4); - { - setState(1544); - number(); - } - break; - case 5: - _localctx = new BooleanLiteralContext(_localctx); - enterOuterAlt(_localctx, 5); - { - setState(1545); - booleanValue(); - } - break; - case 6: - _localctx = new StringLiteralContext(_localctx); - enterOuterAlt(_localctx, 6); - { - setState(1547); - _errHandler.sync(this); - _alt = 1; - do { - switch (_alt) { - case 1: - { - { - setState(1546); - match(STRING); - } - } - break; - default: - throw new NoViableAltException(this); - } - setState(1549); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,210,_ctx); - } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ComparisonOperatorContext extends ParserRuleContext { - public TerminalNode EQ() { return getToken(MixedFormatSqlExtendParser.EQ, 0); } - public TerminalNode NEQ() { return getToken(MixedFormatSqlExtendParser.NEQ, 0); } - public TerminalNode NEQJ() { return getToken(MixedFormatSqlExtendParser.NEQJ, 0); } - public TerminalNode LT() { return getToken(MixedFormatSqlExtendParser.LT, 0); } - public TerminalNode LTE() { return getToken(MixedFormatSqlExtendParser.LTE, 0); } - public TerminalNode GT() { return getToken(MixedFormatSqlExtendParser.GT, 0); } - public TerminalNode GTE() { return getToken(MixedFormatSqlExtendParser.GTE, 0); } - public TerminalNode NSEQ() { return getToken(MixedFormatSqlExtendParser.NSEQ, 0); } - public ComparisonOperatorContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_comparisonOperator; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterComparisonOperator(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitComparisonOperator(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitComparisonOperator(this); - else return visitor.visitChildren(this); - } - } - - public final ComparisonOperatorContext comparisonOperator() throws RecognitionException { - ComparisonOperatorContext _localctx = new ComparisonOperatorContext(_ctx, getState()); - enterRule(_localctx, 152, RULE_comparisonOperator); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1553); - _la = _input.LA(1); - if ( !(((((_la - 271)) & ~0x3f) == 0 && ((1L << (_la - 271)) & ((1L << (EQ - 271)) | (1L << (NSEQ - 271)) | (1L << (NEQ - 271)) | (1L << (NEQJ - 271)) | (1L << (LT - 271)) | (1L << (LTE - 271)) | (1L << (GT - 271)) | (1L << (GTE - 271)))) != 0)) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class BooleanValueContext extends ParserRuleContext { - public TerminalNode TRUE() { return getToken(MixedFormatSqlExtendParser.TRUE, 0); } - public TerminalNode FALSE() { return getToken(MixedFormatSqlExtendParser.FALSE, 0); } - public BooleanValueContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_booleanValue; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterBooleanValue(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitBooleanValue(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitBooleanValue(this); - else return visitor.visitChildren(this); - } - } - - public final BooleanValueContext booleanValue() throws RecognitionException { - BooleanValueContext _localctx = new BooleanValueContext(_ctx, getState()); - enterRule(_localctx, 154, RULE_booleanValue); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1555); - _la = _input.LA(1); - if ( !(_la==FALSE || _la==TRUE) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class IntervalContext extends ParserRuleContext { - public TerminalNode INTERVAL() { return getToken(MixedFormatSqlExtendParser.INTERVAL, 0); } - public ErrorCapturingMultiUnitsIntervalContext errorCapturingMultiUnitsInterval() { - return getRuleContext(ErrorCapturingMultiUnitsIntervalContext.class,0); - } - public ErrorCapturingUnitToUnitIntervalContext errorCapturingUnitToUnitInterval() { - return getRuleContext(ErrorCapturingUnitToUnitIntervalContext.class,0); - } - public IntervalContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_interval; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterInterval(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitInterval(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitInterval(this); - else return visitor.visitChildren(this); - } - } - - public final IntervalContext interval() throws RecognitionException { - IntervalContext _localctx = new IntervalContext(_ctx, getState()); - enterRule(_localctx, 156, RULE_interval); - try { - enterOuterAlt(_localctx, 1); - { - setState(1557); - match(INTERVAL); - setState(1560); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,212,_ctx) ) { - case 1: - { - setState(1558); - errorCapturingMultiUnitsInterval(); - } - break; - case 2: - { - setState(1559); - errorCapturingUnitToUnitInterval(); - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ErrorCapturingMultiUnitsIntervalContext extends ParserRuleContext { - public MultiUnitsIntervalContext body; - public MultiUnitsIntervalContext multiUnitsInterval() { - return getRuleContext(MultiUnitsIntervalContext.class,0); - } - public UnitToUnitIntervalContext unitToUnitInterval() { - return getRuleContext(UnitToUnitIntervalContext.class,0); - } - public ErrorCapturingMultiUnitsIntervalContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_errorCapturingMultiUnitsInterval; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterErrorCapturingMultiUnitsInterval(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitErrorCapturingMultiUnitsInterval(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitErrorCapturingMultiUnitsInterval(this); - else return visitor.visitChildren(this); - } - } - - public final ErrorCapturingMultiUnitsIntervalContext errorCapturingMultiUnitsInterval() throws RecognitionException { - ErrorCapturingMultiUnitsIntervalContext _localctx = new ErrorCapturingMultiUnitsIntervalContext(_ctx, getState()); - enterRule(_localctx, 158, RULE_errorCapturingMultiUnitsInterval); - try { - enterOuterAlt(_localctx, 1); - { - setState(1562); - ((ErrorCapturingMultiUnitsIntervalContext)_localctx).body = multiUnitsInterval(); - setState(1564); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,213,_ctx) ) { - case 1: - { - setState(1563); - unitToUnitInterval(); - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class MultiUnitsIntervalContext extends ParserRuleContext { - public IdentifierContext identifier; - public List unit = new ArrayList(); - public List intervalValue() { - return getRuleContexts(IntervalValueContext.class); - } - public IntervalValueContext intervalValue(int i) { - return getRuleContext(IntervalValueContext.class,i); - } - public List identifier() { - return getRuleContexts(IdentifierContext.class); - } - public IdentifierContext identifier(int i) { - return getRuleContext(IdentifierContext.class,i); - } - public MultiUnitsIntervalContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_multiUnitsInterval; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterMultiUnitsInterval(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitMultiUnitsInterval(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitMultiUnitsInterval(this); - else return visitor.visitChildren(this); - } - } - - public final MultiUnitsIntervalContext multiUnitsInterval() throws RecognitionException { - MultiUnitsIntervalContext _localctx = new MultiUnitsIntervalContext(_ctx, getState()); - enterRule(_localctx, 160, RULE_multiUnitsInterval); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(1569); - _errHandler.sync(this); - _alt = 1; - do { - switch (_alt) { - case 1: - { - { - setState(1566); - intervalValue(); - setState(1567); - ((MultiUnitsIntervalContext)_localctx).identifier = identifier(); - ((MultiUnitsIntervalContext)_localctx).unit.add(((MultiUnitsIntervalContext)_localctx).identifier); - } - } - break; - default: - throw new NoViableAltException(this); - } - setState(1571); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,214,_ctx); - } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ErrorCapturingUnitToUnitIntervalContext extends ParserRuleContext { - public UnitToUnitIntervalContext body; - public MultiUnitsIntervalContext error1; - public UnitToUnitIntervalContext error2; - public List unitToUnitInterval() { - return getRuleContexts(UnitToUnitIntervalContext.class); - } - public UnitToUnitIntervalContext unitToUnitInterval(int i) { - return getRuleContext(UnitToUnitIntervalContext.class,i); - } - public MultiUnitsIntervalContext multiUnitsInterval() { - return getRuleContext(MultiUnitsIntervalContext.class,0); - } - public ErrorCapturingUnitToUnitIntervalContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_errorCapturingUnitToUnitInterval; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterErrorCapturingUnitToUnitInterval(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitErrorCapturingUnitToUnitInterval(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitErrorCapturingUnitToUnitInterval(this); - else return visitor.visitChildren(this); - } - } - - public final ErrorCapturingUnitToUnitIntervalContext errorCapturingUnitToUnitInterval() throws RecognitionException { - ErrorCapturingUnitToUnitIntervalContext _localctx = new ErrorCapturingUnitToUnitIntervalContext(_ctx, getState()); - enterRule(_localctx, 162, RULE_errorCapturingUnitToUnitInterval); - try { - enterOuterAlt(_localctx, 1); - { - setState(1573); - ((ErrorCapturingUnitToUnitIntervalContext)_localctx).body = unitToUnitInterval(); - setState(1576); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,215,_ctx) ) { - case 1: - { - setState(1574); - ((ErrorCapturingUnitToUnitIntervalContext)_localctx).error1 = multiUnitsInterval(); - } - break; - case 2: - { - setState(1575); - ((ErrorCapturingUnitToUnitIntervalContext)_localctx).error2 = unitToUnitInterval(); - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class UnitToUnitIntervalContext extends ParserRuleContext { - public IntervalValueContext value; - public IdentifierContext from; - public IdentifierContext to; - public TerminalNode TO() { return getToken(MixedFormatSqlExtendParser.TO, 0); } - public IntervalValueContext intervalValue() { - return getRuleContext(IntervalValueContext.class,0); - } - public List identifier() { - return getRuleContexts(IdentifierContext.class); - } - public IdentifierContext identifier(int i) { - return getRuleContext(IdentifierContext.class,i); - } - public UnitToUnitIntervalContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_unitToUnitInterval; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterUnitToUnitInterval(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitUnitToUnitInterval(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitUnitToUnitInterval(this); - else return visitor.visitChildren(this); - } - } - - public final UnitToUnitIntervalContext unitToUnitInterval() throws RecognitionException { - UnitToUnitIntervalContext _localctx = new UnitToUnitIntervalContext(_ctx, getState()); - enterRule(_localctx, 164, RULE_unitToUnitInterval); - try { - enterOuterAlt(_localctx, 1); - { - setState(1578); - ((UnitToUnitIntervalContext)_localctx).value = intervalValue(); - setState(1579); - ((UnitToUnitIntervalContext)_localctx).from = identifier(); - setState(1580); - match(TO); - setState(1581); - ((UnitToUnitIntervalContext)_localctx).to = identifier(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class IntervalValueContext extends ParserRuleContext { - public TerminalNode INTEGER_VALUE() { return getToken(MixedFormatSqlExtendParser.INTEGER_VALUE, 0); } - public TerminalNode DECIMAL_VALUE() { return getToken(MixedFormatSqlExtendParser.DECIMAL_VALUE, 0); } - public TerminalNode STRING() { return getToken(MixedFormatSqlExtendParser.STRING, 0); } - public TerminalNode PLUS() { return getToken(MixedFormatSqlExtendParser.PLUS, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public IntervalValueContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_intervalValue; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterIntervalValue(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitIntervalValue(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitIntervalValue(this); - else return visitor.visitChildren(this); - } - } - - public final IntervalValueContext intervalValue() throws RecognitionException { - IntervalValueContext _localctx = new IntervalValueContext(_ctx, getState()); - enterRule(_localctx, 166, RULE_intervalValue); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1584); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==PLUS || _la==MINUS) { - { - setState(1583); - _la = _input.LA(1); - if ( !(_la==PLUS || _la==MINUS) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - } - - setState(1586); - _la = _input.LA(1); - if ( !(((((_la - 289)) & ~0x3f) == 0 && ((1L << (_la - 289)) & ((1L << (STRING - 289)) | (1L << (INTEGER_VALUE - 289)) | (1L << (DECIMAL_VALUE - 289)))) != 0)) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ColPositionContext extends ParserRuleContext { - public Token position; - public ErrorCapturingIdentifierContext afterCol; - public TerminalNode FIRST() { return getToken(MixedFormatSqlExtendParser.FIRST, 0); } - public TerminalNode AFTER() { return getToken(MixedFormatSqlExtendParser.AFTER, 0); } - public ErrorCapturingIdentifierContext errorCapturingIdentifier() { - return getRuleContext(ErrorCapturingIdentifierContext.class,0); - } - public ColPositionContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_colPosition; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterColPosition(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitColPosition(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitColPosition(this); - else return visitor.visitChildren(this); - } - } - - public final ColPositionContext colPosition() throws RecognitionException { - ColPositionContext _localctx = new ColPositionContext(_ctx, getState()); - enterRule(_localctx, 168, RULE_colPosition); - try { - setState(1591); - _errHandler.sync(this); - switch (_input.LA(1)) { - case FIRST: - enterOuterAlt(_localctx, 1); - { - setState(1588); - ((ColPositionContext)_localctx).position = match(FIRST); - } - break; - case AFTER: - enterOuterAlt(_localctx, 2); - { - setState(1589); - ((ColPositionContext)_localctx).position = match(AFTER); - setState(1590); - ((ColPositionContext)_localctx).afterCol = errorCapturingIdentifier(); - } - break; - default: - throw new NoViableAltException(this); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class DataTypeContext extends ParserRuleContext { - public DataTypeContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_dataType; } - - public DataTypeContext() { } - public void copyFrom(DataTypeContext ctx) { - super.copyFrom(ctx); - } - } - public static class ComplexDataTypeContext extends DataTypeContext { - public Token complex; - public TerminalNode LT() { return getToken(MixedFormatSqlExtendParser.LT, 0); } - public List dataType() { - return getRuleContexts(DataTypeContext.class); - } - public DataTypeContext dataType(int i) { - return getRuleContext(DataTypeContext.class,i); - } - public TerminalNode GT() { return getToken(MixedFormatSqlExtendParser.GT, 0); } - public TerminalNode ARRAY() { return getToken(MixedFormatSqlExtendParser.ARRAY, 0); } - public TerminalNode MAP() { return getToken(MixedFormatSqlExtendParser.MAP, 0); } - public TerminalNode STRUCT() { return getToken(MixedFormatSqlExtendParser.STRUCT, 0); } - public TerminalNode NEQ() { return getToken(MixedFormatSqlExtendParser.NEQ, 0); } - public ComplexColTypeListContext complexColTypeList() { - return getRuleContext(ComplexColTypeListContext.class,0); - } - public ComplexDataTypeContext(DataTypeContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterComplexDataType(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitComplexDataType(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitComplexDataType(this); - else return visitor.visitChildren(this); - } - } - public static class YearMonthIntervalDataTypeContext extends DataTypeContext { - public Token from; - public Token to; - public TerminalNode INTERVAL() { return getToken(MixedFormatSqlExtendParser.INTERVAL, 0); } - public TerminalNode YEAR() { return getToken(MixedFormatSqlExtendParser.YEAR, 0); } - public List MONTH() { return getTokens(MixedFormatSqlExtendParser.MONTH); } - public TerminalNode MONTH(int i) { - return getToken(MixedFormatSqlExtendParser.MONTH, i); - } - public TerminalNode TO() { return getToken(MixedFormatSqlExtendParser.TO, 0); } - public YearMonthIntervalDataTypeContext(DataTypeContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterYearMonthIntervalDataType(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitYearMonthIntervalDataType(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitYearMonthIntervalDataType(this); - else return visitor.visitChildren(this); - } - } - public static class DayTimeIntervalDataTypeContext extends DataTypeContext { - public Token from; - public Token to; - public TerminalNode INTERVAL() { return getToken(MixedFormatSqlExtendParser.INTERVAL, 0); } - public TerminalNode DAY() { return getToken(MixedFormatSqlExtendParser.DAY, 0); } - public List HOUR() { return getTokens(MixedFormatSqlExtendParser.HOUR); } - public TerminalNode HOUR(int i) { - return getToken(MixedFormatSqlExtendParser.HOUR, i); - } - public List MINUTE() { return getTokens(MixedFormatSqlExtendParser.MINUTE); } - public TerminalNode MINUTE(int i) { - return getToken(MixedFormatSqlExtendParser.MINUTE, i); - } - public List SECOND() { return getTokens(MixedFormatSqlExtendParser.SECOND); } - public TerminalNode SECOND(int i) { - return getToken(MixedFormatSqlExtendParser.SECOND, i); - } - public TerminalNode TO() { return getToken(MixedFormatSqlExtendParser.TO, 0); } - public DayTimeIntervalDataTypeContext(DataTypeContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterDayTimeIntervalDataType(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitDayTimeIntervalDataType(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitDayTimeIntervalDataType(this); - else return visitor.visitChildren(this); - } - } - public static class PrimitiveDataTypeContext extends DataTypeContext { - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public List INTEGER_VALUE() { return getTokens(MixedFormatSqlExtendParser.INTEGER_VALUE); } - public TerminalNode INTEGER_VALUE(int i) { - return getToken(MixedFormatSqlExtendParser.INTEGER_VALUE, i); - } - public PrimitiveDataTypeContext(DataTypeContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterPrimitiveDataType(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitPrimitiveDataType(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitPrimitiveDataType(this); - else return visitor.visitChildren(this); - } - } - - public final DataTypeContext dataType() throws RecognitionException { - DataTypeContext _localctx = new DataTypeContext(_ctx, getState()); - enterRule(_localctx, 170, RULE_dataType); - int _la; - try { - setState(1639); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,224,_ctx) ) { - case 1: - _localctx = new ComplexDataTypeContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(1593); - ((ComplexDataTypeContext)_localctx).complex = match(ARRAY); - setState(1594); - match(LT); - setState(1595); - dataType(); - setState(1596); - match(GT); - } - break; - case 2: - _localctx = new ComplexDataTypeContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(1598); - ((ComplexDataTypeContext)_localctx).complex = match(MAP); - setState(1599); - match(LT); - setState(1600); - dataType(); - setState(1601); - match(T__2); - setState(1602); - dataType(); - setState(1603); - match(GT); - } - break; - case 3: - _localctx = new ComplexDataTypeContext(_localctx); - enterOuterAlt(_localctx, 3); - { - setState(1605); - ((ComplexDataTypeContext)_localctx).complex = match(STRUCT); - setState(1612); - _errHandler.sync(this); - switch (_input.LA(1)) { - case LT: - { - setState(1606); - match(LT); - setState(1608); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,218,_ctx) ) { - case 1: - { - setState(1607); - complexColTypeList(); - } - break; - } - setState(1610); - match(GT); - } - break; - case NEQ: - { - setState(1611); - match(NEQ); - } - break; - default: - throw new NoViableAltException(this); - } - } - break; - case 4: - _localctx = new YearMonthIntervalDataTypeContext(_localctx); - enterOuterAlt(_localctx, 4); - { - setState(1614); - match(INTERVAL); - setState(1615); - ((YearMonthIntervalDataTypeContext)_localctx).from = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==MONTH || _la==YEAR) ) { - ((YearMonthIntervalDataTypeContext)_localctx).from = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1618); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,220,_ctx) ) { - case 1: - { - setState(1616); - match(TO); - setState(1617); - ((YearMonthIntervalDataTypeContext)_localctx).to = match(MONTH); - } - break; - } - } - break; - case 5: - _localctx = new DayTimeIntervalDataTypeContext(_localctx); - enterOuterAlt(_localctx, 5); - { - setState(1620); - match(INTERVAL); - setState(1621); - ((DayTimeIntervalDataTypeContext)_localctx).from = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==DAY || _la==HOUR || _la==MINUTE || _la==SECOND) ) { - ((DayTimeIntervalDataTypeContext)_localctx).from = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1624); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,221,_ctx) ) { - case 1: - { - setState(1622); - match(TO); - setState(1623); - ((DayTimeIntervalDataTypeContext)_localctx).to = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==HOUR || _la==MINUTE || _la==SECOND) ) { - ((DayTimeIntervalDataTypeContext)_localctx).to = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - break; - } - } - break; - case 6: - _localctx = new PrimitiveDataTypeContext(_localctx); - enterOuterAlt(_localctx, 6); - { - setState(1626); - identifier(); - setState(1637); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,223,_ctx) ) { - case 1: - { - setState(1627); - match(T__1); - setState(1628); - match(INTEGER_VALUE); - setState(1633); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1629); - match(T__2); - setState(1630); - match(INTEGER_VALUE); - } - } - setState(1635); - _errHandler.sync(this); - _la = _input.LA(1); - } - setState(1636); - match(T__3); - } - break; - } - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ColTypeListContext extends ParserRuleContext { - public List colType() { - return getRuleContexts(ColTypeContext.class); - } - public ColTypeContext colType(int i) { - return getRuleContext(ColTypeContext.class,i); - } - public ColTypeListContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_colTypeList; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterColTypeList(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitColTypeList(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitColTypeList(this); - else return visitor.visitChildren(this); - } - } - - public final ColTypeListContext colTypeList() throws RecognitionException { - ColTypeListContext _localctx = new ColTypeListContext(_ctx, getState()); - enterRule(_localctx, 172, RULE_colTypeList); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(1641); - colType(); - setState(1646); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,225,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(1642); - match(T__2); - setState(1643); - colType(); - } - } - } - setState(1648); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,225,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ColTypeContext extends ParserRuleContext { - public ErrorCapturingIdentifierContext colName; - public DataTypeContext dataType() { - return getRuleContext(DataTypeContext.class,0); - } - public ErrorCapturingIdentifierContext errorCapturingIdentifier() { - return getRuleContext(ErrorCapturingIdentifierContext.class,0); - } - public TerminalNode NOT() { return getToken(MixedFormatSqlExtendParser.NOT, 0); } - public TerminalNode NULL() { return getToken(MixedFormatSqlExtendParser.NULL, 0); } - public CommentSpecContext commentSpec() { - return getRuleContext(CommentSpecContext.class,0); - } - public ColTypeContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_colType; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterColType(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitColType(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitColType(this); - else return visitor.visitChildren(this); - } - } - - public final ColTypeContext colType() throws RecognitionException { - ColTypeContext _localctx = new ColTypeContext(_ctx, getState()); - enterRule(_localctx, 174, RULE_colType); - try { - enterOuterAlt(_localctx, 1); - { - setState(1649); - ((ColTypeContext)_localctx).colName = errorCapturingIdentifier(); - setState(1650); - dataType(); - setState(1653); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,226,_ctx) ) { - case 1: - { - setState(1651); - match(NOT); - setState(1652); - match(NULL); - } - break; - } - setState(1656); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,227,_ctx) ) { - case 1: - { - setState(1655); - commentSpec(); - } - break; - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ComplexColTypeListContext extends ParserRuleContext { - public List complexColType() { - return getRuleContexts(ComplexColTypeContext.class); - } - public ComplexColTypeContext complexColType(int i) { - return getRuleContext(ComplexColTypeContext.class,i); - } - public ComplexColTypeListContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_complexColTypeList; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterComplexColTypeList(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitComplexColTypeList(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitComplexColTypeList(this); - else return visitor.visitChildren(this); - } - } - - public final ComplexColTypeListContext complexColTypeList() throws RecognitionException { - ComplexColTypeListContext _localctx = new ComplexColTypeListContext(_ctx, getState()); - enterRule(_localctx, 176, RULE_complexColTypeList); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1658); - complexColType(); - setState(1663); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1659); - match(T__2); - setState(1660); - complexColType(); - } - } - setState(1665); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ComplexColTypeContext extends ParserRuleContext { - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public DataTypeContext dataType() { - return getRuleContext(DataTypeContext.class,0); - } - public TerminalNode NOT() { return getToken(MixedFormatSqlExtendParser.NOT, 0); } - public TerminalNode NULL() { return getToken(MixedFormatSqlExtendParser.NULL, 0); } - public CommentSpecContext commentSpec() { - return getRuleContext(CommentSpecContext.class,0); - } - public ComplexColTypeContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_complexColType; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterComplexColType(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitComplexColType(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitComplexColType(this); - else return visitor.visitChildren(this); - } - } - - public final ComplexColTypeContext complexColType() throws RecognitionException { - ComplexColTypeContext _localctx = new ComplexColTypeContext(_ctx, getState()); - enterRule(_localctx, 178, RULE_complexColType); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1666); - identifier(); - setState(1668); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,229,_ctx) ) { - case 1: - { - setState(1667); - match(T__10); - } - break; - } - setState(1670); - dataType(); - setState(1673); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==NOT) { - { - setState(1671); - match(NOT); - setState(1672); - match(NULL); - } - } - - setState(1676); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==COMMENT) { - { - setState(1675); - commentSpec(); - } - } - - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class WhenClauseContext extends ParserRuleContext { - public ExpressionContext condition; - public ExpressionContext result; - public TerminalNode WHEN() { return getToken(MixedFormatSqlExtendParser.WHEN, 0); } - public TerminalNode THEN() { return getToken(MixedFormatSqlExtendParser.THEN, 0); } - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public WhenClauseContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_whenClause; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterWhenClause(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitWhenClause(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitWhenClause(this); - else return visitor.visitChildren(this); - } - } - - public final WhenClauseContext whenClause() throws RecognitionException { - WhenClauseContext _localctx = new WhenClauseContext(_ctx, getState()); - enterRule(_localctx, 180, RULE_whenClause); - try { - enterOuterAlt(_localctx, 1); - { - setState(1678); - match(WHEN); - setState(1679); - ((WhenClauseContext)_localctx).condition = expression(); - setState(1680); - match(THEN); - setState(1681); - ((WhenClauseContext)_localctx).result = expression(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class WindowClauseContext extends ParserRuleContext { - public TerminalNode WINDOW() { return getToken(MixedFormatSqlExtendParser.WINDOW, 0); } - public List namedWindow() { - return getRuleContexts(NamedWindowContext.class); - } - public NamedWindowContext namedWindow(int i) { - return getRuleContext(NamedWindowContext.class,i); - } - public WindowClauseContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_windowClause; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterWindowClause(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitWindowClause(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitWindowClause(this); - else return visitor.visitChildren(this); - } - } - - public final WindowClauseContext windowClause() throws RecognitionException { - WindowClauseContext _localctx = new WindowClauseContext(_ctx, getState()); - enterRule(_localctx, 182, RULE_windowClause); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(1683); - match(WINDOW); - setState(1684); - namedWindow(); - setState(1689); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,232,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(1685); - match(T__2); - setState(1686); - namedWindow(); - } - } - } - setState(1691); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,232,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class NamedWindowContext extends ParserRuleContext { - public ErrorCapturingIdentifierContext name; - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public WindowSpecContext windowSpec() { - return getRuleContext(WindowSpecContext.class,0); - } - public ErrorCapturingIdentifierContext errorCapturingIdentifier() { - return getRuleContext(ErrorCapturingIdentifierContext.class,0); - } - public NamedWindowContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_namedWindow; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterNamedWindow(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitNamedWindow(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitNamedWindow(this); - else return visitor.visitChildren(this); - } - } - - public final NamedWindowContext namedWindow() throws RecognitionException { - NamedWindowContext _localctx = new NamedWindowContext(_ctx, getState()); - enterRule(_localctx, 184, RULE_namedWindow); - try { - enterOuterAlt(_localctx, 1); - { - setState(1692); - ((NamedWindowContext)_localctx).name = errorCapturingIdentifier(); - setState(1693); - match(AS); - setState(1694); - windowSpec(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class WindowSpecContext extends ParserRuleContext { - public WindowSpecContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_windowSpec; } - - public WindowSpecContext() { } - public void copyFrom(WindowSpecContext ctx) { - super.copyFrom(ctx); - } - } - public static class WindowRefContext extends WindowSpecContext { - public ErrorCapturingIdentifierContext name; - public ErrorCapturingIdentifierContext errorCapturingIdentifier() { - return getRuleContext(ErrorCapturingIdentifierContext.class,0); - } - public WindowRefContext(WindowSpecContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterWindowRef(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitWindowRef(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitWindowRef(this); - else return visitor.visitChildren(this); - } - } - public static class WindowDefContext extends WindowSpecContext { - public ExpressionContext expression; - public List partition = new ArrayList(); - public TerminalNode CLUSTER() { return getToken(MixedFormatSqlExtendParser.CLUSTER, 0); } - public List BY() { return getTokens(MixedFormatSqlExtendParser.BY); } - public TerminalNode BY(int i) { - return getToken(MixedFormatSqlExtendParser.BY, i); - } - public List expression() { - return getRuleContexts(ExpressionContext.class); - } - public ExpressionContext expression(int i) { - return getRuleContext(ExpressionContext.class,i); - } - public WindowFrameContext windowFrame() { - return getRuleContext(WindowFrameContext.class,0); - } - public List sortItem() { - return getRuleContexts(SortItemContext.class); - } - public SortItemContext sortItem(int i) { - return getRuleContext(SortItemContext.class,i); - } - public TerminalNode PARTITION() { return getToken(MixedFormatSqlExtendParser.PARTITION, 0); } - public TerminalNode DISTRIBUTE() { return getToken(MixedFormatSqlExtendParser.DISTRIBUTE, 0); } - public TerminalNode ORDER() { return getToken(MixedFormatSqlExtendParser.ORDER, 0); } - public TerminalNode SORT() { return getToken(MixedFormatSqlExtendParser.SORT, 0); } - public WindowDefContext(WindowSpecContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterWindowDef(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitWindowDef(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitWindowDef(this); - else return visitor.visitChildren(this); - } - } - - public final WindowSpecContext windowSpec() throws RecognitionException { - WindowSpecContext _localctx = new WindowSpecContext(_ctx, getState()); - enterRule(_localctx, 186, RULE_windowSpec); - int _la; - try { - setState(1742); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,240,_ctx) ) { - case 1: - _localctx = new WindowRefContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(1696); - ((WindowRefContext)_localctx).name = errorCapturingIdentifier(); - } - break; - case 2: - _localctx = new WindowRefContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(1697); - match(T__1); - setState(1698); - ((WindowRefContext)_localctx).name = errorCapturingIdentifier(); - setState(1699); - match(T__3); - } - break; - case 3: - _localctx = new WindowDefContext(_localctx); - enterOuterAlt(_localctx, 3); - { - setState(1701); - match(T__1); - setState(1736); - _errHandler.sync(this); - switch (_input.LA(1)) { - case CLUSTER: - { - setState(1702); - match(CLUSTER); - setState(1703); - match(BY); - setState(1704); - ((WindowDefContext)_localctx).expression = expression(); - ((WindowDefContext)_localctx).partition.add(((WindowDefContext)_localctx).expression); - setState(1709); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1705); - match(T__2); - setState(1706); - ((WindowDefContext)_localctx).expression = expression(); - ((WindowDefContext)_localctx).partition.add(((WindowDefContext)_localctx).expression); - } - } - setState(1711); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - break; - case T__3: - case DISTRIBUTE: - case ORDER: - case PARTITION: - case RANGE: - case ROWS: - case SORT: - { - setState(1722); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==DISTRIBUTE || _la==PARTITION) { - { - setState(1712); - _la = _input.LA(1); - if ( !(_la==DISTRIBUTE || _la==PARTITION) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1713); - match(BY); - setState(1714); - ((WindowDefContext)_localctx).expression = expression(); - ((WindowDefContext)_localctx).partition.add(((WindowDefContext)_localctx).expression); - setState(1719); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1715); - match(T__2); - setState(1716); - ((WindowDefContext)_localctx).expression = expression(); - ((WindowDefContext)_localctx).partition.add(((WindowDefContext)_localctx).expression); - } - } - setState(1721); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - } - - setState(1734); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==ORDER || _la==SORT) { - { - setState(1724); - _la = _input.LA(1); - if ( !(_la==ORDER || _la==SORT) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - setState(1725); - match(BY); - setState(1726); - sortItem(); - setState(1731); - _errHandler.sync(this); - _la = _input.LA(1); - while (_la==T__2) { - { - { - setState(1727); - match(T__2); - setState(1728); - sortItem(); - } - } - setState(1733); - _errHandler.sync(this); - _la = _input.LA(1); - } - } - } - - } - break; - default: - throw new NoViableAltException(this); - } - setState(1739); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==RANGE || _la==ROWS) { - { - setState(1738); - windowFrame(); - } - } - - setState(1741); - match(T__3); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class WindowFrameContext extends ParserRuleContext { - public Token frameType; - public FrameBoundContext start; - public FrameBoundContext end; - public TerminalNode RANGE() { return getToken(MixedFormatSqlExtendParser.RANGE, 0); } - public List frameBound() { - return getRuleContexts(FrameBoundContext.class); - } - public FrameBoundContext frameBound(int i) { - return getRuleContext(FrameBoundContext.class,i); - } - public TerminalNode ROWS() { return getToken(MixedFormatSqlExtendParser.ROWS, 0); } - public TerminalNode BETWEEN() { return getToken(MixedFormatSqlExtendParser.BETWEEN, 0); } - public TerminalNode AND() { return getToken(MixedFormatSqlExtendParser.AND, 0); } - public WindowFrameContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_windowFrame; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterWindowFrame(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitWindowFrame(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitWindowFrame(this); - else return visitor.visitChildren(this); - } - } - - public final WindowFrameContext windowFrame() throws RecognitionException { - WindowFrameContext _localctx = new WindowFrameContext(_ctx, getState()); - enterRule(_localctx, 188, RULE_windowFrame); - try { - setState(1760); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,241,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(1744); - ((WindowFrameContext)_localctx).frameType = match(RANGE); - setState(1745); - ((WindowFrameContext)_localctx).start = frameBound(); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(1746); - ((WindowFrameContext)_localctx).frameType = match(ROWS); - setState(1747); - ((WindowFrameContext)_localctx).start = frameBound(); - } - break; - case 3: - enterOuterAlt(_localctx, 3); - { - setState(1748); - ((WindowFrameContext)_localctx).frameType = match(RANGE); - setState(1749); - match(BETWEEN); - setState(1750); - ((WindowFrameContext)_localctx).start = frameBound(); - setState(1751); - match(AND); - setState(1752); - ((WindowFrameContext)_localctx).end = frameBound(); - } - break; - case 4: - enterOuterAlt(_localctx, 4); - { - setState(1754); - ((WindowFrameContext)_localctx).frameType = match(ROWS); - setState(1755); - match(BETWEEN); - setState(1756); - ((WindowFrameContext)_localctx).start = frameBound(); - setState(1757); - match(AND); - setState(1758); - ((WindowFrameContext)_localctx).end = frameBound(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class FrameBoundContext extends ParserRuleContext { - public Token boundType; - public TerminalNode UNBOUNDED() { return getToken(MixedFormatSqlExtendParser.UNBOUNDED, 0); } - public TerminalNode PRECEDING() { return getToken(MixedFormatSqlExtendParser.PRECEDING, 0); } - public TerminalNode FOLLOWING() { return getToken(MixedFormatSqlExtendParser.FOLLOWING, 0); } - public TerminalNode ROW() { return getToken(MixedFormatSqlExtendParser.ROW, 0); } - public TerminalNode CURRENT() { return getToken(MixedFormatSqlExtendParser.CURRENT, 0); } - public ExpressionContext expression() { - return getRuleContext(ExpressionContext.class,0); - } - public FrameBoundContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_frameBound; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterFrameBound(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitFrameBound(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitFrameBound(this); - else return visitor.visitChildren(this); - } - } - - public final FrameBoundContext frameBound() throws RecognitionException { - FrameBoundContext _localctx = new FrameBoundContext(_ctx, getState()); - enterRule(_localctx, 190, RULE_frameBound); - int _la; - try { - setState(1769); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,242,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(1762); - match(UNBOUNDED); - setState(1763); - ((FrameBoundContext)_localctx).boundType = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==FOLLOWING || _la==PRECEDING) ) { - ((FrameBoundContext)_localctx).boundType = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(1764); - ((FrameBoundContext)_localctx).boundType = match(CURRENT); - setState(1765); - match(ROW); - } - break; - case 3: - enterOuterAlt(_localctx, 3); - { - setState(1766); - expression(); - setState(1767); - ((FrameBoundContext)_localctx).boundType = _input.LT(1); - _la = _input.LA(1); - if ( !(_la==FOLLOWING || _la==PRECEDING) ) { - ((FrameBoundContext)_localctx).boundType = (Token)_errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class FunctionNameContext extends ParserRuleContext { - public QualifiedNameContext qualifiedName() { - return getRuleContext(QualifiedNameContext.class,0); - } - public TerminalNode FILTER() { return getToken(MixedFormatSqlExtendParser.FILTER, 0); } - public TerminalNode LEFT() { return getToken(MixedFormatSqlExtendParser.LEFT, 0); } - public TerminalNode RIGHT() { return getToken(MixedFormatSqlExtendParser.RIGHT, 0); } - public FunctionNameContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_functionName; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterFunctionName(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitFunctionName(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitFunctionName(this); - else return visitor.visitChildren(this); - } - } - - public final FunctionNameContext functionName() throws RecognitionException { - FunctionNameContext _localctx = new FunctionNameContext(_ctx, getState()); - enterRule(_localctx, 192, RULE_functionName); - try { - setState(1775); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,243,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(1771); - qualifiedName(); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(1772); - match(FILTER); - } - break; - case 3: - enterOuterAlt(_localctx, 3); - { - setState(1773); - match(LEFT); - } - break; - case 4: - enterOuterAlt(_localctx, 4); - { - setState(1774); - match(RIGHT); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class QualifiedNameContext extends ParserRuleContext { - public List identifier() { - return getRuleContexts(IdentifierContext.class); - } - public IdentifierContext identifier(int i) { - return getRuleContext(IdentifierContext.class,i); - } - public QualifiedNameContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_qualifiedName; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterQualifiedName(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitQualifiedName(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitQualifiedName(this); - else return visitor.visitChildren(this); - } - } - - public final QualifiedNameContext qualifiedName() throws RecognitionException { - QualifiedNameContext _localctx = new QualifiedNameContext(_ctx, getState()); - enterRule(_localctx, 194, RULE_qualifiedName); - try { - int _alt; - enterOuterAlt(_localctx, 1); - { - setState(1777); - identifier(); - setState(1782); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,244,_ctx); - while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { - if ( _alt==1 ) { - { - { - setState(1778); - match(T__4); - setState(1779); - identifier(); - } - } - } - setState(1784); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,244,_ctx); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ErrorCapturingIdentifierContext extends ParserRuleContext { - public IdentifierContext identifier() { - return getRuleContext(IdentifierContext.class,0); - } - public ErrorCapturingIdentifierExtraContext errorCapturingIdentifierExtra() { - return getRuleContext(ErrorCapturingIdentifierExtraContext.class,0); - } - public ErrorCapturingIdentifierContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_errorCapturingIdentifier; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterErrorCapturingIdentifier(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitErrorCapturingIdentifier(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitErrorCapturingIdentifier(this); - else return visitor.visitChildren(this); - } - } - - public final ErrorCapturingIdentifierContext errorCapturingIdentifier() throws RecognitionException { - ErrorCapturingIdentifierContext _localctx = new ErrorCapturingIdentifierContext(_ctx, getState()); - enterRule(_localctx, 196, RULE_errorCapturingIdentifier); - try { - enterOuterAlt(_localctx, 1); - { - setState(1785); - identifier(); - setState(1786); - errorCapturingIdentifierExtra(); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class ErrorCapturingIdentifierExtraContext extends ParserRuleContext { - public ErrorCapturingIdentifierExtraContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_errorCapturingIdentifierExtra; } - - public ErrorCapturingIdentifierExtraContext() { } - public void copyFrom(ErrorCapturingIdentifierExtraContext ctx) { - super.copyFrom(ctx); - } - } - public static class ErrorIdentContext extends ErrorCapturingIdentifierExtraContext { - public List MINUS() { return getTokens(MixedFormatSqlExtendParser.MINUS); } - public TerminalNode MINUS(int i) { - return getToken(MixedFormatSqlExtendParser.MINUS, i); - } - public List identifier() { - return getRuleContexts(IdentifierContext.class); - } - public IdentifierContext identifier(int i) { - return getRuleContext(IdentifierContext.class,i); - } - public ErrorIdentContext(ErrorCapturingIdentifierExtraContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterErrorIdent(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitErrorIdent(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitErrorIdent(this); - else return visitor.visitChildren(this); - } - } - public static class RealIdentContext extends ErrorCapturingIdentifierExtraContext { - public RealIdentContext(ErrorCapturingIdentifierExtraContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterRealIdent(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitRealIdent(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitRealIdent(this); - else return visitor.visitChildren(this); - } - } - - public final ErrorCapturingIdentifierExtraContext errorCapturingIdentifierExtra() throws RecognitionException { - ErrorCapturingIdentifierExtraContext _localctx = new ErrorCapturingIdentifierExtraContext(_ctx, getState()); - enterRule(_localctx, 198, RULE_errorCapturingIdentifierExtra); - try { - int _alt; - setState(1795); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,246,_ctx) ) { - case 1: - _localctx = new ErrorIdentContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(1790); - _errHandler.sync(this); - _alt = 1; - do { - switch (_alt) { - case 1: - { - { - setState(1788); - match(MINUS); - setState(1789); - identifier(); - } - } - break; - default: - throw new NoViableAltException(this); - } - setState(1792); - _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,245,_ctx); - } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); - } - break; - case 2: - _localctx = new RealIdentContext(_localctx); - enterOuterAlt(_localctx, 2); - { - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class IdentifierContext extends ParserRuleContext { - public StrictIdentifierContext strictIdentifier() { - return getRuleContext(StrictIdentifierContext.class,0); - } - public StrictNonReservedContext strictNonReserved() { - return getRuleContext(StrictNonReservedContext.class,0); - } - public IdentifierContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_identifier; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterIdentifier(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitIdentifier(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitIdentifier(this); - else return visitor.visitChildren(this); - } - } - - public final IdentifierContext identifier() throws RecognitionException { - IdentifierContext _localctx = new IdentifierContext(_ctx, getState()); - enterRule(_localctx, 200, RULE_identifier); - try { - setState(1800); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,247,_ctx) ) { - case 1: - enterOuterAlt(_localctx, 1); - { - setState(1797); - strictIdentifier(); - } - break; - case 2: - enterOuterAlt(_localctx, 2); - { - setState(1798); - if (!(!SQL_standard_keyword_behavior)) throw new FailedPredicateException(this, "!SQL_standard_keyword_behavior"); - setState(1799); - strictNonReserved(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class StrictIdentifierContext extends ParserRuleContext { - public StrictIdentifierContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_strictIdentifier; } - - public StrictIdentifierContext() { } - public void copyFrom(StrictIdentifierContext ctx) { - super.copyFrom(ctx); - } - } - public static class QuotedIdentifierAlternativeContext extends StrictIdentifierContext { - public QuotedIdentifierContext quotedIdentifier() { - return getRuleContext(QuotedIdentifierContext.class,0); - } - public QuotedIdentifierAlternativeContext(StrictIdentifierContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterQuotedIdentifierAlternative(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitQuotedIdentifierAlternative(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitQuotedIdentifierAlternative(this); - else return visitor.visitChildren(this); - } - } - public static class UnquotedIdentifierContext extends StrictIdentifierContext { - public TerminalNode IDENTIFIER() { return getToken(MixedFormatSqlExtendParser.IDENTIFIER, 0); } - public AnsiNonReservedContext ansiNonReserved() { - return getRuleContext(AnsiNonReservedContext.class,0); - } - public NonReservedContext nonReserved() { - return getRuleContext(NonReservedContext.class,0); - } - public UnquotedIdentifierContext(StrictIdentifierContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterUnquotedIdentifier(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitUnquotedIdentifier(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitUnquotedIdentifier(this); - else return visitor.visitChildren(this); - } - } - - public final StrictIdentifierContext strictIdentifier() throws RecognitionException { - StrictIdentifierContext _localctx = new StrictIdentifierContext(_ctx, getState()); - enterRule(_localctx, 202, RULE_strictIdentifier); - try { - setState(1808); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,248,_ctx) ) { - case 1: - _localctx = new UnquotedIdentifierContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(1802); - match(IDENTIFIER); - } - break; - case 2: - _localctx = new QuotedIdentifierAlternativeContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(1803); - quotedIdentifier(); - } - break; - case 3: - _localctx = new UnquotedIdentifierContext(_localctx); - enterOuterAlt(_localctx, 3); - { - setState(1804); - if (!(SQL_standard_keyword_behavior)) throw new FailedPredicateException(this, "SQL_standard_keyword_behavior"); - setState(1805); - ansiNonReserved(); - } - break; - case 4: - _localctx = new UnquotedIdentifierContext(_localctx); - enterOuterAlt(_localctx, 4); - { - setState(1806); - if (!(!SQL_standard_keyword_behavior)) throw new FailedPredicateException(this, "!SQL_standard_keyword_behavior"); - setState(1807); - nonReserved(); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class QuotedIdentifierContext extends ParserRuleContext { - public TerminalNode BACKQUOTED_IDENTIFIER() { return getToken(MixedFormatSqlExtendParser.BACKQUOTED_IDENTIFIER, 0); } - public QuotedIdentifierContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_quotedIdentifier; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterQuotedIdentifier(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitQuotedIdentifier(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitQuotedIdentifier(this); - else return visitor.visitChildren(this); - } - } - - public final QuotedIdentifierContext quotedIdentifier() throws RecognitionException { - QuotedIdentifierContext _localctx = new QuotedIdentifierContext(_ctx, getState()); - enterRule(_localctx, 204, RULE_quotedIdentifier); - try { - enterOuterAlt(_localctx, 1); - { - setState(1810); - match(BACKQUOTED_IDENTIFIER); - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class NumberContext extends ParserRuleContext { - public NumberContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_number; } - - public NumberContext() { } - public void copyFrom(NumberContext ctx) { - super.copyFrom(ctx); - } - } - public static class DecimalLiteralContext extends NumberContext { - public TerminalNode DECIMAL_VALUE() { return getToken(MixedFormatSqlExtendParser.DECIMAL_VALUE, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public DecimalLiteralContext(NumberContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterDecimalLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitDecimalLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitDecimalLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class BigIntLiteralContext extends NumberContext { - public TerminalNode BIGINT_LITERAL() { return getToken(MixedFormatSqlExtendParser.BIGINT_LITERAL, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public BigIntLiteralContext(NumberContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterBigIntLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitBigIntLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitBigIntLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class TinyIntLiteralContext extends NumberContext { - public TerminalNode TINYINT_LITERAL() { return getToken(MixedFormatSqlExtendParser.TINYINT_LITERAL, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public TinyIntLiteralContext(NumberContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterTinyIntLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitTinyIntLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitTinyIntLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class LegacyDecimalLiteralContext extends NumberContext { - public TerminalNode EXPONENT_VALUE() { return getToken(MixedFormatSqlExtendParser.EXPONENT_VALUE, 0); } - public TerminalNode DECIMAL_VALUE() { return getToken(MixedFormatSqlExtendParser.DECIMAL_VALUE, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public LegacyDecimalLiteralContext(NumberContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterLegacyDecimalLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitLegacyDecimalLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitLegacyDecimalLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class BigDecimalLiteralContext extends NumberContext { - public TerminalNode BIGDECIMAL_LITERAL() { return getToken(MixedFormatSqlExtendParser.BIGDECIMAL_LITERAL, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public BigDecimalLiteralContext(NumberContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterBigDecimalLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitBigDecimalLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitBigDecimalLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class ExponentLiteralContext extends NumberContext { - public TerminalNode EXPONENT_VALUE() { return getToken(MixedFormatSqlExtendParser.EXPONENT_VALUE, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public ExponentLiteralContext(NumberContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterExponentLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitExponentLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitExponentLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class DoubleLiteralContext extends NumberContext { - public TerminalNode DOUBLE_LITERAL() { return getToken(MixedFormatSqlExtendParser.DOUBLE_LITERAL, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public DoubleLiteralContext(NumberContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterDoubleLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitDoubleLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitDoubleLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class IntegerLiteralContext extends NumberContext { - public TerminalNode INTEGER_VALUE() { return getToken(MixedFormatSqlExtendParser.INTEGER_VALUE, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public IntegerLiteralContext(NumberContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterIntegerLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitIntegerLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitIntegerLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class FloatLiteralContext extends NumberContext { - public TerminalNode FLOAT_LITERAL() { return getToken(MixedFormatSqlExtendParser.FLOAT_LITERAL, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public FloatLiteralContext(NumberContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterFloatLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitFloatLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitFloatLiteral(this); - else return visitor.visitChildren(this); - } - } - public static class SmallIntLiteralContext extends NumberContext { - public TerminalNode SMALLINT_LITERAL() { return getToken(MixedFormatSqlExtendParser.SMALLINT_LITERAL, 0); } - public TerminalNode MINUS() { return getToken(MixedFormatSqlExtendParser.MINUS, 0); } - public SmallIntLiteralContext(NumberContext ctx) { copyFrom(ctx); } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterSmallIntLiteral(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitSmallIntLiteral(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitSmallIntLiteral(this); - else return visitor.visitChildren(this); - } - } - - public final NumberContext number() throws RecognitionException { - NumberContext _localctx = new NumberContext(_ctx, getState()); - enterRule(_localctx, 206, RULE_number); - int _la; - try { - setState(1855); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,259,_ctx) ) { - case 1: - _localctx = new ExponentLiteralContext(_localctx); - enterOuterAlt(_localctx, 1); - { - setState(1812); - if (!(!legacy_exponent_literal_as_decimal_enabled)) throw new FailedPredicateException(this, "!legacy_exponent_literal_as_decimal_enabled"); - setState(1814); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(1813); - match(MINUS); - } - } - - setState(1816); - match(EXPONENT_VALUE); - } - break; - case 2: - _localctx = new DecimalLiteralContext(_localctx); - enterOuterAlt(_localctx, 2); - { - setState(1817); - if (!(!legacy_exponent_literal_as_decimal_enabled)) throw new FailedPredicateException(this, "!legacy_exponent_literal_as_decimal_enabled"); - setState(1819); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(1818); - match(MINUS); - } - } - - setState(1821); - match(DECIMAL_VALUE); - } - break; - case 3: - _localctx = new LegacyDecimalLiteralContext(_localctx); - enterOuterAlt(_localctx, 3); - { - setState(1822); - if (!(legacy_exponent_literal_as_decimal_enabled)) throw new FailedPredicateException(this, "legacy_exponent_literal_as_decimal_enabled"); - setState(1824); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(1823); - match(MINUS); - } - } - - setState(1826); - _la = _input.LA(1); - if ( !(_la==EXPONENT_VALUE || _la==DECIMAL_VALUE) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - break; - case 4: - _localctx = new IntegerLiteralContext(_localctx); - enterOuterAlt(_localctx, 4); - { - setState(1828); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(1827); - match(MINUS); - } - } - - setState(1830); - match(INTEGER_VALUE); - } - break; - case 5: - _localctx = new BigIntLiteralContext(_localctx); - enterOuterAlt(_localctx, 5); - { - setState(1832); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(1831); - match(MINUS); - } - } - - setState(1834); - match(BIGINT_LITERAL); - } - break; - case 6: - _localctx = new SmallIntLiteralContext(_localctx); - enterOuterAlt(_localctx, 6); - { - setState(1836); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(1835); - match(MINUS); - } - } - - setState(1838); - match(SMALLINT_LITERAL); - } - break; - case 7: - _localctx = new TinyIntLiteralContext(_localctx); - enterOuterAlt(_localctx, 7); - { - setState(1840); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(1839); - match(MINUS); - } - } - - setState(1842); - match(TINYINT_LITERAL); - } - break; - case 8: - _localctx = new DoubleLiteralContext(_localctx); - enterOuterAlt(_localctx, 8); - { - setState(1844); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(1843); - match(MINUS); - } - } - - setState(1846); - match(DOUBLE_LITERAL); - } - break; - case 9: - _localctx = new FloatLiteralContext(_localctx); - enterOuterAlt(_localctx, 9); - { - setState(1848); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(1847); - match(MINUS); - } - } - - setState(1850); - match(FLOAT_LITERAL); - } - break; - case 10: - _localctx = new BigDecimalLiteralContext(_localctx); - enterOuterAlt(_localctx, 10); - { - setState(1852); - _errHandler.sync(this); - _la = _input.LA(1); - if (_la==MINUS) { - { - setState(1851); - match(MINUS); - } - } - - setState(1854); - match(BIGDECIMAL_LITERAL); - } - break; - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class AnsiNonReservedContext extends ParserRuleContext { - public TerminalNode ADD() { return getToken(MixedFormatSqlExtendParser.ADD, 0); } - public TerminalNode AFTER() { return getToken(MixedFormatSqlExtendParser.AFTER, 0); } - public TerminalNode ALTER() { return getToken(MixedFormatSqlExtendParser.ALTER, 0); } - public TerminalNode ANALYZE() { return getToken(MixedFormatSqlExtendParser.ANALYZE, 0); } - public TerminalNode ANTI() { return getToken(MixedFormatSqlExtendParser.ANTI, 0); } - public TerminalNode ARCHIVE() { return getToken(MixedFormatSqlExtendParser.ARCHIVE, 0); } - public TerminalNode ARRAY() { return getToken(MixedFormatSqlExtendParser.ARRAY, 0); } - public TerminalNode ASC() { return getToken(MixedFormatSqlExtendParser.ASC, 0); } - public TerminalNode AT() { return getToken(MixedFormatSqlExtendParser.AT, 0); } - public TerminalNode BETWEEN() { return getToken(MixedFormatSqlExtendParser.BETWEEN, 0); } - public TerminalNode BUCKET() { return getToken(MixedFormatSqlExtendParser.BUCKET, 0); } - public TerminalNode BUCKETS() { return getToken(MixedFormatSqlExtendParser.BUCKETS, 0); } - public TerminalNode BY() { return getToken(MixedFormatSqlExtendParser.BY, 0); } - public TerminalNode CACHE() { return getToken(MixedFormatSqlExtendParser.CACHE, 0); } - public TerminalNode CASCADE() { return getToken(MixedFormatSqlExtendParser.CASCADE, 0); } - public TerminalNode CHANGE() { return getToken(MixedFormatSqlExtendParser.CHANGE, 0); } - public TerminalNode CLEAR() { return getToken(MixedFormatSqlExtendParser.CLEAR, 0); } - public TerminalNode CLUSTER() { return getToken(MixedFormatSqlExtendParser.CLUSTER, 0); } - public TerminalNode CLUSTERED() { return getToken(MixedFormatSqlExtendParser.CLUSTERED, 0); } - public TerminalNode CODEGEN() { return getToken(MixedFormatSqlExtendParser.CODEGEN, 0); } - public TerminalNode COLLECTION() { return getToken(MixedFormatSqlExtendParser.COLLECTION, 0); } - public TerminalNode COLUMNS() { return getToken(MixedFormatSqlExtendParser.COLUMNS, 0); } - public TerminalNode COMMENT() { return getToken(MixedFormatSqlExtendParser.COMMENT, 0); } - public TerminalNode COMMIT() { return getToken(MixedFormatSqlExtendParser.COMMIT, 0); } - public TerminalNode COMPACT() { return getToken(MixedFormatSqlExtendParser.COMPACT, 0); } - public TerminalNode COMPACTIONS() { return getToken(MixedFormatSqlExtendParser.COMPACTIONS, 0); } - public TerminalNode COMPUTE() { return getToken(MixedFormatSqlExtendParser.COMPUTE, 0); } - public TerminalNode CONCATENATE() { return getToken(MixedFormatSqlExtendParser.CONCATENATE, 0); } - public TerminalNode COST() { return getToken(MixedFormatSqlExtendParser.COST, 0); } - public TerminalNode CUBE() { return getToken(MixedFormatSqlExtendParser.CUBE, 0); } - public TerminalNode CURRENT() { return getToken(MixedFormatSqlExtendParser.CURRENT, 0); } - public TerminalNode DATA() { return getToken(MixedFormatSqlExtendParser.DATA, 0); } - public TerminalNode DATABASE() { return getToken(MixedFormatSqlExtendParser.DATABASE, 0); } - public TerminalNode DATABASES() { return getToken(MixedFormatSqlExtendParser.DATABASES, 0); } - public TerminalNode DAY() { return getToken(MixedFormatSqlExtendParser.DAY, 0); } - public TerminalNode DBPROPERTIES() { return getToken(MixedFormatSqlExtendParser.DBPROPERTIES, 0); } - public TerminalNode DEFINED() { return getToken(MixedFormatSqlExtendParser.DEFINED, 0); } - public TerminalNode DELETE() { return getToken(MixedFormatSqlExtendParser.DELETE, 0); } - public TerminalNode DELIMITED() { return getToken(MixedFormatSqlExtendParser.DELIMITED, 0); } - public TerminalNode DESC() { return getToken(MixedFormatSqlExtendParser.DESC, 0); } - public TerminalNode DESCRIBE() { return getToken(MixedFormatSqlExtendParser.DESCRIBE, 0); } - public TerminalNode DFS() { return getToken(MixedFormatSqlExtendParser.DFS, 0); } - public TerminalNode DIRECTORIES() { return getToken(MixedFormatSqlExtendParser.DIRECTORIES, 0); } - public TerminalNode DIRECTORY() { return getToken(MixedFormatSqlExtendParser.DIRECTORY, 0); } - public TerminalNode DISTRIBUTE() { return getToken(MixedFormatSqlExtendParser.DISTRIBUTE, 0); } - public TerminalNode DIV() { return getToken(MixedFormatSqlExtendParser.DIV, 0); } - public TerminalNode DROP() { return getToken(MixedFormatSqlExtendParser.DROP, 0); } - public TerminalNode ESCAPED() { return getToken(MixedFormatSqlExtendParser.ESCAPED, 0); } - public TerminalNode EXCHANGE() { return getToken(MixedFormatSqlExtendParser.EXCHANGE, 0); } - public TerminalNode EXISTS() { return getToken(MixedFormatSqlExtendParser.EXISTS, 0); } - public TerminalNode EXPLAIN() { return getToken(MixedFormatSqlExtendParser.EXPLAIN, 0); } - public TerminalNode EXPORT() { return getToken(MixedFormatSqlExtendParser.EXPORT, 0); } - public TerminalNode EXTENDED() { return getToken(MixedFormatSqlExtendParser.EXTENDED, 0); } - public TerminalNode EXTERNAL() { return getToken(MixedFormatSqlExtendParser.EXTERNAL, 0); } - public TerminalNode EXTRACT() { return getToken(MixedFormatSqlExtendParser.EXTRACT, 0); } - public TerminalNode FIELDS() { return getToken(MixedFormatSqlExtendParser.FIELDS, 0); } - public TerminalNode FILEFORMAT() { return getToken(MixedFormatSqlExtendParser.FILEFORMAT, 0); } - public TerminalNode FIRST() { return getToken(MixedFormatSqlExtendParser.FIRST, 0); } - public TerminalNode FOLLOWING() { return getToken(MixedFormatSqlExtendParser.FOLLOWING, 0); } - public TerminalNode FORMAT() { return getToken(MixedFormatSqlExtendParser.FORMAT, 0); } - public TerminalNode FORMATTED() { return getToken(MixedFormatSqlExtendParser.FORMATTED, 0); } - public TerminalNode FUNCTION() { return getToken(MixedFormatSqlExtendParser.FUNCTION, 0); } - public TerminalNode FUNCTIONS() { return getToken(MixedFormatSqlExtendParser.FUNCTIONS, 0); } - public TerminalNode GLOBAL() { return getToken(MixedFormatSqlExtendParser.GLOBAL, 0); } - public TerminalNode GROUPING() { return getToken(MixedFormatSqlExtendParser.GROUPING, 0); } - public TerminalNode HOUR() { return getToken(MixedFormatSqlExtendParser.HOUR, 0); } - public TerminalNode IF() { return getToken(MixedFormatSqlExtendParser.IF, 0); } - public TerminalNode IGNORE() { return getToken(MixedFormatSqlExtendParser.IGNORE, 0); } - public TerminalNode IMPORT() { return getToken(MixedFormatSqlExtendParser.IMPORT, 0); } - public TerminalNode INDEX() { return getToken(MixedFormatSqlExtendParser.INDEX, 0); } - public TerminalNode INDEXES() { return getToken(MixedFormatSqlExtendParser.INDEXES, 0); } - public TerminalNode INPATH() { return getToken(MixedFormatSqlExtendParser.INPATH, 0); } - public TerminalNode INPUTFORMAT() { return getToken(MixedFormatSqlExtendParser.INPUTFORMAT, 0); } - public TerminalNode INSERT() { return getToken(MixedFormatSqlExtendParser.INSERT, 0); } - public TerminalNode INTERVAL() { return getToken(MixedFormatSqlExtendParser.INTERVAL, 0); } - public TerminalNode ITEMS() { return getToken(MixedFormatSqlExtendParser.ITEMS, 0); } - public TerminalNode KEYS() { return getToken(MixedFormatSqlExtendParser.KEYS, 0); } - public TerminalNode LAST() { return getToken(MixedFormatSqlExtendParser.LAST, 0); } - public TerminalNode LAZY() { return getToken(MixedFormatSqlExtendParser.LAZY, 0); } - public TerminalNode LIKE() { return getToken(MixedFormatSqlExtendParser.LIKE, 0); } - public TerminalNode LIMIT() { return getToken(MixedFormatSqlExtendParser.LIMIT, 0); } - public TerminalNode LINES() { return getToken(MixedFormatSqlExtendParser.LINES, 0); } - public TerminalNode LIST() { return getToken(MixedFormatSqlExtendParser.LIST, 0); } - public TerminalNode LOAD() { return getToken(MixedFormatSqlExtendParser.LOAD, 0); } - public TerminalNode LOCAL() { return getToken(MixedFormatSqlExtendParser.LOCAL, 0); } - public TerminalNode LOCATION() { return getToken(MixedFormatSqlExtendParser.LOCATION, 0); } - public TerminalNode LOCK() { return getToken(MixedFormatSqlExtendParser.LOCK, 0); } - public TerminalNode LOCKS() { return getToken(MixedFormatSqlExtendParser.LOCKS, 0); } - public TerminalNode LOGICAL() { return getToken(MixedFormatSqlExtendParser.LOGICAL, 0); } - public TerminalNode MACRO() { return getToken(MixedFormatSqlExtendParser.MACRO, 0); } - public TerminalNode MAP() { return getToken(MixedFormatSqlExtendParser.MAP, 0); } - public TerminalNode MATCHED() { return getToken(MixedFormatSqlExtendParser.MATCHED, 0); } - public TerminalNode MERGE() { return getToken(MixedFormatSqlExtendParser.MERGE, 0); } - public TerminalNode MINUTE() { return getToken(MixedFormatSqlExtendParser.MINUTE, 0); } - public TerminalNode MONTH() { return getToken(MixedFormatSqlExtendParser.MONTH, 0); } - public TerminalNode MSCK() { return getToken(MixedFormatSqlExtendParser.MSCK, 0); } - public TerminalNode NAMESPACE() { return getToken(MixedFormatSqlExtendParser.NAMESPACE, 0); } - public TerminalNode NAMESPACES() { return getToken(MixedFormatSqlExtendParser.NAMESPACES, 0); } - public TerminalNode NO() { return getToken(MixedFormatSqlExtendParser.NO, 0); } - public TerminalNode NULLS() { return getToken(MixedFormatSqlExtendParser.NULLS, 0); } - public TerminalNode OF() { return getToken(MixedFormatSqlExtendParser.OF, 0); } - public TerminalNode OPTION() { return getToken(MixedFormatSqlExtendParser.OPTION, 0); } - public TerminalNode OPTIONS() { return getToken(MixedFormatSqlExtendParser.OPTIONS, 0); } - public TerminalNode OUT() { return getToken(MixedFormatSqlExtendParser.OUT, 0); } - public TerminalNode OUTPUTFORMAT() { return getToken(MixedFormatSqlExtendParser.OUTPUTFORMAT, 0); } - public TerminalNode OVER() { return getToken(MixedFormatSqlExtendParser.OVER, 0); } - public TerminalNode OVERLAY() { return getToken(MixedFormatSqlExtendParser.OVERLAY, 0); } - public TerminalNode OVERWRITE() { return getToken(MixedFormatSqlExtendParser.OVERWRITE, 0); } - public TerminalNode PARTITION() { return getToken(MixedFormatSqlExtendParser.PARTITION, 0); } - public TerminalNode PARTITIONED() { return getToken(MixedFormatSqlExtendParser.PARTITIONED, 0); } - public TerminalNode PARTITIONS() { return getToken(MixedFormatSqlExtendParser.PARTITIONS, 0); } - public TerminalNode PERCENTLIT() { return getToken(MixedFormatSqlExtendParser.PERCENTLIT, 0); } - public TerminalNode PIVOT() { return getToken(MixedFormatSqlExtendParser.PIVOT, 0); } - public TerminalNode PLACING() { return getToken(MixedFormatSqlExtendParser.PLACING, 0); } - public TerminalNode POSITION() { return getToken(MixedFormatSqlExtendParser.POSITION, 0); } - public TerminalNode PRECEDING() { return getToken(MixedFormatSqlExtendParser.PRECEDING, 0); } - public TerminalNode PRINCIPALS() { return getToken(MixedFormatSqlExtendParser.PRINCIPALS, 0); } - public TerminalNode PROPERTIES() { return getToken(MixedFormatSqlExtendParser.PROPERTIES, 0); } - public TerminalNode PURGE() { return getToken(MixedFormatSqlExtendParser.PURGE, 0); } - public TerminalNode QUERY() { return getToken(MixedFormatSqlExtendParser.QUERY, 0); } - public TerminalNode RANGE() { return getToken(MixedFormatSqlExtendParser.RANGE, 0); } - public TerminalNode RECORDREADER() { return getToken(MixedFormatSqlExtendParser.RECORDREADER, 0); } - public TerminalNode RECORDWRITER() { return getToken(MixedFormatSqlExtendParser.RECORDWRITER, 0); } - public TerminalNode RECOVER() { return getToken(MixedFormatSqlExtendParser.RECOVER, 0); } - public TerminalNode REDUCE() { return getToken(MixedFormatSqlExtendParser.REDUCE, 0); } - public TerminalNode REFRESH() { return getToken(MixedFormatSqlExtendParser.REFRESH, 0); } - public TerminalNode RENAME() { return getToken(MixedFormatSqlExtendParser.RENAME, 0); } - public TerminalNode REPAIR() { return getToken(MixedFormatSqlExtendParser.REPAIR, 0); } - public TerminalNode REPLACE() { return getToken(MixedFormatSqlExtendParser.REPLACE, 0); } - public TerminalNode RESET() { return getToken(MixedFormatSqlExtendParser.RESET, 0); } - public TerminalNode RESPECT() { return getToken(MixedFormatSqlExtendParser.RESPECT, 0); } - public TerminalNode RESTRICT() { return getToken(MixedFormatSqlExtendParser.RESTRICT, 0); } - public TerminalNode REVOKE() { return getToken(MixedFormatSqlExtendParser.REVOKE, 0); } - public TerminalNode RLIKE() { return getToken(MixedFormatSqlExtendParser.RLIKE, 0); } - public TerminalNode ROLE() { return getToken(MixedFormatSqlExtendParser.ROLE, 0); } - public TerminalNode ROLES() { return getToken(MixedFormatSqlExtendParser.ROLES, 0); } - public TerminalNode ROLLBACK() { return getToken(MixedFormatSqlExtendParser.ROLLBACK, 0); } - public TerminalNode ROLLUP() { return getToken(MixedFormatSqlExtendParser.ROLLUP, 0); } - public TerminalNode ROW() { return getToken(MixedFormatSqlExtendParser.ROW, 0); } - public TerminalNode ROWS() { return getToken(MixedFormatSqlExtendParser.ROWS, 0); } - public TerminalNode SCHEMA() { return getToken(MixedFormatSqlExtendParser.SCHEMA, 0); } - public TerminalNode SECOND() { return getToken(MixedFormatSqlExtendParser.SECOND, 0); } - public TerminalNode SEMI() { return getToken(MixedFormatSqlExtendParser.SEMI, 0); } - public TerminalNode SEPARATED() { return getToken(MixedFormatSqlExtendParser.SEPARATED, 0); } - public TerminalNode SERDE() { return getToken(MixedFormatSqlExtendParser.SERDE, 0); } - public TerminalNode SERDEPROPERTIES() { return getToken(MixedFormatSqlExtendParser.SERDEPROPERTIES, 0); } - public TerminalNode SET() { return getToken(MixedFormatSqlExtendParser.SET, 0); } - public TerminalNode SETMINUS() { return getToken(MixedFormatSqlExtendParser.SETMINUS, 0); } - public TerminalNode SETS() { return getToken(MixedFormatSqlExtendParser.SETS, 0); } - public TerminalNode SHOW() { return getToken(MixedFormatSqlExtendParser.SHOW, 0); } - public TerminalNode SKEWED() { return getToken(MixedFormatSqlExtendParser.SKEWED, 0); } - public TerminalNode SORT() { return getToken(MixedFormatSqlExtendParser.SORT, 0); } - public TerminalNode SORTED() { return getToken(MixedFormatSqlExtendParser.SORTED, 0); } - public TerminalNode START() { return getToken(MixedFormatSqlExtendParser.START, 0); } - public TerminalNode STATISTICS() { return getToken(MixedFormatSqlExtendParser.STATISTICS, 0); } - public TerminalNode STORED() { return getToken(MixedFormatSqlExtendParser.STORED, 0); } - public TerminalNode STRATIFY() { return getToken(MixedFormatSqlExtendParser.STRATIFY, 0); } - public TerminalNode STRUCT() { return getToken(MixedFormatSqlExtendParser.STRUCT, 0); } - public TerminalNode SUBSTR() { return getToken(MixedFormatSqlExtendParser.SUBSTR, 0); } - public TerminalNode SUBSTRING() { return getToken(MixedFormatSqlExtendParser.SUBSTRING, 0); } - public TerminalNode SYNC() { return getToken(MixedFormatSqlExtendParser.SYNC, 0); } - public TerminalNode TABLES() { return getToken(MixedFormatSqlExtendParser.TABLES, 0); } - public TerminalNode TABLESAMPLE() { return getToken(MixedFormatSqlExtendParser.TABLESAMPLE, 0); } - public TerminalNode TBLPROPERTIES() { return getToken(MixedFormatSqlExtendParser.TBLPROPERTIES, 0); } - public TerminalNode TEMPORARY() { return getToken(MixedFormatSqlExtendParser.TEMPORARY, 0); } - public TerminalNode TERMINATED() { return getToken(MixedFormatSqlExtendParser.TERMINATED, 0); } - public TerminalNode TOUCH() { return getToken(MixedFormatSqlExtendParser.TOUCH, 0); } - public TerminalNode TRANSACTION() { return getToken(MixedFormatSqlExtendParser.TRANSACTION, 0); } - public TerminalNode TRANSACTIONS() { return getToken(MixedFormatSqlExtendParser.TRANSACTIONS, 0); } - public TerminalNode TRANSFORM() { return getToken(MixedFormatSqlExtendParser.TRANSFORM, 0); } - public TerminalNode TRIM() { return getToken(MixedFormatSqlExtendParser.TRIM, 0); } - public TerminalNode TRUE() { return getToken(MixedFormatSqlExtendParser.TRUE, 0); } - public TerminalNode TRUNCATE() { return getToken(MixedFormatSqlExtendParser.TRUNCATE, 0); } - public TerminalNode TRY_CAST() { return getToken(MixedFormatSqlExtendParser.TRY_CAST, 0); } - public TerminalNode TYPE() { return getToken(MixedFormatSqlExtendParser.TYPE, 0); } - public TerminalNode UNARCHIVE() { return getToken(MixedFormatSqlExtendParser.UNARCHIVE, 0); } - public TerminalNode UNBOUNDED() { return getToken(MixedFormatSqlExtendParser.UNBOUNDED, 0); } - public TerminalNode UNCACHE() { return getToken(MixedFormatSqlExtendParser.UNCACHE, 0); } - public TerminalNode UNLOCK() { return getToken(MixedFormatSqlExtendParser.UNLOCK, 0); } - public TerminalNode UNSET() { return getToken(MixedFormatSqlExtendParser.UNSET, 0); } - public TerminalNode UPDATE() { return getToken(MixedFormatSqlExtendParser.UPDATE, 0); } - public TerminalNode USE() { return getToken(MixedFormatSqlExtendParser.USE, 0); } - public TerminalNode VALUES() { return getToken(MixedFormatSqlExtendParser.VALUES, 0); } - public TerminalNode VIEW() { return getToken(MixedFormatSqlExtendParser.VIEW, 0); } - public TerminalNode VIEWS() { return getToken(MixedFormatSqlExtendParser.VIEWS, 0); } - public TerminalNode WINDOW() { return getToken(MixedFormatSqlExtendParser.WINDOW, 0); } - public TerminalNode YEAR() { return getToken(MixedFormatSqlExtendParser.YEAR, 0); } - public TerminalNode ZONE() { return getToken(MixedFormatSqlExtendParser.ZONE, 0); } - public AnsiNonReservedContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_ansiNonReserved; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterAnsiNonReserved(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitAnsiNonReserved(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitAnsiNonReserved(this); - else return visitor.visitChildren(this); - } - } - - public final AnsiNonReservedContext ansiNonReserved() throws RecognitionException { - AnsiNonReservedContext _localctx = new AnsiNonReservedContext(_ctx, getState()); - enterRule(_localctx, 208, RULE_ansiNonReserved); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1857); - _la = _input.LA(1); - if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ADD) | (1L << AFTER) | (1L << ALTER) | (1L << ANALYZE) | (1L << ANTI) | (1L << ARCHIVE) | (1L << ARRAY) | (1L << ASC) | (1L << AT) | (1L << BETWEEN) | (1L << BUCKET) | (1L << BUCKETS) | (1L << BY) | (1L << CACHE) | (1L << CASCADE) | (1L << CHANGE) | (1L << CLEAR) | (1L << CLUSTER) | (1L << CLUSTERED) | (1L << CODEGEN) | (1L << COLLECTION) | (1L << COLUMNS) | (1L << COMMENT) | (1L << COMMIT) | (1L << COMPACT) | (1L << COMPACTIONS) | (1L << COMPUTE) | (1L << CONCATENATE) | (1L << COST) | (1L << CUBE) | (1L << CURRENT) | (1L << DAY) | (1L << DATA) | (1L << DATABASE))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (DATABASES - 64)) | (1L << (DBPROPERTIES - 64)) | (1L << (DEFINED - 64)) | (1L << (DELETE - 64)) | (1L << (DELIMITED - 64)) | (1L << (DESC - 64)) | (1L << (DESCRIBE - 64)) | (1L << (DFS - 64)) | (1L << (DIRECTORIES - 64)) | (1L << (DIRECTORY - 64)) | (1L << (DISTRIBUTE - 64)) | (1L << (DIV - 64)) | (1L << (DROP - 64)) | (1L << (ESCAPED - 64)) | (1L << (EXCHANGE - 64)) | (1L << (EXISTS - 64)) | (1L << (EXPLAIN - 64)) | (1L << (EXPORT - 64)) | (1L << (EXTENDED - 64)) | (1L << (EXTERNAL - 64)) | (1L << (EXTRACT - 64)) | (1L << (FIELDS - 64)) | (1L << (FILEFORMAT - 64)) | (1L << (FIRST - 64)) | (1L << (FOLLOWING - 64)) | (1L << (FORMAT - 64)) | (1L << (FORMATTED - 64)) | (1L << (FUNCTION - 64)) | (1L << (FUNCTIONS - 64)) | (1L << (GLOBAL - 64)) | (1L << (GROUPING - 64)) | (1L << (HOUR - 64)) | (1L << (IF - 64)) | (1L << (IGNORE - 64)) | (1L << (IMPORT - 64)) | (1L << (INDEX - 64)) | (1L << (INDEXES - 64)) | (1L << (INPATH - 64)) | (1L << (INPUTFORMAT - 64)) | (1L << (INSERT - 64)) | (1L << (INTERVAL - 64)) | (1L << (ITEMS - 64)) | (1L << (KEYS - 64)))) != 0) || ((((_la - 128)) & ~0x3f) == 0 && ((1L << (_la - 128)) & ((1L << (LAST - 128)) | (1L << (LAZY - 128)) | (1L << (LIKE - 128)) | (1L << (LIMIT - 128)) | (1L << (LINES - 128)) | (1L << (LIST - 128)) | (1L << (LOAD - 128)) | (1L << (LOCAL - 128)) | (1L << (LOCATION - 128)) | (1L << (LOCK - 128)) | (1L << (LOCKS - 128)) | (1L << (LOGICAL - 128)) | (1L << (MACRO - 128)) | (1L << (MAP - 128)) | (1L << (MATCHED - 128)) | (1L << (MERGE - 128)) | (1L << (MINUTE - 128)) | (1L << (MONTH - 128)) | (1L << (MSCK - 128)) | (1L << (NAMESPACE - 128)) | (1L << (NAMESPACES - 128)) | (1L << (NO - 128)) | (1L << (NULLS - 128)) | (1L << (OF - 128)) | (1L << (OPTION - 128)) | (1L << (OPTIONS - 128)) | (1L << (OUT - 128)) | (1L << (OUTPUTFORMAT - 128)) | (1L << (OVER - 128)) | (1L << (OVERLAY - 128)) | (1L << (OVERWRITE - 128)) | (1L << (PARTITION - 128)) | (1L << (PARTITIONED - 128)) | (1L << (PARTITIONS - 128)) | (1L << (PERCENTLIT - 128)) | (1L << (PIVOT - 128)) | (1L << (PLACING - 128)) | (1L << (POSITION - 128)) | (1L << (PRECEDING - 128)) | (1L << (PRINCIPALS - 128)) | (1L << (PROPERTIES - 128)) | (1L << (PURGE - 128)) | (1L << (QUERY - 128)) | (1L << (RANGE - 128)) | (1L << (RECORDREADER - 128)) | (1L << (RECORDWRITER - 128)) | (1L << (RECOVER - 128)) | (1L << (REDUCE - 128)) | (1L << (REFRESH - 128)) | (1L << (RENAME - 128)))) != 0) || ((((_la - 192)) & ~0x3f) == 0 && ((1L << (_la - 192)) & ((1L << (REPAIR - 192)) | (1L << (REPLACE - 192)) | (1L << (RESET - 192)) | (1L << (RESPECT - 192)) | (1L << (RESTRICT - 192)) | (1L << (REVOKE - 192)) | (1L << (RLIKE - 192)) | (1L << (ROLE - 192)) | (1L << (ROLES - 192)) | (1L << (ROLLBACK - 192)) | (1L << (ROLLUP - 192)) | (1L << (ROW - 192)) | (1L << (ROWS - 192)) | (1L << (SECOND - 192)) | (1L << (SCHEMA - 192)) | (1L << (SEMI - 192)) | (1L << (SEPARATED - 192)) | (1L << (SERDE - 192)) | (1L << (SERDEPROPERTIES - 192)) | (1L << (SET - 192)) | (1L << (SETMINUS - 192)) | (1L << (SETS - 192)) | (1L << (SHOW - 192)) | (1L << (SKEWED - 192)) | (1L << (SORT - 192)) | (1L << (SORTED - 192)) | (1L << (START - 192)) | (1L << (STATISTICS - 192)) | (1L << (STORED - 192)) | (1L << (STRATIFY - 192)) | (1L << (STRUCT - 192)) | (1L << (SUBSTR - 192)) | (1L << (SUBSTRING - 192)) | (1L << (SYNC - 192)) | (1L << (TABLES - 192)) | (1L << (TABLESAMPLE - 192)) | (1L << (TBLPROPERTIES - 192)) | (1L << (TEMPORARY - 192)) | (1L << (TERMINATED - 192)) | (1L << (TOUCH - 192)) | (1L << (TRANSACTION - 192)) | (1L << (TRANSACTIONS - 192)) | (1L << (TRANSFORM - 192)) | (1L << (TRIM - 192)) | (1L << (TRUE - 192)) | (1L << (TRUNCATE - 192)) | (1L << (TRY_CAST - 192)) | (1L << (TYPE - 192)) | (1L << (UNARCHIVE - 192)) | (1L << (UNBOUNDED - 192)) | (1L << (UNCACHE - 192)) | (1L << (UNLOCK - 192)))) != 0) || ((((_la - 256)) & ~0x3f) == 0 && ((1L << (_la - 256)) & ((1L << (UNSET - 256)) | (1L << (UPDATE - 256)) | (1L << (USE - 256)) | (1L << (VALUES - 256)) | (1L << (VIEW - 256)) | (1L << (VIEWS - 256)) | (1L << (WINDOW - 256)) | (1L << (YEAR - 256)) | (1L << (ZONE - 256)))) != 0)) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class StrictNonReservedContext extends ParserRuleContext { - public TerminalNode ANTI() { return getToken(MixedFormatSqlExtendParser.ANTI, 0); } - public TerminalNode CROSS() { return getToken(MixedFormatSqlExtendParser.CROSS, 0); } - public TerminalNode EXCEPT() { return getToken(MixedFormatSqlExtendParser.EXCEPT, 0); } - public TerminalNode FULL() { return getToken(MixedFormatSqlExtendParser.FULL, 0); } - public TerminalNode INNER() { return getToken(MixedFormatSqlExtendParser.INNER, 0); } - public TerminalNode INTERSECT() { return getToken(MixedFormatSqlExtendParser.INTERSECT, 0); } - public TerminalNode JOIN() { return getToken(MixedFormatSqlExtendParser.JOIN, 0); } - public TerminalNode LATERAL() { return getToken(MixedFormatSqlExtendParser.LATERAL, 0); } - public TerminalNode LEFT() { return getToken(MixedFormatSqlExtendParser.LEFT, 0); } - public TerminalNode NATURAL() { return getToken(MixedFormatSqlExtendParser.NATURAL, 0); } - public TerminalNode ON() { return getToken(MixedFormatSqlExtendParser.ON, 0); } - public TerminalNode RIGHT() { return getToken(MixedFormatSqlExtendParser.RIGHT, 0); } - public TerminalNode SEMI() { return getToken(MixedFormatSqlExtendParser.SEMI, 0); } - public TerminalNode SETMINUS() { return getToken(MixedFormatSqlExtendParser.SETMINUS, 0); } - public TerminalNode UNION() { return getToken(MixedFormatSqlExtendParser.UNION, 0); } - public TerminalNode USING() { return getToken(MixedFormatSqlExtendParser.USING, 0); } - public StrictNonReservedContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_strictNonReserved; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterStrictNonReserved(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitStrictNonReserved(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitStrictNonReserved(this); - else return visitor.visitChildren(this); - } - } - - public final StrictNonReservedContext strictNonReserved() throws RecognitionException { - StrictNonReservedContext _localctx = new StrictNonReservedContext(_ctx, getState()); - enterRule(_localctx, 210, RULE_strictNonReserved); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1859); - _la = _input.LA(1); - if ( !(_la==ANTI || _la==CROSS || ((((_la - 82)) & ~0x3f) == 0 && ((1L << (_la - 82)) & ((1L << (EXCEPT - 82)) | (1L << (FULL - 82)) | (1L << (INNER - 82)) | (1L << (INTERSECT - 82)) | (1L << (JOIN - 82)) | (1L << (LATERAL - 82)) | (1L << (LEFT - 82)))) != 0) || ((((_la - 152)) & ~0x3f) == 0 && ((1L << (_la - 152)) & ((1L << (NATURAL - 152)) | (1L << (ON - 152)) | (1L << (RIGHT - 152)) | (1L << (SEMI - 152)) | (1L << (SETMINUS - 152)))) != 0) || _la==UNION || _la==USING) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public static class NonReservedContext extends ParserRuleContext { - public TerminalNode ADD() { return getToken(MixedFormatSqlExtendParser.ADD, 0); } - public TerminalNode AFTER() { return getToken(MixedFormatSqlExtendParser.AFTER, 0); } - public TerminalNode ALL() { return getToken(MixedFormatSqlExtendParser.ALL, 0); } - public TerminalNode ALTER() { return getToken(MixedFormatSqlExtendParser.ALTER, 0); } - public TerminalNode ANALYZE() { return getToken(MixedFormatSqlExtendParser.ANALYZE, 0); } - public TerminalNode AND() { return getToken(MixedFormatSqlExtendParser.AND, 0); } - public TerminalNode ANY() { return getToken(MixedFormatSqlExtendParser.ANY, 0); } - public TerminalNode ARCHIVE() { return getToken(MixedFormatSqlExtendParser.ARCHIVE, 0); } - public TerminalNode ARRAY() { return getToken(MixedFormatSqlExtendParser.ARRAY, 0); } - public TerminalNode AS() { return getToken(MixedFormatSqlExtendParser.AS, 0); } - public TerminalNode ASC() { return getToken(MixedFormatSqlExtendParser.ASC, 0); } - public TerminalNode AT() { return getToken(MixedFormatSqlExtendParser.AT, 0); } - public TerminalNode AUTHORIZATION() { return getToken(MixedFormatSqlExtendParser.AUTHORIZATION, 0); } - public TerminalNode BETWEEN() { return getToken(MixedFormatSqlExtendParser.BETWEEN, 0); } - public TerminalNode BOTH() { return getToken(MixedFormatSqlExtendParser.BOTH, 0); } - public TerminalNode BUCKET() { return getToken(MixedFormatSqlExtendParser.BUCKET, 0); } - public TerminalNode BUCKETS() { return getToken(MixedFormatSqlExtendParser.BUCKETS, 0); } - public TerminalNode BY() { return getToken(MixedFormatSqlExtendParser.BY, 0); } - public TerminalNode CACHE() { return getToken(MixedFormatSqlExtendParser.CACHE, 0); } - public TerminalNode CASCADE() { return getToken(MixedFormatSqlExtendParser.CASCADE, 0); } - public TerminalNode CASE() { return getToken(MixedFormatSqlExtendParser.CASE, 0); } - public TerminalNode CAST() { return getToken(MixedFormatSqlExtendParser.CAST, 0); } - public TerminalNode CHANGE() { return getToken(MixedFormatSqlExtendParser.CHANGE, 0); } - public TerminalNode CHECK() { return getToken(MixedFormatSqlExtendParser.CHECK, 0); } - public TerminalNode CLEAR() { return getToken(MixedFormatSqlExtendParser.CLEAR, 0); } - public TerminalNode CLUSTER() { return getToken(MixedFormatSqlExtendParser.CLUSTER, 0); } - public TerminalNode CLUSTERED() { return getToken(MixedFormatSqlExtendParser.CLUSTERED, 0); } - public TerminalNode CODEGEN() { return getToken(MixedFormatSqlExtendParser.CODEGEN, 0); } - public TerminalNode COLLATE() { return getToken(MixedFormatSqlExtendParser.COLLATE, 0); } - public TerminalNode COLLECTION() { return getToken(MixedFormatSqlExtendParser.COLLECTION, 0); } - public TerminalNode COLUMN() { return getToken(MixedFormatSqlExtendParser.COLUMN, 0); } - public TerminalNode COLUMNS() { return getToken(MixedFormatSqlExtendParser.COLUMNS, 0); } - public TerminalNode COMMENT() { return getToken(MixedFormatSqlExtendParser.COMMENT, 0); } - public TerminalNode COMMIT() { return getToken(MixedFormatSqlExtendParser.COMMIT, 0); } - public TerminalNode COMPACT() { return getToken(MixedFormatSqlExtendParser.COMPACT, 0); } - public TerminalNode COMPACTIONS() { return getToken(MixedFormatSqlExtendParser.COMPACTIONS, 0); } - public TerminalNode COMPUTE() { return getToken(MixedFormatSqlExtendParser.COMPUTE, 0); } - public TerminalNode CONCATENATE() { return getToken(MixedFormatSqlExtendParser.CONCATENATE, 0); } - public TerminalNode CONSTRAINT() { return getToken(MixedFormatSqlExtendParser.CONSTRAINT, 0); } - public TerminalNode COST() { return getToken(MixedFormatSqlExtendParser.COST, 0); } - public TerminalNode CREATE() { return getToken(MixedFormatSqlExtendParser.CREATE, 0); } - public TerminalNode CUBE() { return getToken(MixedFormatSqlExtendParser.CUBE, 0); } - public TerminalNode CURRENT() { return getToken(MixedFormatSqlExtendParser.CURRENT, 0); } - public TerminalNode CURRENT_DATE() { return getToken(MixedFormatSqlExtendParser.CURRENT_DATE, 0); } - public TerminalNode CURRENT_TIME() { return getToken(MixedFormatSqlExtendParser.CURRENT_TIME, 0); } - public TerminalNode CURRENT_TIMESTAMP() { return getToken(MixedFormatSqlExtendParser.CURRENT_TIMESTAMP, 0); } - public TerminalNode CURRENT_USER() { return getToken(MixedFormatSqlExtendParser.CURRENT_USER, 0); } - public TerminalNode DATA() { return getToken(MixedFormatSqlExtendParser.DATA, 0); } - public TerminalNode DATABASE() { return getToken(MixedFormatSqlExtendParser.DATABASE, 0); } - public TerminalNode DATABASES() { return getToken(MixedFormatSqlExtendParser.DATABASES, 0); } - public TerminalNode DAY() { return getToken(MixedFormatSqlExtendParser.DAY, 0); } - public TerminalNode DBPROPERTIES() { return getToken(MixedFormatSqlExtendParser.DBPROPERTIES, 0); } - public TerminalNode DEFINED() { return getToken(MixedFormatSqlExtendParser.DEFINED, 0); } - public TerminalNode DELETE() { return getToken(MixedFormatSqlExtendParser.DELETE, 0); } - public TerminalNode DELIMITED() { return getToken(MixedFormatSqlExtendParser.DELIMITED, 0); } - public TerminalNode DESC() { return getToken(MixedFormatSqlExtendParser.DESC, 0); } - public TerminalNode DESCRIBE() { return getToken(MixedFormatSqlExtendParser.DESCRIBE, 0); } - public TerminalNode DFS() { return getToken(MixedFormatSqlExtendParser.DFS, 0); } - public TerminalNode DIRECTORIES() { return getToken(MixedFormatSqlExtendParser.DIRECTORIES, 0); } - public TerminalNode DIRECTORY() { return getToken(MixedFormatSqlExtendParser.DIRECTORY, 0); } - public TerminalNode DISTINCT() { return getToken(MixedFormatSqlExtendParser.DISTINCT, 0); } - public TerminalNode DISTRIBUTE() { return getToken(MixedFormatSqlExtendParser.DISTRIBUTE, 0); } - public TerminalNode DIV() { return getToken(MixedFormatSqlExtendParser.DIV, 0); } - public TerminalNode DROP() { return getToken(MixedFormatSqlExtendParser.DROP, 0); } - public TerminalNode ELSE() { return getToken(MixedFormatSqlExtendParser.ELSE, 0); } - public TerminalNode END() { return getToken(MixedFormatSqlExtendParser.END, 0); } - public TerminalNode ESCAPE() { return getToken(MixedFormatSqlExtendParser.ESCAPE, 0); } - public TerminalNode ESCAPED() { return getToken(MixedFormatSqlExtendParser.ESCAPED, 0); } - public TerminalNode EXCHANGE() { return getToken(MixedFormatSqlExtendParser.EXCHANGE, 0); } - public TerminalNode EXISTS() { return getToken(MixedFormatSqlExtendParser.EXISTS, 0); } - public TerminalNode EXPLAIN() { return getToken(MixedFormatSqlExtendParser.EXPLAIN, 0); } - public TerminalNode EXPORT() { return getToken(MixedFormatSqlExtendParser.EXPORT, 0); } - public TerminalNode EXTENDED() { return getToken(MixedFormatSqlExtendParser.EXTENDED, 0); } - public TerminalNode EXTERNAL() { return getToken(MixedFormatSqlExtendParser.EXTERNAL, 0); } - public TerminalNode EXTRACT() { return getToken(MixedFormatSqlExtendParser.EXTRACT, 0); } - public TerminalNode FALSE() { return getToken(MixedFormatSqlExtendParser.FALSE, 0); } - public TerminalNode FETCH() { return getToken(MixedFormatSqlExtendParser.FETCH, 0); } - public TerminalNode FILTER() { return getToken(MixedFormatSqlExtendParser.FILTER, 0); } - public TerminalNode FIELDS() { return getToken(MixedFormatSqlExtendParser.FIELDS, 0); } - public TerminalNode FILEFORMAT() { return getToken(MixedFormatSqlExtendParser.FILEFORMAT, 0); } - public TerminalNode FIRST() { return getToken(MixedFormatSqlExtendParser.FIRST, 0); } - public TerminalNode FOLLOWING() { return getToken(MixedFormatSqlExtendParser.FOLLOWING, 0); } - public TerminalNode FOR() { return getToken(MixedFormatSqlExtendParser.FOR, 0); } - public TerminalNode FOREIGN() { return getToken(MixedFormatSqlExtendParser.FOREIGN, 0); } - public TerminalNode FORMAT() { return getToken(MixedFormatSqlExtendParser.FORMAT, 0); } - public TerminalNode FORMATTED() { return getToken(MixedFormatSqlExtendParser.FORMATTED, 0); } - public TerminalNode FROM() { return getToken(MixedFormatSqlExtendParser.FROM, 0); } - public TerminalNode FUNCTION() { return getToken(MixedFormatSqlExtendParser.FUNCTION, 0); } - public TerminalNode FUNCTIONS() { return getToken(MixedFormatSqlExtendParser.FUNCTIONS, 0); } - public TerminalNode GLOBAL() { return getToken(MixedFormatSqlExtendParser.GLOBAL, 0); } - public TerminalNode GRANT() { return getToken(MixedFormatSqlExtendParser.GRANT, 0); } - public TerminalNode GROUP() { return getToken(MixedFormatSqlExtendParser.GROUP, 0); } - public TerminalNode GROUPING() { return getToken(MixedFormatSqlExtendParser.GROUPING, 0); } - public TerminalNode HAVING() { return getToken(MixedFormatSqlExtendParser.HAVING, 0); } - public TerminalNode HOUR() { return getToken(MixedFormatSqlExtendParser.HOUR, 0); } - public TerminalNode IF() { return getToken(MixedFormatSqlExtendParser.IF, 0); } - public TerminalNode IGNORE() { return getToken(MixedFormatSqlExtendParser.IGNORE, 0); } - public TerminalNode IMPORT() { return getToken(MixedFormatSqlExtendParser.IMPORT, 0); } - public TerminalNode IN() { return getToken(MixedFormatSqlExtendParser.IN, 0); } - public TerminalNode INDEX() { return getToken(MixedFormatSqlExtendParser.INDEX, 0); } - public TerminalNode INDEXES() { return getToken(MixedFormatSqlExtendParser.INDEXES, 0); } - public TerminalNode INPATH() { return getToken(MixedFormatSqlExtendParser.INPATH, 0); } - public TerminalNode INPUTFORMAT() { return getToken(MixedFormatSqlExtendParser.INPUTFORMAT, 0); } - public TerminalNode INSERT() { return getToken(MixedFormatSqlExtendParser.INSERT, 0); } - public TerminalNode INTERVAL() { return getToken(MixedFormatSqlExtendParser.INTERVAL, 0); } - public TerminalNode INTO() { return getToken(MixedFormatSqlExtendParser.INTO, 0); } - public TerminalNode IS() { return getToken(MixedFormatSqlExtendParser.IS, 0); } - public TerminalNode ITEMS() { return getToken(MixedFormatSqlExtendParser.ITEMS, 0); } - public TerminalNode KEYS() { return getToken(MixedFormatSqlExtendParser.KEYS, 0); } - public TerminalNode LAST() { return getToken(MixedFormatSqlExtendParser.LAST, 0); } - public TerminalNode LAZY() { return getToken(MixedFormatSqlExtendParser.LAZY, 0); } - public TerminalNode LEADING() { return getToken(MixedFormatSqlExtendParser.LEADING, 0); } - public TerminalNode LIKE() { return getToken(MixedFormatSqlExtendParser.LIKE, 0); } - public TerminalNode LIMIT() { return getToken(MixedFormatSqlExtendParser.LIMIT, 0); } - public TerminalNode LINES() { return getToken(MixedFormatSqlExtendParser.LINES, 0); } - public TerminalNode LIST() { return getToken(MixedFormatSqlExtendParser.LIST, 0); } - public TerminalNode LOAD() { return getToken(MixedFormatSqlExtendParser.LOAD, 0); } - public TerminalNode LOCAL() { return getToken(MixedFormatSqlExtendParser.LOCAL, 0); } - public TerminalNode LOCATION() { return getToken(MixedFormatSqlExtendParser.LOCATION, 0); } - public TerminalNode LOCK() { return getToken(MixedFormatSqlExtendParser.LOCK, 0); } - public TerminalNode LOCKS() { return getToken(MixedFormatSqlExtendParser.LOCKS, 0); } - public TerminalNode LOGICAL() { return getToken(MixedFormatSqlExtendParser.LOGICAL, 0); } - public TerminalNode MACRO() { return getToken(MixedFormatSqlExtendParser.MACRO, 0); } - public TerminalNode MAP() { return getToken(MixedFormatSqlExtendParser.MAP, 0); } - public TerminalNode MATCHED() { return getToken(MixedFormatSqlExtendParser.MATCHED, 0); } - public TerminalNode MERGE() { return getToken(MixedFormatSqlExtendParser.MERGE, 0); } - public TerminalNode MINUTE() { return getToken(MixedFormatSqlExtendParser.MINUTE, 0); } - public TerminalNode MONTH() { return getToken(MixedFormatSqlExtendParser.MONTH, 0); } - public TerminalNode MSCK() { return getToken(MixedFormatSqlExtendParser.MSCK, 0); } - public TerminalNode NAMESPACE() { return getToken(MixedFormatSqlExtendParser.NAMESPACE, 0); } - public TerminalNode NAMESPACES() { return getToken(MixedFormatSqlExtendParser.NAMESPACES, 0); } - public TerminalNode NO() { return getToken(MixedFormatSqlExtendParser.NO, 0); } - public TerminalNode NOT() { return getToken(MixedFormatSqlExtendParser.NOT, 0); } - public TerminalNode NULL() { return getToken(MixedFormatSqlExtendParser.NULL, 0); } - public TerminalNode NULLS() { return getToken(MixedFormatSqlExtendParser.NULLS, 0); } - public TerminalNode OF() { return getToken(MixedFormatSqlExtendParser.OF, 0); } - public TerminalNode ONLY() { return getToken(MixedFormatSqlExtendParser.ONLY, 0); } - public TerminalNode OPTION() { return getToken(MixedFormatSqlExtendParser.OPTION, 0); } - public TerminalNode OPTIONS() { return getToken(MixedFormatSqlExtendParser.OPTIONS, 0); } - public TerminalNode OR() { return getToken(MixedFormatSqlExtendParser.OR, 0); } - public TerminalNode ORDER() { return getToken(MixedFormatSqlExtendParser.ORDER, 0); } - public TerminalNode OUT() { return getToken(MixedFormatSqlExtendParser.OUT, 0); } - public TerminalNode OUTER() { return getToken(MixedFormatSqlExtendParser.OUTER, 0); } - public TerminalNode OUTPUTFORMAT() { return getToken(MixedFormatSqlExtendParser.OUTPUTFORMAT, 0); } - public TerminalNode OVER() { return getToken(MixedFormatSqlExtendParser.OVER, 0); } - public TerminalNode OVERLAPS() { return getToken(MixedFormatSqlExtendParser.OVERLAPS, 0); } - public TerminalNode OVERLAY() { return getToken(MixedFormatSqlExtendParser.OVERLAY, 0); } - public TerminalNode OVERWRITE() { return getToken(MixedFormatSqlExtendParser.OVERWRITE, 0); } - public TerminalNode PARTITION() { return getToken(MixedFormatSqlExtendParser.PARTITION, 0); } - public TerminalNode PARTITIONED() { return getToken(MixedFormatSqlExtendParser.PARTITIONED, 0); } - public TerminalNode PARTITIONS() { return getToken(MixedFormatSqlExtendParser.PARTITIONS, 0); } - public TerminalNode PERCENTLIT() { return getToken(MixedFormatSqlExtendParser.PERCENTLIT, 0); } - public TerminalNode PIVOT() { return getToken(MixedFormatSqlExtendParser.PIVOT, 0); } - public TerminalNode PLACING() { return getToken(MixedFormatSqlExtendParser.PLACING, 0); } - public TerminalNode POSITION() { return getToken(MixedFormatSqlExtendParser.POSITION, 0); } - public TerminalNode PRECEDING() { return getToken(MixedFormatSqlExtendParser.PRECEDING, 0); } - public TerminalNode PRIMARY() { return getToken(MixedFormatSqlExtendParser.PRIMARY, 0); } - public TerminalNode PRINCIPALS() { return getToken(MixedFormatSqlExtendParser.PRINCIPALS, 0); } - public TerminalNode PROPERTIES() { return getToken(MixedFormatSqlExtendParser.PROPERTIES, 0); } - public TerminalNode PURGE() { return getToken(MixedFormatSqlExtendParser.PURGE, 0); } - public TerminalNode QUERY() { return getToken(MixedFormatSqlExtendParser.QUERY, 0); } - public TerminalNode RANGE() { return getToken(MixedFormatSqlExtendParser.RANGE, 0); } - public TerminalNode RECORDREADER() { return getToken(MixedFormatSqlExtendParser.RECORDREADER, 0); } - public TerminalNode RECORDWRITER() { return getToken(MixedFormatSqlExtendParser.RECORDWRITER, 0); } - public TerminalNode RECOVER() { return getToken(MixedFormatSqlExtendParser.RECOVER, 0); } - public TerminalNode REDUCE() { return getToken(MixedFormatSqlExtendParser.REDUCE, 0); } - public TerminalNode REFERENCES() { return getToken(MixedFormatSqlExtendParser.REFERENCES, 0); } - public TerminalNode REFRESH() { return getToken(MixedFormatSqlExtendParser.REFRESH, 0); } - public TerminalNode RENAME() { return getToken(MixedFormatSqlExtendParser.RENAME, 0); } - public TerminalNode REPAIR() { return getToken(MixedFormatSqlExtendParser.REPAIR, 0); } - public TerminalNode REPLACE() { return getToken(MixedFormatSqlExtendParser.REPLACE, 0); } - public TerminalNode RESET() { return getToken(MixedFormatSqlExtendParser.RESET, 0); } - public TerminalNode RESPECT() { return getToken(MixedFormatSqlExtendParser.RESPECT, 0); } - public TerminalNode RESTRICT() { return getToken(MixedFormatSqlExtendParser.RESTRICT, 0); } - public TerminalNode REVOKE() { return getToken(MixedFormatSqlExtendParser.REVOKE, 0); } - public TerminalNode RLIKE() { return getToken(MixedFormatSqlExtendParser.RLIKE, 0); } - public TerminalNode ROLE() { return getToken(MixedFormatSqlExtendParser.ROLE, 0); } - public TerminalNode ROLES() { return getToken(MixedFormatSqlExtendParser.ROLES, 0); } - public TerminalNode ROLLBACK() { return getToken(MixedFormatSqlExtendParser.ROLLBACK, 0); } - public TerminalNode ROLLUP() { return getToken(MixedFormatSqlExtendParser.ROLLUP, 0); } - public TerminalNode ROW() { return getToken(MixedFormatSqlExtendParser.ROW, 0); } - public TerminalNode ROWS() { return getToken(MixedFormatSqlExtendParser.ROWS, 0); } - public TerminalNode SCHEMA() { return getToken(MixedFormatSqlExtendParser.SCHEMA, 0); } - public TerminalNode SECOND() { return getToken(MixedFormatSqlExtendParser.SECOND, 0); } - public TerminalNode SELECT() { return getToken(MixedFormatSqlExtendParser.SELECT, 0); } - public TerminalNode SEPARATED() { return getToken(MixedFormatSqlExtendParser.SEPARATED, 0); } - public TerminalNode SERDE() { return getToken(MixedFormatSqlExtendParser.SERDE, 0); } - public TerminalNode SERDEPROPERTIES() { return getToken(MixedFormatSqlExtendParser.SERDEPROPERTIES, 0); } - public TerminalNode SESSION_USER() { return getToken(MixedFormatSqlExtendParser.SESSION_USER, 0); } - public TerminalNode SET() { return getToken(MixedFormatSqlExtendParser.SET, 0); } - public TerminalNode SETS() { return getToken(MixedFormatSqlExtendParser.SETS, 0); } - public TerminalNode SHOW() { return getToken(MixedFormatSqlExtendParser.SHOW, 0); } - public TerminalNode SKEWED() { return getToken(MixedFormatSqlExtendParser.SKEWED, 0); } - public TerminalNode SOME() { return getToken(MixedFormatSqlExtendParser.SOME, 0); } - public TerminalNode SORT() { return getToken(MixedFormatSqlExtendParser.SORT, 0); } - public TerminalNode SORTED() { return getToken(MixedFormatSqlExtendParser.SORTED, 0); } - public TerminalNode START() { return getToken(MixedFormatSqlExtendParser.START, 0); } - public TerminalNode STATISTICS() { return getToken(MixedFormatSqlExtendParser.STATISTICS, 0); } - public TerminalNode STORED() { return getToken(MixedFormatSqlExtendParser.STORED, 0); } - public TerminalNode STRATIFY() { return getToken(MixedFormatSqlExtendParser.STRATIFY, 0); } - public TerminalNode STRUCT() { return getToken(MixedFormatSqlExtendParser.STRUCT, 0); } - public TerminalNode SUBSTR() { return getToken(MixedFormatSqlExtendParser.SUBSTR, 0); } - public TerminalNode SUBSTRING() { return getToken(MixedFormatSqlExtendParser.SUBSTRING, 0); } - public TerminalNode SYNC() { return getToken(MixedFormatSqlExtendParser.SYNC, 0); } - public TerminalNode TABLE() { return getToken(MixedFormatSqlExtendParser.TABLE, 0); } - public TerminalNode TABLES() { return getToken(MixedFormatSqlExtendParser.TABLES, 0); } - public TerminalNode TABLESAMPLE() { return getToken(MixedFormatSqlExtendParser.TABLESAMPLE, 0); } - public TerminalNode TBLPROPERTIES() { return getToken(MixedFormatSqlExtendParser.TBLPROPERTIES, 0); } - public TerminalNode TEMPORARY() { return getToken(MixedFormatSqlExtendParser.TEMPORARY, 0); } - public TerminalNode TERMINATED() { return getToken(MixedFormatSqlExtendParser.TERMINATED, 0); } - public TerminalNode THEN() { return getToken(MixedFormatSqlExtendParser.THEN, 0); } - public TerminalNode TIME() { return getToken(MixedFormatSqlExtendParser.TIME, 0); } - public TerminalNode TO() { return getToken(MixedFormatSqlExtendParser.TO, 0); } - public TerminalNode TOUCH() { return getToken(MixedFormatSqlExtendParser.TOUCH, 0); } - public TerminalNode TRAILING() { return getToken(MixedFormatSqlExtendParser.TRAILING, 0); } - public TerminalNode TRANSACTION() { return getToken(MixedFormatSqlExtendParser.TRANSACTION, 0); } - public TerminalNode TRANSACTIONS() { return getToken(MixedFormatSqlExtendParser.TRANSACTIONS, 0); } - public TerminalNode TRANSFORM() { return getToken(MixedFormatSqlExtendParser.TRANSFORM, 0); } - public TerminalNode TRIM() { return getToken(MixedFormatSqlExtendParser.TRIM, 0); } - public TerminalNode TRUE() { return getToken(MixedFormatSqlExtendParser.TRUE, 0); } - public TerminalNode TRUNCATE() { return getToken(MixedFormatSqlExtendParser.TRUNCATE, 0); } - public TerminalNode TRY_CAST() { return getToken(MixedFormatSqlExtendParser.TRY_CAST, 0); } - public TerminalNode TYPE() { return getToken(MixedFormatSqlExtendParser.TYPE, 0); } - public TerminalNode UNARCHIVE() { return getToken(MixedFormatSqlExtendParser.UNARCHIVE, 0); } - public TerminalNode UNBOUNDED() { return getToken(MixedFormatSqlExtendParser.UNBOUNDED, 0); } - public TerminalNode UNCACHE() { return getToken(MixedFormatSqlExtendParser.UNCACHE, 0); } - public TerminalNode UNIQUE() { return getToken(MixedFormatSqlExtendParser.UNIQUE, 0); } - public TerminalNode UNKNOWN() { return getToken(MixedFormatSqlExtendParser.UNKNOWN, 0); } - public TerminalNode UNLOCK() { return getToken(MixedFormatSqlExtendParser.UNLOCK, 0); } - public TerminalNode UNSET() { return getToken(MixedFormatSqlExtendParser.UNSET, 0); } - public TerminalNode UPDATE() { return getToken(MixedFormatSqlExtendParser.UPDATE, 0); } - public TerminalNode USE() { return getToken(MixedFormatSqlExtendParser.USE, 0); } - public TerminalNode USER() { return getToken(MixedFormatSqlExtendParser.USER, 0); } - public TerminalNode VALUES() { return getToken(MixedFormatSqlExtendParser.VALUES, 0); } - public TerminalNode VIEW() { return getToken(MixedFormatSqlExtendParser.VIEW, 0); } - public TerminalNode VIEWS() { return getToken(MixedFormatSqlExtendParser.VIEWS, 0); } - public TerminalNode WHEN() { return getToken(MixedFormatSqlExtendParser.WHEN, 0); } - public TerminalNode WHERE() { return getToken(MixedFormatSqlExtendParser.WHERE, 0); } - public TerminalNode WINDOW() { return getToken(MixedFormatSqlExtendParser.WINDOW, 0); } - public TerminalNode WITH() { return getToken(MixedFormatSqlExtendParser.WITH, 0); } - public TerminalNode YEAR() { return getToken(MixedFormatSqlExtendParser.YEAR, 0); } - public TerminalNode ZONE() { return getToken(MixedFormatSqlExtendParser.ZONE, 0); } - public NonReservedContext(ParserRuleContext parent, int invokingState) { - super(parent, invokingState); - } - @Override public int getRuleIndex() { return RULE_nonReserved; } - @Override - public void enterRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).enterNonReserved(this); - } - @Override - public void exitRule(ParseTreeListener listener) { - if ( listener instanceof MixedFormatSqlExtendListener ) ((MixedFormatSqlExtendListener)listener).exitNonReserved(this); - } - @Override - public T accept(ParseTreeVisitor visitor) { - if ( visitor instanceof MixedFormatSqlExtendVisitor ) return ((MixedFormatSqlExtendVisitor)visitor).visitNonReserved(this); - else return visitor.visitChildren(this); - } - } - - public final NonReservedContext nonReserved() throws RecognitionException { - NonReservedContext _localctx = new NonReservedContext(_ctx, getState()); - enterRule(_localctx, 212, RULE_nonReserved); - int _la; - try { - enterOuterAlt(_localctx, 1); - { - setState(1861); - _la = _input.LA(1); - if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ADD) | (1L << AFTER) | (1L << ALL) | (1L << ALTER) | (1L << ANALYZE) | (1L << AND) | (1L << ANY) | (1L << ARCHIVE) | (1L << ARRAY) | (1L << AS) | (1L << ASC) | (1L << AT) | (1L << AUTHORIZATION) | (1L << BETWEEN) | (1L << BOTH) | (1L << BUCKET) | (1L << BUCKETS) | (1L << BY) | (1L << CACHE) | (1L << CASCADE) | (1L << CASE) | (1L << CAST) | (1L << CHANGE) | (1L << CHECK) | (1L << CLEAR) | (1L << CLUSTER) | (1L << CLUSTERED) | (1L << CODEGEN) | (1L << COLLATE) | (1L << COLLECTION) | (1L << COLUMN) | (1L << COLUMNS) | (1L << COMMENT) | (1L << COMMIT) | (1L << COMPACT) | (1L << COMPACTIONS) | (1L << COMPUTE) | (1L << CONCATENATE) | (1L << CONSTRAINT) | (1L << COST) | (1L << CREATE) | (1L << CUBE) | (1L << CURRENT) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << CURRENT_USER) | (1L << DAY) | (1L << DATA) | (1L << DATABASE))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (DATABASES - 64)) | (1L << (DBPROPERTIES - 64)) | (1L << (DEFINED - 64)) | (1L << (DELETE - 64)) | (1L << (DELIMITED - 64)) | (1L << (DESC - 64)) | (1L << (DESCRIBE - 64)) | (1L << (DFS - 64)) | (1L << (DIRECTORIES - 64)) | (1L << (DIRECTORY - 64)) | (1L << (DISTINCT - 64)) | (1L << (DISTRIBUTE - 64)) | (1L << (DIV - 64)) | (1L << (DROP - 64)) | (1L << (ELSE - 64)) | (1L << (END - 64)) | (1L << (ESCAPE - 64)) | (1L << (ESCAPED - 64)) | (1L << (EXCHANGE - 64)) | (1L << (EXISTS - 64)) | (1L << (EXPLAIN - 64)) | (1L << (EXPORT - 64)) | (1L << (EXTENDED - 64)) | (1L << (EXTERNAL - 64)) | (1L << (EXTRACT - 64)) | (1L << (FALSE - 64)) | (1L << (FETCH - 64)) | (1L << (FIELDS - 64)) | (1L << (FILTER - 64)) | (1L << (FILEFORMAT - 64)) | (1L << (FIRST - 64)) | (1L << (FOLLOWING - 64)) | (1L << (FOR - 64)) | (1L << (FOREIGN - 64)) | (1L << (FORMAT - 64)) | (1L << (FORMATTED - 64)) | (1L << (FROM - 64)) | (1L << (FUNCTION - 64)) | (1L << (FUNCTIONS - 64)) | (1L << (GLOBAL - 64)) | (1L << (GRANT - 64)) | (1L << (GROUP - 64)) | (1L << (GROUPING - 64)) | (1L << (HAVING - 64)) | (1L << (HOUR - 64)) | (1L << (IF - 64)) | (1L << (IGNORE - 64)) | (1L << (IMPORT - 64)) | (1L << (IN - 64)) | (1L << (INDEX - 64)) | (1L << (INDEXES - 64)) | (1L << (INPATH - 64)) | (1L << (INPUTFORMAT - 64)) | (1L << (INSERT - 64)) | (1L << (INTERVAL - 64)) | (1L << (INTO - 64)) | (1L << (IS - 64)) | (1L << (ITEMS - 64)) | (1L << (KEYS - 64)))) != 0) || ((((_la - 128)) & ~0x3f) == 0 && ((1L << (_la - 128)) & ((1L << (LAST - 128)) | (1L << (LAZY - 128)) | (1L << (LEADING - 128)) | (1L << (LIKE - 128)) | (1L << (LIMIT - 128)) | (1L << (LINES - 128)) | (1L << (LIST - 128)) | (1L << (LOAD - 128)) | (1L << (LOCAL - 128)) | (1L << (LOCATION - 128)) | (1L << (LOCK - 128)) | (1L << (LOCKS - 128)) | (1L << (LOGICAL - 128)) | (1L << (MACRO - 128)) | (1L << (MAP - 128)) | (1L << (MATCHED - 128)) | (1L << (MERGE - 128)) | (1L << (MINUTE - 128)) | (1L << (MONTH - 128)) | (1L << (MSCK - 128)) | (1L << (NAMESPACE - 128)) | (1L << (NAMESPACES - 128)) | (1L << (NO - 128)) | (1L << (NOT - 128)) | (1L << (NULL - 128)) | (1L << (NULLS - 128)) | (1L << (OF - 128)) | (1L << (ONLY - 128)) | (1L << (OPTION - 128)) | (1L << (OPTIONS - 128)) | (1L << (OR - 128)) | (1L << (ORDER - 128)) | (1L << (OUT - 128)) | (1L << (OUTER - 128)) | (1L << (OUTPUTFORMAT - 128)) | (1L << (OVER - 128)) | (1L << (OVERLAPS - 128)) | (1L << (OVERLAY - 128)) | (1L << (OVERWRITE - 128)) | (1L << (PARTITION - 128)) | (1L << (PARTITIONED - 128)) | (1L << (PARTITIONS - 128)) | (1L << (PERCENTLIT - 128)) | (1L << (PIVOT - 128)) | (1L << (PLACING - 128)) | (1L << (POSITION - 128)) | (1L << (PRECEDING - 128)) | (1L << (PRIMARY - 128)) | (1L << (PRINCIPALS - 128)) | (1L << (PROPERTIES - 128)) | (1L << (PURGE - 128)) | (1L << (QUERY - 128)) | (1L << (RANGE - 128)) | (1L << (RECORDREADER - 128)) | (1L << (RECORDWRITER - 128)) | (1L << (RECOVER - 128)) | (1L << (REDUCE - 128)) | (1L << (REFERENCES - 128)) | (1L << (REFRESH - 128)) | (1L << (RENAME - 128)))) != 0) || ((((_la - 192)) & ~0x3f) == 0 && ((1L << (_la - 192)) & ((1L << (REPAIR - 192)) | (1L << (REPLACE - 192)) | (1L << (RESET - 192)) | (1L << (RESPECT - 192)) | (1L << (RESTRICT - 192)) | (1L << (REVOKE - 192)) | (1L << (RLIKE - 192)) | (1L << (ROLE - 192)) | (1L << (ROLES - 192)) | (1L << (ROLLBACK - 192)) | (1L << (ROLLUP - 192)) | (1L << (ROW - 192)) | (1L << (ROWS - 192)) | (1L << (SECOND - 192)) | (1L << (SCHEMA - 192)) | (1L << (SELECT - 192)) | (1L << (SEPARATED - 192)) | (1L << (SERDE - 192)) | (1L << (SERDEPROPERTIES - 192)) | (1L << (SESSION_USER - 192)) | (1L << (SET - 192)) | (1L << (SETS - 192)) | (1L << (SHOW - 192)) | (1L << (SKEWED - 192)) | (1L << (SOME - 192)) | (1L << (SORT - 192)) | (1L << (SORTED - 192)) | (1L << (START - 192)) | (1L << (STATISTICS - 192)) | (1L << (STORED - 192)) | (1L << (STRATIFY - 192)) | (1L << (STRUCT - 192)) | (1L << (SUBSTR - 192)) | (1L << (SUBSTRING - 192)) | (1L << (SYNC - 192)) | (1L << (TABLE - 192)) | (1L << (TABLES - 192)) | (1L << (TABLESAMPLE - 192)) | (1L << (TBLPROPERTIES - 192)) | (1L << (TEMPORARY - 192)) | (1L << (TERMINATED - 192)) | (1L << (THEN - 192)) | (1L << (TIME - 192)) | (1L << (TO - 192)) | (1L << (TOUCH - 192)) | (1L << (TRAILING - 192)) | (1L << (TRANSACTION - 192)) | (1L << (TRANSACTIONS - 192)) | (1L << (TRANSFORM - 192)) | (1L << (TRIM - 192)) | (1L << (TRUE - 192)) | (1L << (TRUNCATE - 192)) | (1L << (TRY_CAST - 192)) | (1L << (TYPE - 192)) | (1L << (UNARCHIVE - 192)) | (1L << (UNBOUNDED - 192)) | (1L << (UNCACHE - 192)) | (1L << (UNIQUE - 192)) | (1L << (UNKNOWN - 192)) | (1L << (UNLOCK - 192)))) != 0) || ((((_la - 256)) & ~0x3f) == 0 && ((1L << (_la - 256)) & ((1L << (UNSET - 256)) | (1L << (UPDATE - 256)) | (1L << (USE - 256)) | (1L << (USER - 256)) | (1L << (VALUES - 256)) | (1L << (VIEW - 256)) | (1L << (VIEWS - 256)) | (1L << (WHEN - 256)) | (1L << (WHERE - 256)) | (1L << (WINDOW - 256)) | (1L << (WITH - 256)) | (1L << (YEAR - 256)) | (1L << (ZONE - 256)))) != 0)) ) { - _errHandler.recoverInline(this); - } - else { - if ( _input.LA(1)==Token.EOF ) matchedEOF = true; - _errHandler.reportMatch(this); - consume(); - } - } - } - catch (RecognitionException re) { - _localctx.exception = re; - _errHandler.reportError(this, re); - _errHandler.recover(this, re); - } - finally { - exitRule(); - } - return _localctx; - } - - public boolean sempred(RuleContext _localctx, int ruleIndex, int predIndex) { - switch (ruleIndex) { - case 24: - return queryTerm_sempred((QueryTermContext)_localctx, predIndex); - case 71: - return booleanExpression_sempred((BooleanExpressionContext)_localctx, predIndex); - case 73: - return valueExpression_sempred((ValueExpressionContext)_localctx, predIndex); - case 74: - return primaryExpression_sempred((PrimaryExpressionContext)_localctx, predIndex); - case 100: - return identifier_sempred((IdentifierContext)_localctx, predIndex); - case 101: - return strictIdentifier_sempred((StrictIdentifierContext)_localctx, predIndex); - case 103: - return number_sempred((NumberContext)_localctx, predIndex); - } - return true; - } - private boolean queryTerm_sempred(QueryTermContext _localctx, int predIndex) { - switch (predIndex) { - case 0: - return precpred(_ctx, 3); - case 1: - return legacy_setops_precedence_enabled; - case 2: - return precpred(_ctx, 2); - case 3: - return !legacy_setops_precedence_enabled; - case 4: - return precpred(_ctx, 1); - case 5: - return !legacy_setops_precedence_enabled; - } - return true; - } - private boolean booleanExpression_sempred(BooleanExpressionContext _localctx, int predIndex) { - switch (predIndex) { - case 6: - return precpred(_ctx, 2); - case 7: - return precpred(_ctx, 1); - } - return true; - } - private boolean valueExpression_sempred(ValueExpressionContext _localctx, int predIndex) { - switch (predIndex) { - case 8: - return precpred(_ctx, 6); - case 9: - return precpred(_ctx, 5); - case 10: - return precpred(_ctx, 4); - case 11: - return precpred(_ctx, 3); - case 12: - return precpred(_ctx, 2); - case 13: - return precpred(_ctx, 1); - } - return true; - } - private boolean primaryExpression_sempred(PrimaryExpressionContext _localctx, int predIndex) { - switch (predIndex) { - case 14: - return precpred(_ctx, 8); - case 15: - return precpred(_ctx, 6); - } - return true; - } - private boolean identifier_sempred(IdentifierContext _localctx, int predIndex) { - switch (predIndex) { - case 16: - return !SQL_standard_keyword_behavior; - } - return true; - } - private boolean strictIdentifier_sempred(StrictIdentifierContext _localctx, int predIndex) { - switch (predIndex) { - case 17: - return SQL_standard_keyword_behavior; - case 18: - return !SQL_standard_keyword_behavior; - } - return true; - } - private boolean number_sempred(NumberContext _localctx, int predIndex) { - switch (predIndex) { - case 19: - return !legacy_exponent_literal_as_decimal_enabled; - case 20: - return !legacy_exponent_literal_as_decimal_enabled; - case 21: - return legacy_exponent_literal_as_decimal_enabled; - } - return true; - } - - public static final String _serializedATN = - "\3\u608b\ua72a\u8133\ub9ed\u417c\u3be7\u7786\u5964\3\u0132\u074a\4\2\t"+ - "\2\4\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4\13"+ - "\t\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22\t\22"+ - "\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\4\30\t\30\4\31\t\31"+ - "\4\32\t\32\4\33\t\33\4\34\t\34\4\35\t\35\4\36\t\36\4\37\t\37\4 \t \4!"+ - "\t!\4\"\t\"\4#\t#\4$\t$\4%\t%\4&\t&\4\'\t\'\4(\t(\4)\t)\4*\t*\4+\t+\4"+ - ",\t,\4-\t-\4.\t.\4/\t/\4\60\t\60\4\61\t\61\4\62\t\62\4\63\t\63\4\64\t"+ - "\64\4\65\t\65\4\66\t\66\4\67\t\67\48\t8\49\t9\4:\t:\4;\t;\4<\t<\4=\t="+ - "\4>\t>\4?\t?\4@\t@\4A\tA\4B\tB\4C\tC\4D\tD\4E\tE\4F\tF\4G\tG\4H\tH\4I"+ - "\tI\4J\tJ\4K\tK\4L\tL\4M\tM\4N\tN\4O\tO\4P\tP\4Q\tQ\4R\tR\4S\tS\4T\tT"+ - "\4U\tU\4V\tV\4W\tW\4X\tX\4Y\tY\4Z\tZ\4[\t[\4\\\t\\\4]\t]\4^\t^\4_\t_\4"+ - "`\t`\4a\ta\4b\tb\4c\tc\4d\td\4e\te\4f\tf\4g\tg\4h\th\4i\ti\4j\tj\4k\t"+ - "k\4l\tl\3\2\3\2\7\2\u00db\n\2\f\2\16\2\u00de\13\2\3\2\3\2\3\3\3\3\3\3"+ - "\5\3\u00e5\n\3\3\3\3\3\5\3\u00e9\n\3\3\3\5\3\u00ec\n\3\3\3\3\3\5\3\u00f0"+ - "\n\3\3\3\5\3\u00f3\n\3\3\4\3\4\5\4\u00f7\n\4\3\4\5\4\u00fa\n\4\3\4\3\4"+ - "\3\4\3\4\5\4\u0100\n\4\3\4\3\4\3\5\3\5\3\5\3\5\5\5\u0108\n\5\3\5\3\5\3"+ - "\5\5\5\u010d\n\5\3\6\3\6\3\6\3\6\3\7\3\7\3\7\3\7\3\7\3\7\5\7\u0119\n\7"+ - "\3\7\3\7\3\7\3\7\3\b\3\b\3\b\3\b\3\b\3\b\5\b\u0125\n\b\3\b\3\b\3\b\5\b"+ - "\u012a\n\b\3\t\3\t\3\t\3\n\3\n\3\n\3\13\5\13\u0133\n\13\3\13\3\13\3\13"+ - "\3\f\3\f\3\f\3\f\7\f\u013c\n\f\f\f\16\f\u013f\13\f\3\r\3\r\5\r\u0143\n"+ - "\r\3\r\5\r\u0146\n\r\3\r\3\r\3\r\3\r\3\16\3\16\3\16\3\17\3\17\3\17\3\17"+ - "\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\7\17\u015c\n\17\f\17\16"+ - "\17\u015f\13\17\3\20\3\20\3\20\3\20\7\20\u0165\n\20\f\20\16\20\u0168\13"+ - "\20\3\20\3\20\3\21\3\21\5\21\u016e\n\21\3\21\5\21\u0171\n\21\3\22\3\22"+ - "\3\22\7\22\u0176\n\22\f\22\16\22\u0179\13\22\3\22\5\22\u017c\n\22\3\23"+ - "\3\23\3\23\3\23\5\23\u0182\n\23\3\24\3\24\3\24\3\24\7\24\u0188\n\24\f"+ - "\24\16\24\u018b\13\24\3\24\3\24\3\25\3\25\3\25\3\25\7\25\u0193\n\25\f"+ - "\25\16\25\u0196\13\25\3\25\3\25\3\26\3\26\3\26\3\26\3\26\3\26\5\26\u01a0"+ - "\n\26\3\27\3\27\3\27\3\27\3\27\5\27\u01a7\n\27\3\30\3\30\3\30\3\30\5\30"+ - "\u01ad\n\30\3\31\3\31\3\31\3\31\3\31\7\31\u01b4\n\31\f\31\16\31\u01b7"+ - "\13\31\5\31\u01b9\n\31\3\31\3\31\3\31\3\31\3\31\7\31\u01c0\n\31\f\31\16"+ - "\31\u01c3\13\31\5\31\u01c5\n\31\3\31\3\31\3\31\3\31\3\31\7\31\u01cc\n"+ - "\31\f\31\16\31\u01cf\13\31\5\31\u01d1\n\31\3\31\3\31\3\31\3\31\3\31\7"+ - "\31\u01d8\n\31\f\31\16\31\u01db\13\31\5\31\u01dd\n\31\3\31\5\31\u01e0"+ - "\n\31\3\31\3\31\3\31\5\31\u01e5\n\31\5\31\u01e7\n\31\3\32\3\32\3\32\3"+ - "\32\3\32\3\32\3\32\5\32\u01f0\n\32\3\32\3\32\3\32\3\32\3\32\5\32\u01f7"+ - "\n\32\3\32\3\32\3\32\3\32\3\32\5\32\u01fe\n\32\3\32\7\32\u0201\n\32\f"+ - "\32\16\32\u0204\13\32\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\5\33"+ - "\u020f\n\33\3\34\3\34\5\34\u0213\n\34\3\34\3\34\5\34\u0217\n\34\3\35\3"+ - "\35\6\35\u021b\n\35\r\35\16\35\u021c\3\36\3\36\5\36\u0221\n\36\3\36\3"+ - "\36\3\36\3\36\7\36\u0227\n\36\f\36\16\36\u022a\13\36\3\36\5\36\u022d\n"+ - "\36\3\36\5\36\u0230\n\36\3\36\5\36\u0233\n\36\3\36\5\36\u0236\n\36\3\36"+ - "\3\36\5\36\u023a\n\36\3\37\3\37\5\37\u023e\n\37\3\37\7\37\u0241\n\37\f"+ - "\37\16\37\u0244\13\37\3\37\5\37\u0247\n\37\3\37\5\37\u024a\n\37\3\37\5"+ - "\37\u024d\n\37\3\37\5\37\u0250\n\37\3\37\3\37\5\37\u0254\n\37\3\37\7\37"+ - "\u0257\n\37\f\37\16\37\u025a\13\37\3\37\5\37\u025d\n\37\3\37\5\37\u0260"+ - "\n\37\3\37\5\37\u0263\n\37\3\37\5\37\u0266\n\37\5\37\u0268\n\37\3 \3 "+ - "\3 \3 \5 \u026e\n \3 \3 \3 \3 \3 \5 \u0275\n \3 \3 \3 \5 \u027a\n \3 "+ - "\5 \u027d\n \3 \5 \u0280\n \3 \3 \5 \u0284\n \3 \3 \3 \3 \3 \3 \3 \3 "+ - "\5 \u028e\n \3 \3 \5 \u0292\n \5 \u0294\n \3 \5 \u0297\n \3 \3 \5 \u029b"+ - "\n \3!\3!\7!\u029f\n!\f!\16!\u02a2\13!\3!\5!\u02a5\n!\3!\3!\3\"\3\"\3"+ - "\"\3#\3#\3#\3$\3$\3$\5$\u02b2\n$\3$\7$\u02b5\n$\f$\16$\u02b8\13$\3$\3"+ - "$\3%\3%\3%\3%\3%\3%\7%\u02c2\n%\f%\16%\u02c5\13%\3%\3%\5%\u02c9\n%\3&"+ - "\3&\3&\3&\7&\u02cf\n&\f&\16&\u02d2\13&\3&\7&\u02d5\n&\f&\16&\u02d8\13"+ - "&\3&\5&\u02db\n&\3\'\3\'\3\'\3\'\3\'\7\'\u02e2\n\'\f\'\16\'\u02e5\13\'"+ - "\3\'\3\'\3\'\3\'\3\'\7\'\u02ec\n\'\f\'\16\'\u02ef\13\'\3\'\3\'\3\'\3\'"+ - "\3\'\3\'\3\'\3\'\3\'\3\'\7\'\u02fb\n\'\f\'\16\'\u02fe\13\'\3\'\3\'\5\'"+ - "\u0302\n\'\5\'\u0304\n\'\3(\3(\5(\u0308\n(\3)\3)\3)\3)\3)\7)\u030f\n)"+ - "\f)\16)\u0312\13)\3)\3)\3)\3)\3)\3)\3)\3)\7)\u031c\n)\f)\16)\u031f\13"+ - ")\3)\3)\5)\u0323\n)\3*\3*\5*\u0327\n*\3+\3+\3+\3+\7+\u032d\n+\f+\16+\u0330"+ - "\13+\5+\u0332\n+\3+\3+\5+\u0336\n+\3,\3,\3,\3,\3,\3,\3,\3,\3,\3,\7,\u0342"+ - "\n,\f,\16,\u0345\13,\3,\3,\3,\3-\3-\3-\3-\3-\7-\u034f\n-\f-\16-\u0352"+ - "\13-\3-\3-\5-\u0356\n-\3.\3.\5.\u035a\n.\3.\5.\u035d\n.\3/\3/\3/\5/\u0362"+ - "\n/\3/\3/\3/\3/\3/\7/\u0369\n/\f/\16/\u036c\13/\5/\u036e\n/\3/\3/\3/\5"+ - "/\u0373\n/\3/\3/\3/\7/\u0378\n/\f/\16/\u037b\13/\5/\u037d\n/\3\60\3\60"+ - "\3\61\5\61\u0382\n\61\3\61\3\61\7\61\u0386\n\61\f\61\16\61\u0389\13\61"+ - "\3\62\3\62\3\62\5\62\u038e\n\62\3\62\3\62\5\62\u0392\n\62\3\62\3\62\3"+ - "\62\3\62\5\62\u0398\n\62\3\62\3\62\5\62\u039c\n\62\3\63\5\63\u039f\n\63"+ - "\3\63\3\63\3\63\5\63\u03a4\n\63\3\63\5\63\u03a7\n\63\3\63\3\63\3\63\5"+ - "\63\u03ac\n\63\3\63\3\63\5\63\u03b0\n\63\3\63\5\63\u03b3\n\63\3\63\5\63"+ - "\u03b6\n\63\3\64\3\64\3\64\3\64\5\64\u03bc\n\64\3\65\3\65\3\65\5\65\u03c1"+ - "\n\65\3\65\3\65\3\66\5\66\u03c6\n\66\3\66\3\66\3\66\3\66\3\66\3\66\3\66"+ - "\3\66\3\66\3\66\3\66\3\66\3\66\3\66\3\66\3\66\5\66\u03d8\n\66\5\66\u03da"+ - "\n\66\3\66\5\66\u03dd\n\66\3\67\3\67\3\67\3\67\38\38\38\78\u03e6\n8\f"+ - "8\168\u03e9\138\39\39\39\39\79\u03ef\n9\f9\169\u03f2\139\39\39\3:\3:\5"+ - ":\u03f8\n:\3;\3;\5;\u03fc\n;\3;\3;\3;\3;\3;\3;\5;\u0404\n;\3;\3;\3;\3"+ - ";\3;\3;\5;\u040c\n;\3;\3;\3;\3;\5;\u0412\n;\3<\3<\3<\3<\7<\u0418\n<\f"+ - "<\16<\u041b\13<\3<\3<\3=\3=\3=\3=\3=\7=\u0424\n=\f=\16=\u0427\13=\5=\u0429"+ - "\n=\3=\3=\3=\3>\5>\u042f\n>\3>\3>\5>\u0433\n>\5>\u0435\n>\3?\3?\3?\3?"+ - "\3?\3?\3?\5?\u043e\n?\3?\3?\3?\3?\3?\3?\3?\3?\3?\3?\5?\u044a\n?\5?\u044c"+ - "\n?\3?\3?\3?\3?\3?\5?\u0453\n?\3?\3?\3?\3?\3?\5?\u045a\n?\3?\3?\3?\3?"+ - "\5?\u0460\n?\3?\3?\3?\3?\5?\u0466\n?\5?\u0468\n?\3@\3@\3@\7@\u046d\n@"+ - "\f@\16@\u0470\13@\3A\3A\5A\u0474\nA\3A\3A\5A\u0478\nA\5A\u047a\nA\3B\3"+ - "B\3B\7B\u047f\nB\fB\16B\u0482\13B\3C\3C\3C\3C\7C\u0488\nC\fC\16C\u048b"+ - "\13C\3C\3C\3D\3D\5D\u0491\nD\3E\3E\3E\3E\3E\3E\7E\u0499\nE\fE\16E\u049c"+ - "\13E\3E\3E\5E\u04a0\nE\3F\3F\5F\u04a4\nF\3G\3G\3H\3H\3H\7H\u04ab\nH\f"+ - "H\16H\u04ae\13H\3I\3I\3I\3I\3I\3I\3I\3I\3I\3I\5I\u04ba\nI\5I\u04bc\nI"+ - "\3I\3I\3I\3I\3I\3I\7I\u04c4\nI\fI\16I\u04c7\13I\3J\5J\u04ca\nJ\3J\3J\3"+ - "J\3J\3J\3J\5J\u04d2\nJ\3J\3J\3J\3J\3J\7J\u04d9\nJ\fJ\16J\u04dc\13J\3J"+ - "\3J\3J\5J\u04e1\nJ\3J\3J\3J\3J\3J\3J\5J\u04e9\nJ\3J\3J\3J\5J\u04ee\nJ"+ - "\3J\3J\3J\3J\3J\3J\3J\3J\7J\u04f8\nJ\fJ\16J\u04fb\13J\3J\3J\5J\u04ff\n"+ - "J\3J\5J\u0502\nJ\3J\3J\3J\3J\5J\u0508\nJ\3J\3J\5J\u050c\nJ\3J\3J\3J\5"+ - "J\u0511\nJ\3J\3J\3J\5J\u0516\nJ\3J\3J\3J\5J\u051b\nJ\3K\3K\3K\3K\5K\u0521"+ - "\nK\3K\3K\3K\3K\3K\3K\3K\3K\3K\3K\3K\3K\3K\3K\3K\3K\3K\3K\3K\7K\u0536"+ - "\nK\fK\16K\u0539\13K\3L\3L\3L\3L\6L\u053f\nL\rL\16L\u0540\3L\3L\5L\u0545"+ - "\nL\3L\3L\3L\3L\3L\6L\u054c\nL\rL\16L\u054d\3L\3L\5L\u0552\nL\3L\3L\3"+ - "L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\7L\u0562\nL\fL\16L\u0565\13L\5L\u0567"+ - "\nL\3L\3L\3L\3L\3L\3L\5L\u056f\nL\3L\3L\3L\3L\3L\3L\3L\5L\u0578\nL\3L"+ - "\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\6L\u058d\nL\rL"+ - "\16L\u058e\3L\3L\3L\3L\3L\3L\3L\3L\3L\5L\u059a\nL\3L\3L\3L\7L\u059f\n"+ - "L\fL\16L\u05a2\13L\5L\u05a4\nL\3L\3L\3L\3L\3L\3L\3L\5L\u05ad\nL\3L\3L"+ - "\5L\u05b1\nL\3L\3L\5L\u05b5\nL\3L\3L\3L\3L\3L\3L\3L\3L\6L\u05bf\nL\rL"+ - "\16L\u05c0\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3"+ - "L\3L\3L\3L\5L\u05da\nL\3L\3L\3L\3L\3L\5L\u05e1\nL\3L\5L\u05e4\nL\3L\3"+ - "L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\3L\5L\u05f3\nL\3L\3L\5L\u05f7\nL\3L\3"+ - "L\3L\3L\3L\3L\3L\3L\7L\u0601\nL\fL\16L\u0604\13L\3M\3M\3M\3M\3M\3M\3M"+ - "\3M\6M\u060e\nM\rM\16M\u060f\5M\u0612\nM\3N\3N\3O\3O\3P\3P\3P\5P\u061b"+ - "\nP\3Q\3Q\5Q\u061f\nQ\3R\3R\3R\6R\u0624\nR\rR\16R\u0625\3S\3S\3S\5S\u062b"+ - "\nS\3T\3T\3T\3T\3T\3U\5U\u0633\nU\3U\3U\3V\3V\3V\5V\u063a\nV\3W\3W\3W"+ - "\3W\3W\3W\3W\3W\3W\3W\3W\3W\3W\3W\3W\5W\u064b\nW\3W\3W\5W\u064f\nW\3W"+ - "\3W\3W\3W\5W\u0655\nW\3W\3W\3W\3W\5W\u065b\nW\3W\3W\3W\3W\3W\7W\u0662"+ - "\nW\fW\16W\u0665\13W\3W\5W\u0668\nW\5W\u066a\nW\3X\3X\3X\7X\u066f\nX\f"+ - "X\16X\u0672\13X\3Y\3Y\3Y\3Y\5Y\u0678\nY\3Y\5Y\u067b\nY\3Z\3Z\3Z\7Z\u0680"+ - "\nZ\fZ\16Z\u0683\13Z\3[\3[\5[\u0687\n[\3[\3[\3[\5[\u068c\n[\3[\5[\u068f"+ - "\n[\3\\\3\\\3\\\3\\\3\\\3]\3]\3]\3]\7]\u069a\n]\f]\16]\u069d\13]\3^\3"+ - "^\3^\3^\3_\3_\3_\3_\3_\3_\3_\3_\3_\3_\3_\7_\u06ae\n_\f_\16_\u06b1\13_"+ - "\3_\3_\3_\3_\3_\7_\u06b8\n_\f_\16_\u06bb\13_\5_\u06bd\n_\3_\3_\3_\3_\3"+ - "_\7_\u06c4\n_\f_\16_\u06c7\13_\5_\u06c9\n_\5_\u06cb\n_\3_\5_\u06ce\n_"+ - "\3_\5_\u06d1\n_\3`\3`\3`\3`\3`\3`\3`\3`\3`\3`\3`\3`\3`\3`\3`\3`\5`\u06e3"+ - "\n`\3a\3a\3a\3a\3a\3a\3a\5a\u06ec\na\3b\3b\3b\3b\5b\u06f2\nb\3c\3c\3c"+ - "\7c\u06f7\nc\fc\16c\u06fa\13c\3d\3d\3d\3e\3e\6e\u0701\ne\re\16e\u0702"+ - "\3e\5e\u0706\ne\3f\3f\3f\5f\u070b\nf\3g\3g\3g\3g\3g\3g\5g\u0713\ng\3h"+ - "\3h\3i\3i\5i\u0719\ni\3i\3i\3i\5i\u071e\ni\3i\3i\3i\5i\u0723\ni\3i\3i"+ - "\5i\u0727\ni\3i\3i\5i\u072b\ni\3i\3i\5i\u072f\ni\3i\3i\5i\u0733\ni\3i"+ - "\3i\5i\u0737\ni\3i\3i\5i\u073b\ni\3i\3i\5i\u073f\ni\3i\5i\u0742\ni\3j"+ - "\3j\3k\3k\3l\3l\3l\2\6\62\u0090\u0094\u0096m\2\4\6\b\n\f\16\20\22\24\26"+ - "\30\32\34\36 \"$&(*,.\60\62\64\668:<>@BDFHJLNPRTVXZ\\^`bdfhjlnprtvxz|"+ - "~\u0080\u0082\u0084\u0086\u0088\u008a\u008c\u008e\u0090\u0092\u0094\u0096"+ - "\u0098\u009a\u009c\u009e\u00a0\u00a2\u00a4\u00a6\u00a8\u00aa\u00ac\u00ae"+ - "\u00b0\u00b2\u00b4\u00b6\u00b8\u00ba\u00bc\u00be\u00c0\u00c2\u00c4\u00c6"+ - "\u00c8\u00ca\u00cc\u00ce\u00d0\u00d2\u00d4\u00d6\2$\7\2**\66\66YYff\u0090"+ - "\u0090\6\2TT{{\u00d9\u00d9\u00fe\u00fe\5\2TT\u00d9\u00d9\u00fe\u00fe\4"+ - "\2\31\31GG\4\2aa\u0082\u0082\4\299\u00cd\u00cd\4\2\20\20LL\4\2\u0127\u0127"+ - "\u0129\u0129\5\2\20\20\25\25\u00dd\u00dd\5\2\\\\\u00f7\u00f7\u0100\u0100"+ - "\4\2\u0119\u011a\u011e\u011e\4\2NN\u011b\u011d\4\2\u0119\u011a\u0121\u0121"+ - "\4\2;;=>\4\2$$\u00f9\u00f9\4\2rr\u00c5\u00c5\3\2\u00e5\u00e6\4\2\5\5g"+ - "g\4\2\5\5cc\5\2\35\35\u0085\u0085\u00f2\u00f2\3\2\u0111\u0118\4\2\\\\"+ - "\u00f7\u00f7\3\2\u0119\u011a\5\2\u0123\u0123\u0127\u0127\u0129\u0129\4"+ - "\2\u0096\u0096\u010e\u010e\6\2??pp\u0095\u0095\u00d0\u00d0\5\2pp\u0095"+ - "\u0095\u00d0\u00d0\4\2MM\u00ad\u00ad\4\2\u00a5\u00a5\u00de\u00de\4\2b"+ - "b\u00b4\u00b4\3\2\u0128\u0129\63\2\16\17\21\22\24\24\26\27\31\32\34\34"+ - "\36\"%%\'*,,.\64\66\669:?KMOSSU[^^`befiknnpsuvxz||\177\177\u0081\u0082"+ - "\u0084\u0084\u0087\u0099\u009b\u009b\u009e\u009f\u00a2\u00a3\u00a6\u00a6"+ - "\u00a8\u00a9\u00ab\u00b4\u00b6\u00be\u00c0\u00c7\u00c9\u00d1\u00d3\u00d6"+ - "\u00d8\u00dc\u00de\u00e7\u00e9\u00ed\u00f1\u00f1\u00f3\u00fd\u0101\u0104"+ - "\u0107\u0109\u010c\u010c\u010e\u010f\22\2\24\2488TThhww{{\u0080\u0080"+ - "\u0083\u0083\u0086\u0086\u009a\u009a\u00a0\u00a0\u00c8\u00c8\u00d3\u00d3"+ - "\u00d9\u00d9\u00fe\u00fe\u0106\u0106\23\2\16\23\25\679SUgivxz|\177\u0081"+ - "\u0082\u0084\u0085\u0087\u0099\u009b\u009f\u00a1\u00c7\u00c9\u00d2\u00d4"+ - "\u00d8\u00da\u00fd\u00ff\u0105\u0107\u010f\2\u0837\2\u00d8\3\2\2\2\4\u00f2"+ - "\3\2\2\2\6\u00f4\3\2\2\2\b\u010c\3\2\2\2\n\u010e\3\2\2\2\f\u0112\3\2\2"+ - "\2\16\u011e\3\2\2\2\20\u012b\3\2\2\2\22\u012e\3\2\2\2\24\u0132\3\2\2\2"+ - "\26\u0137\3\2\2\2\30\u0140\3\2\2\2\32\u014b\3\2\2\2\34\u015d\3\2\2\2\36"+ - "\u0160\3\2\2\2 \u016b\3\2\2\2\"\u017b\3\2\2\2$\u0181\3\2\2\2&\u0183\3"+ - "\2\2\2(\u018e\3\2\2\2*\u019f\3\2\2\2,\u01a6\3\2\2\2.\u01a8\3\2\2\2\60"+ - "\u01b8\3\2\2\2\62\u01e8\3\2\2\2\64\u020e\3\2\2\2\66\u0210\3\2\2\28\u0218"+ - "\3\2\2\2:\u0239\3\2\2\2<\u0267\3\2\2\2>\u027c\3\2\2\2@\u029c\3\2\2\2B"+ - "\u02a8\3\2\2\2D\u02ab\3\2\2\2F\u02ae\3\2\2\2H\u02c8\3\2\2\2J\u02ca\3\2"+ - "\2\2L\u0303\3\2\2\2N\u0307\3\2\2\2P\u0322\3\2\2\2R\u0326\3\2\2\2T\u0335"+ - "\3\2\2\2V\u0337\3\2\2\2X\u0355\3\2\2\2Z\u0357\3\2\2\2\\\u035e\3\2\2\2"+ - "^\u037e\3\2\2\2`\u0381\3\2\2\2b\u039b\3\2\2\2d\u03b5\3\2\2\2f\u03bb\3"+ - "\2\2\2h\u03bd\3\2\2\2j\u03dc\3\2\2\2l\u03de\3\2\2\2n\u03e2\3\2\2\2p\u03ea"+ - "\3\2\2\2r\u03f5\3\2\2\2t\u0411\3\2\2\2v\u0413\3\2\2\2x\u041e\3\2\2\2z"+ - "\u0434\3\2\2\2|\u0467\3\2\2\2~\u0469\3\2\2\2\u0080\u0471\3\2\2\2\u0082"+ - "\u047b\3\2\2\2\u0084\u0483\3\2\2\2\u0086\u0490\3\2\2\2\u0088\u049f\3\2"+ - "\2\2\u008a\u04a3\3\2\2\2\u008c\u04a5\3\2\2\2\u008e\u04a7\3\2\2\2\u0090"+ - "\u04bb\3\2\2\2\u0092\u051a\3\2\2\2\u0094\u0520\3\2\2\2\u0096\u05f6\3\2"+ - "\2\2\u0098\u0611\3\2\2\2\u009a\u0613\3\2\2\2\u009c\u0615\3\2\2\2\u009e"+ - "\u0617\3\2\2\2\u00a0\u061c\3\2\2\2\u00a2\u0623\3\2\2\2\u00a4\u0627\3\2"+ - "\2\2\u00a6\u062c\3\2\2\2\u00a8\u0632\3\2\2\2\u00aa\u0639\3\2\2\2\u00ac"+ - "\u0669\3\2\2\2\u00ae\u066b\3\2\2\2\u00b0\u0673\3\2\2\2\u00b2\u067c\3\2"+ - "\2\2\u00b4\u0684\3\2\2\2\u00b6\u0690\3\2\2\2\u00b8\u0695\3\2\2\2\u00ba"+ - "\u069e\3\2\2\2\u00bc\u06d0\3\2\2\2\u00be\u06e2\3\2\2\2\u00c0\u06eb\3\2"+ - "\2\2\u00c2\u06f1\3\2\2\2\u00c4\u06f3\3\2\2\2\u00c6\u06fb\3\2\2\2\u00c8"+ - "\u0705\3\2\2\2\u00ca\u070a\3\2\2\2\u00cc\u0712\3\2\2\2\u00ce\u0714\3\2"+ - "\2\2\u00d0\u0741\3\2\2\2\u00d2\u0743\3\2\2\2\u00d4\u0745\3\2\2\2\u00d6"+ - "\u0747\3\2\2\2\u00d8\u00dc\5\4\3\2\u00d9\u00db\7\3\2\2\u00da\u00d9\3\2"+ - "\2\2\u00db\u00de\3\2\2\2\u00dc\u00da\3\2\2\2\u00dc\u00dd\3\2\2\2\u00dd"+ - "\u00df\3\2\2\2\u00de\u00dc\3\2\2\2\u00df\u00e0\7\2\2\3\u00e0\3\3\2\2\2"+ - "\u00e1\u00e2\5\6\4\2\u00e2\u00e4\5\b\5\2\u00e3\u00e5\5\32\16\2\u00e4\u00e3"+ - "\3\2\2\2\u00e4\u00e5\3\2\2\2\u00e5\u00e6\3\2\2\2\u00e6\u00eb\5\34\17\2"+ - "\u00e7\u00e9\7\30\2\2\u00e8\u00e7\3\2\2\2\u00e8\u00e9\3\2\2\2\u00e9\u00ea"+ - "\3\2\2\2\u00ea\u00ec\5\24\13\2\u00eb\u00e8\3\2\2\2\u00eb\u00ec\3\2\2\2"+ - "\u00ec\u00f3\3\2\2\2\u00ed\u00ef\7W\2\2\u00ee\u00f0\t\2\2\2\u00ef\u00ee"+ - "\3\2\2\2\u00ef\u00f0\3\2\2\2\u00f0\u00f1\3\2\2\2\u00f1\u00f3\5\4\3\2\u00f2"+ - "\u00e1\3\2\2\2\u00f2\u00ed\3\2\2\2\u00f3\5\3\2\2\2\u00f4\u00f6\7\67\2"+ - "\2\u00f5\u00f7\7\u00ec\2\2\u00f6\u00f5\3\2\2\2\u00f6\u00f7\3\2\2\2\u00f7"+ - "\u00f9\3\2\2\2\u00f8\u00fa\7Z\2\2\u00f9\u00f8\3\2\2\2\u00f9\u00fa\3\2"+ - "\2\2\u00fa\u00fb\3\2\2\2\u00fb\u00ff\7\u00e8\2\2\u00fc\u00fd\7q\2\2\u00fd"+ - "\u00fe\7\u009c\2\2\u00fe\u0100\7V\2\2\u00ff\u00fc\3\2\2\2\u00ff\u0100"+ - "\3\2\2\2\u0100\u0101\3\2\2\2\u0101\u0102\5~@\2\u0102\7\3\2\2\2\u0103\u0104"+ - "\7\4\2\2\u0104\u0107\5\u00aeX\2\u0105\u0106\7\5\2\2\u0106\u0108\5\n\6"+ - "\2\u0107\u0105\3\2\2\2\u0107\u0108\3\2\2\2\u0108\u0109\3\2\2\2\u0109\u010a"+ - "\7\6\2\2\u010a\u010d\3\2\2\2\u010b\u010d\5\n\6\2\u010c\u0103\3\2\2\2\u010c"+ - "\u010b\3\2\2\2\u010d\t\3\2\2\2\u010e\u010f\7\u00b5\2\2\u010f\u0110\7\u0110"+ - "\2\2\u0110\u0111\5l\67\2\u0111\13\3\2\2\2\u0112\u0113\7)\2\2\u0113\u0114"+ - "\7 \2\2\u0114\u0118\5l\67\2\u0115\u0116\7\u00df\2\2\u0116\u0117\7 \2\2"+ - "\u0117\u0119\5p9\2\u0118\u0115\3\2\2\2\u0118\u0119\3\2\2\2\u0119\u011a"+ - "\3\2\2\2\u011a\u011b\7}\2\2\u011b\u011c\7\u0127\2\2\u011c\u011d\7\37\2"+ - "\2\u011d\r\3\2\2\2\u011e\u011f\7\u00dc\2\2\u011f\u0120\7 \2\2\u0120\u0121"+ - "\5l\67\2\u0121\u0124\7\u00a0\2\2\u0122\u0125\5&\24\2\u0123\u0125\5(\25"+ - "\2\u0124\u0122\3\2\2\2\u0124\u0123\3\2\2\2\u0125\u0129\3\2\2\2\u0126\u0127"+ - "\7\u00e2\2\2\u0127\u0128\7\30\2\2\u0128\u012a\7J\2\2\u0129\u0126\3\2\2"+ - "\2\u0129\u012a\3\2\2\2\u012a\17\3\2\2\2\u012b\u012c\7\u008d\2\2\u012c"+ - "\u012d\7\u0123\2\2\u012d\21\3\2\2\2\u012e\u012f\7/\2\2\u012f\u0130\7\u0123"+ - "\2\2\u0130\23\3\2\2\2\u0131\u0133\5\26\f\2\u0132\u0131\3\2\2\2\u0132\u0133"+ - "\3\2\2\2\u0133\u0134\3\2\2\2\u0134\u0135\5\62\32\2\u0135\u0136\5\60\31"+ - "\2\u0136\25\3\2\2\2\u0137\u0138\7\u010d\2\2\u0138\u013d\5\30\r\2\u0139"+ - "\u013a\7\5\2\2\u013a\u013c\5\30\r\2\u013b\u0139\3\2\2\2\u013c\u013f\3"+ - "\2\2\2\u013d\u013b\3\2\2\2\u013d\u013e\3\2\2\2\u013e\27\3\2\2\2\u013f"+ - "\u013d\3\2\2\2\u0140\u0142\5\u00c6d\2\u0141\u0143\5l\67\2\u0142\u0141"+ - "\3\2\2\2\u0142\u0143\3\2\2\2\u0143\u0145\3\2\2\2\u0144\u0146\7\30\2\2"+ - "\u0145\u0144\3\2\2\2\u0145\u0146\3\2\2\2\u0146\u0147\3\2\2\2\u0147\u0148"+ - "\7\4\2\2\u0148\u0149\5\24\13\2\u0149\u014a\7\6\2\2\u014a\31\3\2\2\2\u014b"+ - "\u014c\7\u0106\2\2\u014c\u014d\5~@\2\u014d\33\3\2\2\2\u014e\u014f\7\u00a3"+ - "\2\2\u014f\u015c\5\36\20\2\u0150\u0151\7\u00ae\2\2\u0151\u0152\7 \2\2"+ - "\u0152\u015c\5\u0084C\2\u0153\u015c\5\16\b\2\u0154\u015c\5\f\7\2\u0155"+ - "\u015c\5|?\2\u0156\u015c\5*\26\2\u0157\u015c\5\20\t\2\u0158\u015c\5\22"+ - "\n\2\u0159\u015a\7\u00eb\2\2\u015a\u015c\5\36\20\2\u015b\u014e\3\2\2\2"+ - "\u015b\u0150\3\2\2\2\u015b\u0153\3\2\2\2\u015b\u0154\3\2\2\2\u015b\u0155"+ - "\3\2\2\2\u015b\u0156\3\2\2\2\u015b\u0157\3\2\2\2\u015b\u0158\3\2\2\2\u015b"+ - "\u0159\3\2\2\2\u015c\u015f\3\2\2\2\u015d\u015b\3\2\2\2\u015d\u015e\3\2"+ - "\2\2\u015e\35\3\2\2\2\u015f\u015d\3\2\2\2\u0160\u0161\7\4\2\2\u0161\u0166"+ - "\5 \21\2\u0162\u0163\7\5\2\2\u0163\u0165\5 \21\2\u0164\u0162\3\2\2\2\u0165"+ - "\u0168\3\2\2\2\u0166\u0164\3\2\2\2\u0166\u0167\3\2\2\2\u0167\u0169\3\2"+ - "\2\2\u0168\u0166\3\2\2\2\u0169\u016a\7\6\2\2\u016a\37\3\2\2\2\u016b\u0170"+ - "\5\"\22\2\u016c\u016e\7\u0111\2\2\u016d\u016c\3\2\2\2\u016d\u016e\3\2"+ - "\2\2\u016e\u016f\3\2\2\2\u016f\u0171\5$\23\2\u0170\u016d\3\2\2\2\u0170"+ - "\u0171\3\2\2\2\u0171!\3\2\2\2\u0172\u0177\5\u00caf\2\u0173\u0174\7\7\2"+ - "\2\u0174\u0176\5\u00caf\2\u0175\u0173\3\2\2\2\u0176\u0179\3\2\2\2\u0177"+ - "\u0175\3\2\2\2\u0177\u0178\3\2\2\2\u0178\u017c\3\2\2\2\u0179\u0177\3\2"+ - "\2\2\u017a\u017c\7\u0123\2\2\u017b\u0172\3\2\2\2\u017b\u017a\3\2\2\2\u017c"+ - "#\3\2\2\2\u017d\u0182\7\u0127\2\2\u017e\u0182\7\u0129\2\2\u017f\u0182"+ - "\5\u009cO\2\u0180\u0182\7\u0123\2\2\u0181\u017d\3\2\2\2\u0181\u017e\3"+ - "\2\2\2\u0181\u017f\3\2\2\2\u0181\u0180\3\2\2\2\u0182%\3\2\2\2\u0183\u0184"+ - "\7\4\2\2\u0184\u0189\5\u0098M\2\u0185\u0186\7\5\2\2\u0186\u0188\5\u0098"+ - "M\2\u0187\u0185\3\2\2\2\u0188\u018b\3\2\2\2\u0189\u0187\3\2\2\2\u0189"+ - "\u018a\3\2\2\2\u018a\u018c\3\2\2\2\u018b\u0189\3\2\2\2\u018c\u018d\7\6"+ - "\2\2\u018d\'\3\2\2\2\u018e\u018f\7\4\2\2\u018f\u0194\5&\24\2\u0190\u0191"+ - "\7\5\2\2\u0191\u0193\5&\24\2\u0192\u0190\3\2\2\2\u0193\u0196\3\2\2\2\u0194"+ - "\u0192\3\2\2\2\u0194\u0195\3\2\2\2\u0195\u0197\3\2\2\2\u0196\u0194\3\2"+ - "\2\2\u0197\u0198\7\6\2\2\u0198)\3\2\2\2\u0199\u019a\7\u00e2\2\2\u019a"+ - "\u019b\7\30\2\2\u019b\u01a0\5,\27\2\u019c\u019d\7\u00e2\2\2\u019d\u019e"+ - "\7 \2\2\u019e\u01a0\5.\30\2\u019f\u0199\3\2\2\2\u019f\u019c\3\2\2\2\u01a0"+ - "+\3\2\2\2\u01a1\u01a2\7y\2\2\u01a2\u01a3\7\u0123\2\2\u01a3\u01a4\7\u00a8"+ - "\2\2\u01a4\u01a7\7\u0123\2\2\u01a5\u01a7\5\u00caf\2\u01a6\u01a1\3\2\2"+ - "\2\u01a6\u01a5\3\2\2\2\u01a7-\3\2\2\2\u01a8\u01ac\7\u0123\2\2\u01a9\u01aa"+ - "\7\u010d\2\2\u01aa\u01ab\7\u00d6\2\2\u01ab\u01ad\5\36\20\2\u01ac\u01a9"+ - "\3\2\2\2\u01ac\u01ad\3\2\2\2\u01ad/\3\2\2\2\u01ae\u01af\7\u00a5\2\2\u01af"+ - "\u01b0\7 \2\2\u01b0\u01b5\5\66\34\2\u01b1\u01b2\7\5\2\2\u01b2\u01b4\5"+ - "\66\34\2\u01b3\u01b1\3\2\2\2\u01b4\u01b7\3\2\2\2\u01b5\u01b3\3\2\2\2\u01b5"+ - "\u01b6\3\2\2\2\u01b6\u01b9\3\2\2\2\u01b7\u01b5\3\2\2\2\u01b8\u01ae\3\2"+ - "\2\2\u01b8\u01b9\3\2\2\2\u01b9\u01c4\3\2\2\2\u01ba\u01bb\7(\2\2\u01bb"+ - "\u01bc\7 \2\2\u01bc\u01c1\5\u008cG\2\u01bd\u01be\7\5\2\2\u01be\u01c0\5"+ - "\u008cG\2\u01bf\u01bd\3\2\2\2\u01c0\u01c3\3\2\2\2\u01c1\u01bf\3\2\2\2"+ - "\u01c1\u01c2\3\2\2\2\u01c2\u01c5\3\2\2\2\u01c3\u01c1\3\2\2\2\u01c4\u01ba"+ - "\3\2\2\2\u01c4\u01c5\3\2\2\2\u01c5\u01d0\3\2\2\2\u01c6\u01c7\7M\2\2\u01c7"+ - "\u01c8\7 \2\2\u01c8\u01cd\5\u008cG\2\u01c9\u01ca\7\5\2\2\u01ca\u01cc\5"+ - "\u008cG\2\u01cb\u01c9\3\2\2\2\u01cc\u01cf\3\2\2\2\u01cd\u01cb\3\2\2\2"+ - "\u01cd\u01ce\3\2\2\2\u01ce\u01d1\3\2\2\2\u01cf\u01cd\3\2\2\2\u01d0\u01c6"+ - "\3\2\2\2\u01d0\u01d1\3\2\2\2\u01d1\u01dc\3\2\2\2\u01d2\u01d3\7\u00de\2"+ - "\2\u01d3\u01d4\7 \2\2\u01d4\u01d9\5\66\34\2\u01d5\u01d6\7\5\2\2\u01d6"+ - "\u01d8\5\66\34\2\u01d7\u01d5\3\2\2\2\u01d8\u01db\3\2\2\2\u01d9\u01d7\3"+ - "\2\2\2\u01d9\u01da\3\2\2\2\u01da\u01dd\3\2\2\2\u01db\u01d9\3\2\2\2\u01dc"+ - "\u01d2\3\2\2\2\u01dc\u01dd\3\2\2\2\u01dd\u01df\3\2\2\2\u01de\u01e0\5\u00b8"+ - "]\2\u01df\u01de\3\2\2\2\u01df\u01e0\3\2\2\2\u01e0\u01e6\3\2\2\2\u01e1"+ - "\u01e4\7\u0088\2\2\u01e2\u01e5\7\20\2\2\u01e3\u01e5\5\u008cG\2\u01e4\u01e2"+ - "\3\2\2\2\u01e4\u01e3\3\2\2\2\u01e5\u01e7\3\2\2\2\u01e6\u01e1\3\2\2\2\u01e6"+ - "\u01e7\3\2\2\2\u01e7\61\3\2\2\2\u01e8\u01e9\b\32\1\2\u01e9\u01ea\5\64"+ - "\33\2\u01ea\u0202\3\2\2\2\u01eb\u01ec\f\5\2\2\u01ec\u01ed\6\32\3\2\u01ed"+ - "\u01ef\t\3\2\2\u01ee\u01f0\5^\60\2\u01ef\u01ee\3\2\2\2\u01ef\u01f0\3\2"+ - "\2\2\u01f0\u01f1\3\2\2\2\u01f1\u0201\5\62\32\6\u01f2\u01f3\f\4\2\2\u01f3"+ - "\u01f4\6\32\5\2\u01f4\u01f6\7{\2\2\u01f5\u01f7\5^\60\2\u01f6\u01f5\3\2"+ - "\2\2\u01f6\u01f7\3\2\2\2\u01f7\u01f8\3\2\2\2\u01f8\u0201\5\62\32\5\u01f9"+ - "\u01fa\f\3\2\2\u01fa\u01fb\6\32\7\2\u01fb\u01fd\t\4\2\2\u01fc\u01fe\5"+ - "^\60\2\u01fd\u01fc\3\2\2\2\u01fd\u01fe\3\2\2\2\u01fe\u01ff\3\2\2\2\u01ff"+ - "\u0201\5\62\32\4\u0200\u01eb\3\2\2\2\u0200\u01f2\3\2\2\2\u0200\u01f9\3"+ - "\2\2\2\u0201\u0204\3\2\2\2\u0202\u0200\3\2\2\2\u0202\u0203\3\2\2\2\u0203"+ - "\63\3\2\2\2\u0204\u0202\3\2\2\2\u0205\u020f\5<\37\2\u0206\u020f\58\35"+ - "\2\u0207\u0208\7\u00e8\2\2\u0208\u020f\5~@\2\u0209\u020f\5v<\2\u020a\u020b"+ - "\7\4\2\2\u020b\u020c\5\24\13\2\u020c\u020d\7\6\2\2\u020d\u020f\3\2\2\2"+ - "\u020e\u0205\3\2\2\2\u020e\u0206\3\2\2\2\u020e\u0207\3\2\2\2\u020e\u0209"+ - "\3\2\2\2\u020e\u020a\3\2\2\2\u020f\65\3\2\2\2\u0210\u0212\5\u008cG\2\u0211"+ - "\u0213\t\5\2\2\u0212\u0211\3\2\2\2\u0212\u0213\3\2\2\2\u0213\u0216\3\2"+ - "\2\2\u0214\u0215\7\u009e\2\2\u0215\u0217\t\6\2\2\u0216\u0214\3\2\2\2\u0216"+ - "\u0217\3\2\2\2\u0217\67\3\2\2\2\u0218\u021a\5J&\2\u0219\u021b\5:\36\2"+ - "\u021a\u0219\3\2\2\2\u021b\u021c\3\2\2\2\u021c\u021a\3\2\2\2\u021c\u021d"+ - "\3\2\2\2\u021d9\3\2\2\2\u021e\u0220\5> \2\u021f\u0221\5B\"\2\u0220\u021f"+ - "\3\2\2\2\u0220\u0221\3\2\2\2\u0221\u0222\3\2\2\2\u0222\u0223\5\60\31\2"+ - "\u0223\u023a\3\2\2\2\u0224\u0228\5@!\2\u0225\u0227\5\\/\2\u0226\u0225"+ - "\3\2\2\2\u0227\u022a\3\2\2\2\u0228\u0226\3\2\2\2\u0228\u0229\3\2\2\2\u0229"+ - "\u022c\3\2\2\2\u022a\u0228\3\2\2\2\u022b\u022d\5B\"\2\u022c\u022b\3\2"+ - "\2\2\u022c\u022d\3\2\2\2\u022d\u022f\3\2\2\2\u022e\u0230\5L\'\2\u022f"+ - "\u022e\3\2\2\2\u022f\u0230\3\2\2\2\u0230\u0232\3\2\2\2\u0231\u0233\5D"+ - "#\2\u0232\u0231\3\2\2\2\u0232\u0233\3\2\2\2\u0233\u0235\3\2\2\2\u0234"+ - "\u0236\5\u00b8]\2\u0235\u0234\3\2\2\2\u0235\u0236\3\2\2\2\u0236\u0237"+ - "\3\2\2\2\u0237\u0238\5\60\31\2\u0238\u023a\3\2\2\2\u0239\u021e\3\2\2\2"+ - "\u0239\u0224\3\2\2\2\u023a;\3\2\2\2\u023b\u023d\5> \2\u023c\u023e\5J&"+ - "\2\u023d\u023c\3\2\2\2\u023d\u023e\3\2\2\2\u023e\u0242\3\2\2\2\u023f\u0241"+ - "\5\\/\2\u0240\u023f\3\2\2\2\u0241\u0244\3\2\2\2\u0242\u0240\3\2\2\2\u0242"+ - "\u0243\3\2\2\2\u0243\u0246\3\2\2\2\u0244\u0242\3\2\2\2\u0245\u0247\5B"+ - "\"\2\u0246\u0245\3\2\2\2\u0246\u0247\3\2\2\2\u0247\u0249\3\2\2\2\u0248"+ - "\u024a\5L\'\2\u0249\u0248\3\2\2\2\u0249\u024a\3\2\2\2\u024a\u024c\3\2"+ - "\2\2\u024b\u024d\5D#\2\u024c\u024b\3\2\2\2\u024c\u024d\3\2\2\2\u024d\u024f"+ - "\3\2\2\2\u024e\u0250\5\u00b8]\2\u024f\u024e\3\2\2\2\u024f\u0250\3\2\2"+ - "\2\u0250\u0268\3\2\2\2\u0251\u0253\5@!\2\u0252\u0254\5J&\2\u0253\u0252"+ - "\3\2\2\2\u0253\u0254\3\2\2\2\u0254\u0258\3\2\2\2\u0255\u0257\5\\/\2\u0256"+ - "\u0255\3\2\2\2\u0257\u025a\3\2\2\2\u0258\u0256\3\2\2\2\u0258\u0259\3\2"+ - "\2\2\u0259\u025c\3\2\2\2\u025a\u0258\3\2\2\2\u025b\u025d\5B\"\2\u025c"+ - "\u025b\3\2\2\2\u025c\u025d\3\2\2\2\u025d\u025f\3\2\2\2\u025e\u0260\5L"+ - "\'\2\u025f\u025e\3\2\2\2\u025f\u0260\3\2\2\2\u0260\u0262\3\2\2\2\u0261"+ - "\u0263\5D#\2\u0262\u0261\3\2\2\2\u0262\u0263\3\2\2\2\u0263\u0265\3\2\2"+ - "\2\u0264\u0266\5\u00b8]\2\u0265\u0264\3\2\2\2\u0265\u0266\3\2\2\2\u0266"+ - "\u0268\3\2\2\2\u0267\u023b\3\2\2\2\u0267\u0251\3\2\2\2\u0268=\3\2\2\2"+ - "\u0269\u026a\7\u00d2\2\2\u026a\u026b\7\u00f5\2\2\u026b\u026d\7\4\2\2\u026c"+ - "\u026e\5^\60\2\u026d\u026c\3\2\2\2\u026d\u026e\3\2\2\2\u026e\u026f\3\2"+ - "\2\2\u026f\u0270\5\u008eH\2\u0270\u0271\7\6\2\2\u0271\u027d\3\2\2\2\u0272"+ - "\u0274\7\u0092\2\2\u0273\u0275\5^\60\2\u0274\u0273\3\2\2\2\u0274\u0275"+ - "\3\2\2\2\u0275\u0276\3\2\2\2\u0276\u027d\5\u008eH\2\u0277\u0279\7\u00be"+ - "\2\2\u0278\u027a\5^\60\2\u0279\u0278\3\2\2\2\u0279\u027a\3\2\2\2\u027a"+ - "\u027b\3\2\2\2\u027b\u027d\5\u008eH\2\u027c\u0269\3\2\2\2\u027c\u0272"+ - "\3\2\2\2\u027c\u0277\3\2\2\2\u027d\u027f\3\2\2\2\u027e\u0280\5|?\2\u027f"+ - "\u027e\3\2\2\2\u027f\u0280\3\2\2\2\u0280\u0283\3\2\2\2\u0281\u0282\7\u00bc"+ - "\2\2\u0282\u0284\7\u0123\2\2\u0283\u0281\3\2\2\2\u0283\u0284\3\2\2\2\u0284"+ - "\u0285\3\2\2\2\u0285\u0286\7\u0106\2\2\u0286\u0293\7\u0123\2\2\u0287\u0291"+ - "\7\30\2\2\u0288\u0292\5n8\2\u0289\u0292\5\u00aeX\2\u028a\u028d\7\4\2\2"+ - "\u028b\u028e\5n8\2\u028c\u028e\5\u00aeX\2\u028d\u028b\3\2\2\2\u028d\u028c"+ - "\3\2\2\2\u028e\u028f\3\2\2\2\u028f\u0290\7\6\2\2\u0290\u0292\3\2\2\2\u0291"+ - "\u0288\3\2\2\2\u0291\u0289\3\2\2\2\u0291\u028a\3\2\2\2\u0292\u0294\3\2"+ - "\2\2\u0293\u0287\3\2\2\2\u0293\u0294\3\2\2\2\u0294\u0296\3\2\2\2\u0295"+ - "\u0297\5|?\2\u0296\u0295\3\2\2\2\u0296\u0297\3\2\2\2\u0297\u029a\3\2\2"+ - "\2\u0298\u0299\7\u00bb\2\2\u0299\u029b\7\u0123\2\2\u029a\u0298\3\2\2\2"+ - "\u029a\u029b\3\2\2\2\u029b?\3\2\2\2\u029c\u02a0\7\u00d2\2\2\u029d\u029f"+ - "\5F$\2\u029e\u029d\3\2\2\2\u029f\u02a2\3\2\2\2\u02a0\u029e\3\2\2\2\u02a0"+ - "\u02a1\3\2\2\2\u02a1\u02a4\3\2\2\2\u02a2\u02a0\3\2\2\2\u02a3\u02a5\5^"+ - "\60\2\u02a4\u02a3\3\2\2\2\u02a4\u02a5\3\2\2\2\u02a5\u02a6\3\2\2\2\u02a6"+ - "\u02a7\5\u0082B\2\u02a7A\3\2\2\2\u02a8\u02a9\7\u010b\2\2\u02a9\u02aa\5"+ - "\u0090I\2\u02aaC\3\2\2\2\u02ab\u02ac\7o\2\2\u02ac\u02ad\5\u0090I\2\u02ad"+ - "E\3\2\2\2\u02ae\u02af\7\b\2\2\u02af\u02b6\5H%\2\u02b0\u02b2\7\5\2\2\u02b1"+ - "\u02b0\3\2\2\2\u02b1\u02b2\3\2\2\2\u02b2\u02b3\3\2\2\2\u02b3\u02b5\5H"+ - "%\2\u02b4\u02b1\3\2\2\2\u02b5\u02b8\3\2\2\2\u02b6\u02b4\3\2\2\2\u02b6"+ - "\u02b7\3\2\2\2\u02b7\u02b9\3\2\2\2\u02b8\u02b6\3\2\2\2\u02b9\u02ba\7\t"+ - "\2\2\u02baG\3\2\2\2\u02bb\u02c9\5\u00caf\2\u02bc\u02bd\5\u00caf\2\u02bd"+ - "\u02be\7\4\2\2\u02be\u02c3\5\u0096L\2\u02bf\u02c0\7\5\2\2\u02c0\u02c2"+ - "\5\u0096L\2\u02c1\u02bf\3\2\2\2\u02c2\u02c5\3\2\2\2\u02c3\u02c1\3\2\2"+ - "\2\u02c3\u02c4\3\2\2\2\u02c4\u02c6\3\2\2\2\u02c5\u02c3\3\2\2\2\u02c6\u02c7"+ - "\7\6\2\2\u02c7\u02c9\3\2\2\2\u02c8\u02bb\3\2\2\2\u02c8\u02bc\3\2\2\2\u02c9"+ - "I\3\2\2\2\u02ca\u02cb\7g\2\2\u02cb\u02d0\5`\61\2\u02cc\u02cd\7\5\2\2\u02cd"+ - "\u02cf\5`\61\2\u02ce\u02cc\3\2\2\2\u02cf\u02d2\3\2\2\2\u02d0\u02ce\3\2"+ - "\2\2\u02d0\u02d1\3\2\2\2\u02d1\u02d6\3\2\2\2\u02d2\u02d0\3\2\2\2\u02d3"+ - "\u02d5\5\\/\2\u02d4\u02d3\3\2\2\2\u02d5\u02d8\3\2\2\2\u02d6\u02d4\3\2"+ - "\2\2\u02d6\u02d7\3\2\2\2\u02d7\u02da\3\2\2\2\u02d8\u02d6\3\2\2\2\u02d9"+ - "\u02db\5V,\2\u02da\u02d9\3\2\2\2\u02da\u02db\3\2\2\2\u02dbK\3\2\2\2\u02dc"+ - "\u02dd\7m\2\2\u02dd\u02de\7 \2\2\u02de\u02e3\5N(\2\u02df\u02e0\7\5\2\2"+ - "\u02e0\u02e2\5N(\2\u02e1\u02df\3\2\2\2\u02e2\u02e5\3\2\2\2\u02e3\u02e1"+ - "\3\2\2\2\u02e3\u02e4\3\2\2\2\u02e4\u0304\3\2\2\2\u02e5\u02e3\3\2\2\2\u02e6"+ - "\u02e7\7m\2\2\u02e7\u02e8\7 \2\2\u02e8\u02ed\5\u008cG\2\u02e9\u02ea\7"+ - "\5\2\2\u02ea\u02ec\5\u008cG\2\u02eb\u02e9\3\2\2\2\u02ec\u02ef\3\2\2\2"+ - "\u02ed\u02eb\3\2\2\2\u02ed\u02ee\3\2\2\2\u02ee\u0301\3\2\2\2\u02ef\u02ed"+ - "\3\2\2\2\u02f0\u02f1\7\u010d\2\2\u02f1\u0302\7\u00cd\2\2\u02f2\u02f3\7"+ - "\u010d\2\2\u02f3\u0302\79\2\2\u02f4\u02f5\7n\2\2\u02f5\u02f6\7\u00da\2"+ - "\2\u02f6\u02f7\7\4\2\2\u02f7\u02fc\5T+\2\u02f8\u02f9\7\5\2\2\u02f9\u02fb"+ - "\5T+\2\u02fa\u02f8\3\2\2\2\u02fb\u02fe\3\2\2\2\u02fc\u02fa\3\2\2\2\u02fc"+ - "\u02fd\3\2\2\2\u02fd\u02ff\3\2\2\2\u02fe\u02fc\3\2\2\2\u02ff\u0300\7\6"+ - "\2\2\u0300\u0302\3\2\2\2\u0301\u02f0\3\2\2\2\u0301\u02f2\3\2\2\2\u0301"+ - "\u02f4\3\2\2\2\u0301\u0302\3\2\2\2\u0302\u0304\3\2\2\2\u0303\u02dc\3\2"+ - "\2\2\u0303\u02e6\3\2\2\2\u0304M\3\2\2\2\u0305\u0308\5P)\2\u0306\u0308"+ - "\5\u008cG\2\u0307\u0305\3\2\2\2\u0307\u0306\3\2\2\2\u0308O\3\2\2\2\u0309"+ - "\u030a\t\7\2\2\u030a\u030b\7\4\2\2\u030b\u0310\5T+\2\u030c\u030d\7\5\2"+ - "\2\u030d\u030f\5T+\2\u030e\u030c\3\2\2\2\u030f\u0312\3\2\2\2\u0310\u030e"+ - "\3\2\2\2\u0310\u0311\3\2\2\2\u0311\u0313\3\2\2\2\u0312\u0310\3\2\2\2\u0313"+ - "\u0314\7\6\2\2\u0314\u0323\3\2\2\2\u0315\u0316\7n\2\2\u0316\u0317\7\u00da"+ - "\2\2\u0317\u0318\7\4\2\2\u0318\u031d\5R*\2\u0319\u031a\7\5\2\2\u031a\u031c"+ - "\5R*\2\u031b\u0319\3\2\2\2\u031c\u031f\3\2\2\2\u031d\u031b\3\2\2\2\u031d"+ - "\u031e\3\2\2\2\u031e\u0320\3\2\2\2\u031f\u031d\3\2\2\2\u0320\u0321\7\6"+ - "\2\2\u0321\u0323\3\2\2\2\u0322\u0309\3\2\2\2\u0322\u0315\3\2\2\2\u0323"+ - "Q\3\2\2\2\u0324\u0327\5P)\2\u0325\u0327\5T+\2\u0326\u0324\3\2\2\2\u0326"+ - "\u0325\3\2\2\2\u0327S\3\2\2\2\u0328\u0331\7\4\2\2\u0329\u032e\5\u008c"+ - "G\2\u032a\u032b\7\5\2\2\u032b\u032d\5\u008cG\2\u032c\u032a\3\2\2\2\u032d"+ - "\u0330\3\2\2\2\u032e\u032c\3\2\2\2\u032e\u032f\3\2\2\2\u032f\u0332\3\2"+ - "\2\2\u0330\u032e\3\2\2\2\u0331\u0329\3\2\2\2\u0331\u0332\3\2\2\2\u0332"+ - "\u0333\3\2\2\2\u0333\u0336\7\6\2\2\u0334\u0336\5\u008cG\2\u0335\u0328"+ - "\3\2\2\2\u0335\u0334\3\2\2\2\u0336U\3\2\2\2\u0337\u0338\7\u00b1\2\2\u0338"+ - "\u0339\7\4\2\2\u0339\u033a\5\u0082B\2\u033a\u033b\7c\2\2\u033b\u033c\5"+ - "X-\2\u033c\u033d\7t\2\2\u033d\u033e\7\4\2\2\u033e\u0343\5Z.\2\u033f\u0340"+ - "\7\5\2\2\u0340\u0342\5Z.\2\u0341\u033f\3\2\2\2\u0342\u0345\3\2\2\2\u0343"+ - "\u0341\3\2\2\2\u0343\u0344\3\2\2\2\u0344\u0346\3\2\2\2\u0345\u0343\3\2"+ - "\2\2\u0346\u0347\7\6\2\2\u0347\u0348\7\6\2\2\u0348W\3\2\2\2\u0349\u0356"+ - "\5\u00caf\2\u034a\u034b\7\4\2\2\u034b\u0350\5\u00caf\2\u034c\u034d\7\5"+ - "\2\2\u034d\u034f\5\u00caf\2\u034e\u034c\3\2\2\2\u034f\u0352\3\2\2\2\u0350"+ - "\u034e\3\2\2\2\u0350\u0351\3\2\2\2\u0351\u0353\3\2\2\2\u0352\u0350\3\2"+ - "\2\2\u0353\u0354\7\6\2\2\u0354\u0356\3\2\2\2\u0355\u0349\3\2\2\2\u0355"+ - "\u034a\3\2\2\2\u0356Y\3\2\2\2\u0357\u035c\5\u008cG\2\u0358\u035a\7\30"+ - "\2\2\u0359\u0358\3\2\2\2\u0359\u035a\3\2\2\2\u035a\u035b\3\2\2\2\u035b"+ - "\u035d\5\u00caf\2\u035c\u0359\3\2\2\2\u035c\u035d\3\2\2\2\u035d[\3\2\2"+ - "\2\u035e\u035f\7\u0083\2\2\u035f\u0361\7\u0108\2\2\u0360\u0362\7\u00a7"+ - "\2\2\u0361\u0360\3\2\2\2\u0361\u0362\3\2\2\2\u0362\u0363\3\2\2\2\u0363"+ - "\u0364\5\u00c4c\2\u0364\u036d\7\4\2\2\u0365\u036a\5\u008cG\2\u0366\u0367"+ - "\7\5\2\2\u0367\u0369\5\u008cG\2\u0368\u0366\3\2\2\2\u0369\u036c\3\2\2"+ - "\2\u036a\u0368\3\2\2\2\u036a\u036b\3\2\2\2\u036b\u036e\3\2\2\2\u036c\u036a"+ - "\3\2\2\2\u036d\u0365\3\2\2\2\u036d\u036e\3\2\2\2\u036e\u036f\3\2\2\2\u036f"+ - "\u0370\7\6\2\2\u0370\u037c\5\u00caf\2\u0371\u0373\7\30\2\2\u0372\u0371"+ - "\3\2\2\2\u0372\u0373\3\2\2\2\u0373\u0374\3\2\2\2\u0374\u0379\5\u00caf"+ - "\2\u0375\u0376\7\5\2\2\u0376\u0378\5\u00caf\2\u0377\u0375\3\2\2\2\u0378"+ - "\u037b\3\2\2\2\u0379\u0377\3\2\2\2\u0379\u037a\3\2\2\2\u037a\u037d\3\2"+ - "\2\2\u037b\u0379\3\2\2\2\u037c\u0372\3\2\2\2\u037c\u037d\3\2\2\2\u037d"+ - "]\3\2\2\2\u037e\u037f\t\b\2\2\u037f_\3\2\2\2\u0380\u0382\7\u0083\2\2\u0381"+ - "\u0380\3\2\2\2\u0381\u0382\3\2\2\2\u0382\u0383\3\2\2\2\u0383\u0387\5t"+ - ";\2\u0384\u0386\5b\62\2\u0385\u0384\3\2\2\2\u0386\u0389\3\2\2\2\u0387"+ - "\u0385\3\2\2\2\u0387\u0388\3\2\2\2\u0388a\3\2\2\2\u0389\u0387\3\2\2\2"+ - "\u038a\u038b\5d\63\2\u038b\u038d\7\u0080\2\2\u038c\u038e\7\u0083\2\2\u038d"+ - "\u038c\3\2\2\2\u038d\u038e\3\2\2\2\u038e\u038f\3\2\2\2\u038f\u0391\5t"+ - ";\2\u0390\u0392\5f\64\2\u0391\u0390\3\2\2\2\u0391\u0392\3\2\2\2\u0392"+ - "\u039c\3\2\2\2\u0393\u0394\7\u009a\2\2\u0394\u0395\5d\63\2\u0395\u0397"+ - "\7\u0080\2\2\u0396\u0398\7\u0083\2\2\u0397\u0396\3\2\2\2\u0397\u0398\3"+ - "\2\2\2\u0398\u0399\3\2\2\2\u0399\u039a\5t;\2\u039a\u039c\3\2\2\2\u039b"+ - "\u038a\3\2\2\2\u039b\u0393\3\2\2\2\u039cc\3\2\2\2\u039d\u039f\7w\2\2\u039e"+ - "\u039d\3\2\2\2\u039e\u039f\3\2\2\2\u039f\u03b6\3\2\2\2\u03a0\u03b6\78"+ - "\2\2\u03a1\u03a3\7\u0086\2\2\u03a2\u03a4\7\u00a7\2\2\u03a3\u03a2\3\2\2"+ - "\2\u03a3\u03a4\3\2\2\2\u03a4\u03b6\3\2\2\2\u03a5\u03a7\7\u0086\2\2\u03a6"+ - "\u03a5\3\2\2\2\u03a6\u03a7\3\2\2\2\u03a7\u03a8\3\2\2\2\u03a8\u03b6\7\u00d3"+ - "\2\2\u03a9\u03ab\7\u00c8\2\2\u03aa\u03ac\7\u00a7\2\2\u03ab\u03aa\3\2\2"+ - "\2\u03ab\u03ac\3\2\2\2\u03ac\u03b6\3\2\2\2\u03ad\u03af\7h\2\2\u03ae\u03b0"+ - "\7\u00a7\2\2\u03af\u03ae\3\2\2\2\u03af\u03b0\3\2\2\2\u03b0\u03b6\3\2\2"+ - "\2\u03b1\u03b3\7\u0086\2\2\u03b2\u03b1\3\2\2\2\u03b2\u03b3\3\2\2\2\u03b3"+ - "\u03b4\3\2\2\2\u03b4\u03b6\7\24\2\2\u03b5\u039e\3\2\2\2\u03b5\u03a0\3"+ - "\2\2\2\u03b5\u03a1\3\2\2\2\u03b5\u03a6\3\2\2\2\u03b5\u03a9\3\2\2\2\u03b5"+ - "\u03ad\3\2\2\2\u03b5\u03b2\3\2\2\2\u03b6e\3\2\2\2\u03b7\u03b8\7\u00a0"+ - "\2\2\u03b8\u03bc\5\u0090I\2\u03b9\u03ba\7\u0106\2\2\u03ba\u03bc\5l\67"+ - "\2\u03bb\u03b7\3\2\2\2\u03bb\u03b9\3\2\2\2\u03bcg\3\2\2\2\u03bd\u03be"+ - "\7\u00ea\2\2\u03be\u03c0\7\4\2\2\u03bf\u03c1\5j\66\2\u03c0\u03bf\3\2\2"+ - "\2\u03c0\u03c1\3\2\2\2\u03c1\u03c2\3\2\2\2\u03c2\u03c3\7\6\2\2\u03c3i"+ - "\3\2\2\2\u03c4\u03c6\7\u011a\2\2\u03c5\u03c4\3\2\2\2\u03c5\u03c6\3\2\2"+ - "\2\u03c6\u03c7\3\2\2\2\u03c7\u03c8\t\t\2\2\u03c8\u03dd\7\u00b0\2\2\u03c9"+ - "\u03ca\5\u008cG\2\u03ca\u03cb\7\u00cf\2\2\u03cb\u03dd\3\2\2\2\u03cc\u03cd"+ - "\7\36\2\2\u03cd\u03ce\7\u0127\2\2\u03ce\u03cf\7\u00a6\2\2\u03cf\u03d0"+ - "\7\u009f\2\2\u03d0\u03d9\7\u0127\2\2\u03d1\u03d7\7\u00a0\2\2\u03d2\u03d8"+ - "\5\u00caf\2\u03d3\u03d4\5\u00c4c\2\u03d4\u03d5\7\4\2\2\u03d5\u03d6\7\6"+ - "\2\2\u03d6\u03d8\3\2\2\2\u03d7\u03d2\3\2\2\2\u03d7\u03d3\3\2\2\2\u03d8"+ - "\u03da\3\2\2\2\u03d9\u03d1\3\2\2\2\u03d9\u03da\3\2\2\2\u03da\u03dd\3\2"+ - "\2\2\u03db\u03dd\5\u008cG\2\u03dc\u03c5\3\2\2\2\u03dc\u03c9\3\2\2\2\u03dc"+ - "\u03cc\3\2\2\2\u03dc\u03db\3\2\2\2\u03ddk\3\2\2\2\u03de\u03df\7\4\2\2"+ - "\u03df\u03e0\5n8\2\u03e0\u03e1\7\6\2\2\u03e1m\3\2\2\2\u03e2\u03e7\5\u00c6"+ - "d\2\u03e3\u03e4\7\5\2\2\u03e4\u03e6\5\u00c6d\2\u03e5\u03e3\3\2\2\2\u03e6"+ - "\u03e9\3\2\2\2\u03e7\u03e5\3\2\2\2\u03e7\u03e8\3\2\2\2\u03e8o\3\2\2\2"+ - "\u03e9\u03e7\3\2\2\2\u03ea\u03eb\7\4\2\2\u03eb\u03f0\5r:\2\u03ec\u03ed"+ - "\7\5\2\2\u03ed\u03ef\5r:\2\u03ee\u03ec\3\2\2\2\u03ef\u03f2\3\2\2\2\u03f0"+ - "\u03ee\3\2\2\2\u03f0\u03f1\3\2\2\2\u03f1\u03f3\3\2\2\2\u03f2\u03f0\3\2"+ - "\2\2\u03f3\u03f4\7\6\2\2\u03f4q\3\2\2\2\u03f5\u03f7\5\u00c6d\2\u03f6\u03f8"+ - "\t\5\2\2\u03f7\u03f6\3\2\2\2\u03f7\u03f8\3\2\2\2\u03f8s\3\2\2\2\u03f9"+ - "\u03fb\5~@\2\u03fa\u03fc\5h\65\2\u03fb\u03fa\3\2\2\2\u03fb\u03fc\3\2\2"+ - "\2\u03fc\u03fd\3\2\2\2\u03fd\u03fe\5z>\2\u03fe\u0412\3\2\2\2\u03ff\u0400"+ - "\7\4\2\2\u0400\u0401\5\24\13\2\u0401\u0403\7\6\2\2\u0402\u0404\5h\65\2"+ - "\u0403\u0402\3\2\2\2\u0403\u0404\3\2\2\2\u0404\u0405\3\2\2\2\u0405\u0406"+ - "\5z>\2\u0406\u0412\3\2\2\2\u0407\u0408\7\4\2\2\u0408\u0409\5`\61\2\u0409"+ - "\u040b\7\6\2\2\u040a\u040c\5h\65\2\u040b\u040a\3\2\2\2\u040b\u040c\3\2"+ - "\2\2\u040c\u040d\3\2\2\2\u040d\u040e\5z>\2\u040e\u0412\3\2\2\2\u040f\u0412"+ - "\5v<\2\u0410\u0412\5x=\2\u0411\u03f9\3\2\2\2\u0411\u03ff\3\2\2\2\u0411"+ - "\u0407\3\2\2\2\u0411\u040f\3\2\2\2\u0411\u0410\3\2\2\2\u0412u\3\2\2\2"+ - "\u0413\u0414\7\u0107\2\2\u0414\u0419\5\u008cG\2\u0415\u0416\7\5\2\2\u0416"+ - "\u0418\5\u008cG\2\u0417\u0415\3\2\2\2\u0418\u041b\3\2\2\2\u0419\u0417"+ - "\3\2\2\2\u0419\u041a\3\2\2\2\u041a\u041c\3\2\2\2\u041b\u0419\3\2\2\2\u041c"+ - "\u041d\5z>\2\u041dw\3\2\2\2\u041e\u041f\5\u00c2b\2\u041f\u0428\7\4\2\2"+ - "\u0420\u0425\5\u008cG\2\u0421\u0422\7\5\2\2\u0422\u0424\5\u008cG\2\u0423"+ - "\u0421\3\2\2\2\u0424\u0427\3\2\2\2\u0425\u0423\3\2\2\2\u0425\u0426\3\2"+ - "\2\2\u0426\u0429\3\2\2\2\u0427\u0425\3\2\2\2\u0428\u0420\3\2\2\2\u0428"+ - "\u0429\3\2\2\2\u0429\u042a\3\2\2\2\u042a\u042b\7\6\2\2\u042b\u042c\5z"+ - ">\2\u042cy\3\2\2\2\u042d\u042f\7\30\2\2\u042e\u042d\3\2\2\2\u042e\u042f"+ - "\3\2\2\2\u042f\u0430\3\2\2\2\u0430\u0432\5\u00ccg\2\u0431\u0433\5l\67"+ - "\2\u0432\u0431\3\2\2\2\u0432\u0433\3\2\2\2\u0433\u0435\3\2\2\2\u0434\u042e"+ - "\3\2\2\2\u0434\u0435\3\2\2\2\u0435{\3\2\2\2\u0436\u0437\7\u00ce\2\2\u0437"+ - "\u0438\7e\2\2\u0438\u0439\7\u00d5\2\2\u0439\u043d\7\u0123\2\2\u043a\u043b"+ - "\7\u010d\2\2\u043b\u043c\7\u00d6\2\2\u043c\u043e\5\36\20\2\u043d\u043a"+ - "\3\2\2\2\u043d\u043e\3\2\2\2\u043e\u0468\3\2\2\2\u043f\u0440\7\u00ce\2"+ - "\2\u0440\u0441\7e\2\2\u0441\u044b\7F\2\2\u0442\u0443\7^\2\2\u0443\u0444"+ - "\7\u00ed\2\2\u0444\u0445\7 \2\2\u0445\u0449\7\u0123\2\2\u0446\u0447\7"+ - "S\2\2\u0447\u0448\7 \2\2\u0448\u044a\7\u0123\2\2\u0449\u0446\3\2\2\2\u0449"+ - "\u044a\3\2\2\2\u044a\u044c\3\2\2\2\u044b\u0442\3\2\2\2\u044b\u044c\3\2"+ - "\2\2\u044c\u0452\3\2\2\2\u044d\u044e\7,\2\2\u044e\u044f\7\177\2\2\u044f"+ - "\u0450\7\u00ed\2\2\u0450\u0451\7 \2\2\u0451\u0453\7\u0123\2\2\u0452\u044d"+ - "\3\2\2\2\u0452\u0453\3\2\2\2\u0453\u0459\3\2\2\2\u0454\u0455\7\u0092\2"+ - "\2\u0455\u0456\7\u0081\2\2\u0456\u0457\7\u00ed\2\2\u0457\u0458\7 \2\2"+ - "\u0458\u045a\7\u0123\2\2\u0459\u0454\3\2\2\2\u0459\u045a\3\2\2\2\u045a"+ - "\u045f\3\2\2\2\u045b\u045c\7\u0089\2\2\u045c\u045d\7\u00ed\2\2\u045d\u045e"+ - "\7 \2\2\u045e\u0460\7\u0123\2\2\u045f\u045b\3\2\2\2\u045f\u0460\3\2\2"+ - "\2\u0460\u0465\3\2\2\2\u0461\u0462\7\u009d\2\2\u0462\u0463\7D\2\2\u0463"+ - "\u0464\7\30\2\2\u0464\u0466\7\u0123\2\2\u0465\u0461\3\2\2\2\u0465\u0466"+ - "\3\2\2\2\u0466\u0468\3\2\2\2\u0467\u0436\3\2\2\2\u0467\u043f\3\2\2\2\u0468"+ - "}\3\2\2\2\u0469\u046e\5\u00c6d\2\u046a\u046b\7\7\2\2\u046b\u046d\5\u00c6"+ - "d\2\u046c\u046a\3\2\2\2\u046d\u0470\3\2\2\2\u046e\u046c\3\2\2\2\u046e"+ - "\u046f\3\2\2\2\u046f\177\3\2\2\2\u0470\u046e\3\2\2\2\u0471\u0479\5\u008c"+ - "G\2\u0472\u0474\7\30\2\2\u0473\u0472\3\2\2\2\u0473\u0474\3\2\2\2\u0474"+ - "\u0477\3\2\2\2\u0475\u0478\5\u00c6d\2\u0476\u0478\5l\67\2\u0477\u0475"+ - "\3\2\2\2\u0477\u0476\3\2\2\2\u0478\u047a\3\2\2\2\u0479\u0473\3\2\2\2\u0479"+ - "\u047a\3\2\2\2\u047a\u0081\3\2\2\2\u047b\u0480\5\u0080A\2\u047c\u047d"+ - "\7\5\2\2\u047d\u047f\5\u0080A\2\u047e\u047c\3\2\2\2\u047f\u0482\3\2\2"+ - "\2\u0480\u047e\3\2\2\2\u0480\u0481\3\2\2\2\u0481\u0083\3\2\2\2\u0482\u0480"+ - "\3\2\2\2\u0483\u0484\7\4\2\2\u0484\u0489\5\u0086D\2\u0485\u0486\7\5\2"+ - "\2\u0486\u0488\5\u0086D\2\u0487\u0485\3\2\2\2\u0488\u048b\3\2\2\2\u0489"+ - "\u0487\3\2\2\2\u0489\u048a\3\2\2\2\u048a\u048c\3\2\2\2\u048b\u0489\3\2"+ - "\2\2\u048c\u048d\7\6\2\2\u048d\u0085\3\2\2\2\u048e\u0491\5\u0088E\2\u048f"+ - "\u0491\5\u00b0Y\2\u0490\u048e\3\2\2\2\u0490\u048f\3\2\2\2\u0491\u0087"+ - "\3\2\2\2\u0492\u04a0\5\u00c4c\2\u0493\u0494\5\u00caf\2\u0494\u0495\7\4"+ - "\2\2\u0495\u049a\5\u008aF\2\u0496\u0497\7\5\2\2\u0497\u0499\5\u008aF\2"+ - "\u0498\u0496\3\2\2\2\u0499\u049c\3\2\2\2\u049a\u0498\3\2\2\2\u049a\u049b"+ - "\3\2\2\2\u049b\u049d\3\2\2\2\u049c\u049a\3\2\2\2\u049d\u049e\7\6\2\2\u049e"+ - "\u04a0\3\2\2\2\u049f\u0492\3\2\2\2\u049f\u0493\3\2\2\2\u04a0\u0089\3\2"+ - "\2\2\u04a1\u04a4\5\u00c4c\2\u04a2\u04a4\5\u0098M\2\u04a3\u04a1\3\2\2\2"+ - "\u04a3\u04a2\3\2\2\2\u04a4\u008b\3\2\2\2\u04a5\u04a6\5\u0090I\2\u04a6"+ - "\u008d\3\2\2\2\u04a7\u04ac\5\u008cG\2\u04a8\u04a9\7\5\2\2\u04a9\u04ab"+ - "\5\u008cG\2\u04aa\u04a8\3\2\2\2\u04ab\u04ae\3\2\2\2\u04ac\u04aa\3\2\2"+ - "\2\u04ac\u04ad\3\2\2\2\u04ad\u008f\3\2\2\2\u04ae\u04ac\3\2\2\2\u04af\u04b0"+ - "\bI\1\2\u04b0\u04b1\7\u009c\2\2\u04b1\u04bc\5\u0090I\7\u04b2\u04b3\7V"+ - "\2\2\u04b3\u04b4\7\4\2\2\u04b4\u04b5\5\24\13\2\u04b5\u04b6\7\6\2\2\u04b6"+ - "\u04bc\3\2\2\2\u04b7\u04b9\5\u0094K\2\u04b8\u04ba\5\u0092J\2\u04b9\u04b8"+ - "\3\2\2\2\u04b9\u04ba\3\2\2\2\u04ba\u04bc\3\2\2\2\u04bb\u04af\3\2\2\2\u04bb"+ - "\u04b2\3\2\2\2\u04bb\u04b7\3\2\2\2\u04bc\u04c5\3\2\2\2\u04bd\u04be\f\4"+ - "\2\2\u04be\u04bf\7\23\2\2\u04bf\u04c4\5\u0090I\5\u04c0\u04c1\f\3\2\2\u04c1"+ - "\u04c2\7\u00a4\2\2\u04c2\u04c4\5\u0090I\4\u04c3\u04bd\3\2\2\2\u04c3\u04c0"+ - "\3\2\2\2\u04c4\u04c7\3\2\2\2\u04c5\u04c3\3\2\2\2\u04c5\u04c6\3\2\2\2\u04c6"+ - "\u0091\3\2\2\2\u04c7\u04c5\3\2\2\2\u04c8\u04ca\7\u009c\2\2\u04c9\u04c8"+ - "\3\2\2\2\u04c9\u04ca\3\2\2\2\u04ca\u04cb\3\2\2\2\u04cb\u04cc\7\34\2\2"+ - "\u04cc\u04cd\5\u0094K\2\u04cd\u04ce\7\23\2\2\u04ce\u04cf\5\u0094K\2\u04cf"+ - "\u051b\3\2\2\2\u04d0\u04d2\7\u009c\2\2\u04d1\u04d0\3\2\2\2\u04d1\u04d2"+ - "\3\2\2\2\u04d2\u04d3\3\2\2\2\u04d3\u04d4\7t\2\2\u04d4\u04d5\7\4\2\2\u04d5"+ - "\u04da\5\u008cG\2\u04d6\u04d7\7\5\2\2\u04d7\u04d9\5\u008cG\2\u04d8\u04d6"+ - "\3\2\2\2\u04d9\u04dc\3\2\2\2\u04da\u04d8\3\2\2\2\u04da\u04db\3\2\2\2\u04db"+ - "\u04dd\3\2\2\2\u04dc\u04da\3\2\2\2\u04dd\u04de\7\6\2\2\u04de\u051b\3\2"+ - "\2\2\u04df\u04e1\7\u009c\2\2\u04e0\u04df\3\2\2\2\u04e0\u04e1\3\2\2\2\u04e1"+ - "\u04e2\3\2\2\2\u04e2\u04e3\7t\2\2\u04e3\u04e4\7\4\2\2\u04e4\u04e5\5\24"+ - "\13\2\u04e5\u04e6\7\6\2\2\u04e6\u051b\3\2\2\2\u04e7\u04e9\7\u009c\2\2"+ - "\u04e8\u04e7\3\2\2\2\u04e8\u04e9\3\2\2\2\u04e9\u04ea\3\2\2\2\u04ea\u04eb"+ - "\7\u00c9\2\2\u04eb\u051b\5\u0094K\2\u04ec\u04ee\7\u009c\2\2\u04ed\u04ec"+ - "\3\2\2\2\u04ed\u04ee\3\2\2\2\u04ee\u04ef\3\2\2\2\u04ef\u04f0\7\u0087\2"+ - "\2\u04f0\u04fe\t\n\2\2\u04f1\u04f2\7\4\2\2\u04f2\u04ff\7\6\2\2\u04f3\u04f4"+ - "\7\4\2\2\u04f4\u04f9\5\u008cG\2\u04f5\u04f6\7\5\2\2\u04f6\u04f8\5\u008c"+ - "G\2\u04f7\u04f5\3\2\2\2\u04f8\u04fb\3\2\2\2\u04f9\u04f7\3\2\2\2\u04f9"+ - "\u04fa\3\2\2\2\u04fa\u04fc\3\2\2\2\u04fb\u04f9\3\2\2\2\u04fc\u04fd\7\6"+ - "\2\2\u04fd\u04ff\3\2\2\2\u04fe\u04f1\3\2\2\2\u04fe\u04f3\3\2\2\2\u04ff"+ - "\u051b\3\2\2\2\u0500\u0502\7\u009c\2\2\u0501\u0500\3\2\2\2\u0501\u0502"+ - "\3\2\2\2\u0502\u0503\3\2\2\2\u0503\u0504\7\u0087\2\2\u0504\u0507\5\u0094"+ - "K\2\u0505\u0506\7R\2\2\u0506\u0508\7\u0123\2\2\u0507\u0505\3\2\2\2\u0507"+ - "\u0508\3\2\2\2\u0508\u051b\3\2\2\2\u0509\u050b\7~\2\2\u050a\u050c\7\u009c"+ - "\2\2\u050b\u050a\3\2\2\2\u050b\u050c\3\2\2\2\u050c\u050d\3\2\2\2\u050d"+ - "\u051b\7\u009d\2\2\u050e\u0510\7~\2\2\u050f\u0511\7\u009c\2\2\u0510\u050f"+ - "\3\2\2\2\u0510\u0511\3\2\2\2\u0511\u0512\3\2\2\2\u0512\u051b\t\13\2\2"+ - "\u0513\u0515\7~\2\2\u0514\u0516\7\u009c\2\2\u0515\u0514\3\2\2\2\u0515"+ - "\u0516\3\2\2\2\u0516\u0517\3\2\2\2\u0517\u0518\7L\2\2\u0518\u0519\7g\2"+ - "\2\u0519\u051b\5\u0094K\2\u051a\u04c9\3\2\2\2\u051a\u04d1\3\2\2\2\u051a"+ - "\u04e0\3\2\2\2\u051a\u04e8\3\2\2\2\u051a\u04ed\3\2\2\2\u051a\u0501\3\2"+ - "\2\2\u051a\u0509\3\2\2\2\u051a\u050e\3\2\2\2\u051a\u0513\3\2\2\2\u051b"+ - "\u0093\3\2\2\2\u051c\u051d\bK\1\2\u051d\u0521\5\u0096L\2\u051e\u051f\t"+ - "\f\2\2\u051f\u0521\5\u0094K\t\u0520\u051c\3\2\2\2\u0520\u051e\3\2\2\2"+ - "\u0521\u0537\3\2\2\2\u0522\u0523\f\b\2\2\u0523\u0524\t\r\2\2\u0524\u0536"+ - "\5\u0094K\t\u0525\u0526\f\7\2\2\u0526\u0527\t\16\2\2\u0527\u0536\5\u0094"+ - "K\b\u0528\u0529\f\6\2\2\u0529\u052a\7\u011f\2\2\u052a\u0536\5\u0094K\7"+ - "\u052b\u052c\f\5\2\2\u052c\u052d\7\u0122\2\2\u052d\u0536\5\u0094K\6\u052e"+ - "\u052f\f\4\2\2\u052f\u0530\7\u0120\2\2\u0530\u0536\5\u0094K\5\u0531\u0532"+ - "\f\3\2\2\u0532\u0533\5\u009aN\2\u0533\u0534\5\u0094K\4\u0534\u0536\3\2"+ - "\2\2\u0535\u0522\3\2\2\2\u0535\u0525\3\2\2\2\u0535\u0528\3\2\2\2\u0535"+ - "\u052b\3\2\2\2\u0535\u052e\3\2\2\2\u0535\u0531\3\2\2\2\u0536\u0539\3\2"+ - "\2\2\u0537\u0535\3\2\2\2\u0537\u0538\3\2\2\2\u0538\u0095\3\2\2\2\u0539"+ - "\u0537\3\2\2\2\u053a\u053b\bL\1\2\u053b\u05f7\t\17\2\2\u053c\u053e\7#"+ - "\2\2\u053d\u053f\5\u00b6\\\2\u053e\u053d\3\2\2\2\u053f\u0540\3\2\2\2\u0540"+ - "\u053e\3\2\2\2\u0540\u0541\3\2\2\2\u0541\u0544\3\2\2\2\u0542\u0543\7P"+ - "\2\2\u0543\u0545\5\u008cG\2\u0544\u0542\3\2\2\2\u0544\u0545\3\2\2\2\u0545"+ - "\u0546\3\2\2\2\u0546\u0547\7Q\2\2\u0547\u05f7\3\2\2\2\u0548\u0549\7#\2"+ - "\2\u0549\u054b\5\u008cG\2\u054a\u054c\5\u00b6\\\2\u054b\u054a\3\2\2\2"+ - "\u054c\u054d\3\2\2\2\u054d\u054b\3\2\2\2\u054d\u054e\3\2\2\2\u054e\u0551"+ - "\3\2\2\2\u054f\u0550\7P\2\2\u0550\u0552\5\u008cG\2\u0551\u054f\3\2\2\2"+ - "\u0551\u0552\3\2\2\2\u0552\u0553\3\2\2\2\u0553\u0554\7Q\2\2\u0554\u05f7"+ - "\3\2\2\2\u0555\u0556\t\20\2\2\u0556\u0557\7\4\2\2\u0557\u0558\5\u008c"+ - "G\2\u0558\u0559\7\30\2\2\u0559\u055a\5\u00acW\2\u055a\u055b\7\6\2\2\u055b"+ - "\u05f7\3\2\2\2\u055c\u055d\7\u00e4\2\2\u055d\u0566\7\4\2\2\u055e\u0563"+ - "\5\u0080A\2\u055f\u0560\7\5\2\2\u0560\u0562\5\u0080A\2\u0561\u055f\3\2"+ - "\2\2\u0562\u0565\3\2\2\2\u0563\u0561\3\2\2\2\u0563\u0564\3\2\2\2\u0564"+ - "\u0567\3\2\2\2\u0565\u0563\3\2\2\2\u0566\u055e\3\2\2\2\u0566\u0567\3\2"+ - "\2\2\u0567\u0568\3\2\2\2\u0568\u05f7\7\6\2\2\u0569\u056a\7a\2\2\u056a"+ - "\u056b\7\4\2\2\u056b\u056e\5\u008cG\2\u056c\u056d\7r\2\2\u056d\u056f\7"+ - "\u009e\2\2\u056e\u056c\3\2\2\2\u056e\u056f\3\2\2\2\u056f\u0570\3\2\2\2"+ - "\u0570\u0571\7\6\2\2\u0571\u05f7\3\2\2\2\u0572\u0573\7\u0082\2\2\u0573"+ - "\u0574\7\4\2\2\u0574\u0577\5\u008cG\2\u0575\u0576\7r\2\2\u0576\u0578\7"+ - "\u009e\2\2\u0577\u0575\3\2\2\2\u0577\u0578\3\2\2\2\u0578\u0579\3\2\2\2"+ - "\u0579\u057a\7\6\2\2\u057a\u05f7\3\2\2\2\u057b\u057c\7\u00b3\2\2\u057c"+ - "\u057d\7\4\2\2\u057d\u057e\5\u0094K\2\u057e\u057f\7t\2\2\u057f\u0580\5"+ - "\u0094K\2\u0580\u0581\7\6\2\2\u0581\u05f7\3\2\2\2\u0582\u05f7\5\u0098"+ - "M\2\u0583\u05f7\7\u011b\2\2\u0584\u0585\5\u00c4c\2\u0585\u0586\7\7\2\2"+ - "\u0586\u0587\7\u011b\2\2\u0587\u05f7\3\2\2\2\u0588\u0589\7\4\2\2\u0589"+ - "\u058c\5\u0080A\2\u058a\u058b\7\5\2\2\u058b\u058d\5\u0080A\2\u058c\u058a"+ - "\3\2\2\2\u058d\u058e\3\2\2\2\u058e\u058c\3\2\2\2\u058e\u058f\3\2\2\2\u058f"+ - "\u0590\3\2\2\2\u0590\u0591\7\6\2\2\u0591\u05f7\3\2\2\2\u0592\u0593\7\4"+ - "\2\2\u0593\u0594\5\24\13\2\u0594\u0595\7\6\2\2\u0595\u05f7\3\2\2\2\u0596"+ - "\u0597\5\u00c2b\2\u0597\u05a3\7\4\2\2\u0598\u059a\5^\60\2\u0599\u0598"+ - "\3\2\2\2\u0599\u059a\3\2\2\2\u059a\u059b\3\2\2\2\u059b\u05a0\5\u008cG"+ - "\2\u059c\u059d\7\5\2\2\u059d\u059f\5\u008cG\2\u059e\u059c\3\2\2\2\u059f"+ - "\u05a2\3\2\2\2\u05a0\u059e\3\2\2\2\u05a0\u05a1\3\2\2\2\u05a1\u05a4\3\2"+ - "\2\2\u05a2\u05a0\3\2\2\2\u05a3\u0599\3\2\2\2\u05a3\u05a4\3\2\2\2\u05a4"+ - "\u05a5\3\2\2\2\u05a5\u05ac\7\6\2\2\u05a6\u05a7\7_\2\2\u05a7\u05a8\7\4"+ - "\2\2\u05a8\u05a9\7\u010b\2\2\u05a9\u05aa\5\u0090I\2\u05aa\u05ab\7\6\2"+ - "\2\u05ab\u05ad\3\2\2\2\u05ac\u05a6\3\2\2\2\u05ac\u05ad\3\2\2\2\u05ad\u05b0"+ - "\3\2\2\2\u05ae\u05af\t\21\2\2\u05af\u05b1\7\u009e\2\2\u05b0\u05ae\3\2"+ - "\2\2\u05b0\u05b1\3\2\2\2\u05b1\u05b4\3\2\2\2\u05b2\u05b3\7\u00a9\2\2\u05b3"+ - "\u05b5\5\u00bc_\2\u05b4\u05b2\3\2\2\2\u05b4\u05b5\3\2\2\2\u05b5\u05f7"+ - "\3\2\2\2\u05b6\u05b7\5\u00caf\2\u05b7\u05b8\7\n\2\2\u05b8\u05b9\5\u008c"+ - "G\2\u05b9\u05f7\3\2\2\2\u05ba\u05bb\7\4\2\2\u05bb\u05be\5\u00caf\2\u05bc"+ - "\u05bd\7\5\2\2\u05bd\u05bf\5\u00caf\2\u05be\u05bc\3\2\2\2\u05bf\u05c0"+ - "\3\2\2\2\u05c0\u05be\3\2\2\2\u05c0\u05c1\3\2\2\2\u05c1\u05c2\3\2\2\2\u05c2"+ - "\u05c3\7\6\2\2\u05c3\u05c4\7\n\2\2\u05c4\u05c5\5\u008cG\2\u05c5\u05f7"+ - "\3\2\2\2\u05c6\u05f7\5\u00caf\2\u05c7\u05c8\7\4\2\2\u05c8\u05c9\5\u008c"+ - "G\2\u05c9\u05ca\7\6\2\2\u05ca\u05f7\3\2\2\2\u05cb\u05cc\7[\2\2\u05cc\u05cd"+ - "\7\4\2\2\u05cd\u05ce\5\u00caf\2\u05ce\u05cf\7g\2\2\u05cf\u05d0\5\u0094"+ - "K\2\u05d0\u05d1\7\6\2\2\u05d1\u05f7\3\2\2\2\u05d2\u05d3\t\22\2\2\u05d3"+ - "\u05d4\7\4\2\2\u05d4\u05d5\5\u0094K\2\u05d5\u05d6\t\23\2\2\u05d6\u05d9"+ - "\5\u0094K\2\u05d7\u05d8\t\24\2\2\u05d8\u05da\5\u0094K\2\u05d9\u05d7\3"+ - "\2\2\2\u05d9\u05da\3\2\2\2\u05da\u05db\3\2\2\2\u05db\u05dc\7\6\2\2\u05dc"+ - "\u05f7\3\2\2\2\u05dd\u05de\7\u00f6\2\2\u05de\u05e0\7\4\2\2\u05df\u05e1"+ - "\t\25\2\2\u05e0\u05df\3\2\2\2\u05e0\u05e1\3\2\2\2\u05e1\u05e3\3\2\2\2"+ - "\u05e2\u05e4\5\u0094K\2\u05e3\u05e2\3\2\2\2\u05e3\u05e4\3\2\2\2\u05e4"+ - "\u05e5\3\2\2\2\u05e5\u05e6\7g\2\2\u05e6\u05e7\5\u0094K\2\u05e7\u05e8\7"+ - "\6\2\2\u05e8\u05f7\3\2\2\2\u05e9\u05ea\7\u00ab\2\2\u05ea\u05eb\7\4\2\2"+ - "\u05eb\u05ec\5\u0094K\2\u05ec\u05ed\7\u00b2\2\2\u05ed\u05ee\5\u0094K\2"+ - "\u05ee\u05ef\7g\2\2\u05ef\u05f2\5\u0094K\2\u05f0\u05f1\7c\2\2\u05f1\u05f3"+ - "\5\u0094K\2\u05f2\u05f0\3\2\2\2\u05f2\u05f3\3\2\2\2\u05f3\u05f4\3\2\2"+ - "\2\u05f4\u05f5\7\6\2\2\u05f5\u05f7\3\2\2\2\u05f6\u053a\3\2\2\2\u05f6\u053c"+ - "\3\2\2\2\u05f6\u0548\3\2\2\2\u05f6\u0555\3\2\2\2\u05f6\u055c\3\2\2\2\u05f6"+ - "\u0569\3\2\2\2\u05f6\u0572\3\2\2\2\u05f6\u057b\3\2\2\2\u05f6\u0582\3\2"+ - "\2\2\u05f6\u0583\3\2\2\2\u05f6\u0584\3\2\2\2\u05f6\u0588\3\2\2\2\u05f6"+ - "\u0592\3\2\2\2\u05f6\u0596\3\2\2\2\u05f6\u05b6\3\2\2\2\u05f6\u05ba\3\2"+ - "\2\2\u05f6\u05c6\3\2\2\2\u05f6\u05c7\3\2\2\2\u05f6\u05cb\3\2\2\2\u05f6"+ - "\u05d2\3\2\2\2\u05f6\u05dd\3\2\2\2\u05f6\u05e9\3\2\2\2\u05f7\u0602\3\2"+ - "\2\2\u05f8\u05f9\f\n\2\2\u05f9\u05fa\7\13\2\2\u05fa\u05fb\5\u0094K\2\u05fb"+ - "\u05fc\7\f\2\2\u05fc\u0601\3\2\2\2\u05fd\u05fe\f\b\2\2\u05fe\u05ff\7\7"+ - "\2\2\u05ff\u0601\5\u00caf\2\u0600\u05f8\3\2\2\2\u0600\u05fd\3\2\2\2\u0601"+ - "\u0604\3\2\2\2\u0602\u0600\3\2\2\2\u0602\u0603\3\2\2\2\u0603\u0097\3\2"+ - "\2\2\u0604\u0602\3\2\2\2\u0605\u0612\7\u009d\2\2\u0606\u0612\5\u009eP"+ - "\2\u0607\u0608\5\u00caf\2\u0608\u0609\7\u0123\2\2\u0609\u0612\3\2\2\2"+ - "\u060a\u0612\5\u00d0i\2\u060b\u0612\5\u009cO\2\u060c\u060e\7\u0123\2\2"+ - "\u060d\u060c\3\2\2\2\u060e\u060f\3\2\2\2\u060f\u060d\3\2\2\2\u060f\u0610"+ - "\3\2\2\2\u0610\u0612\3\2\2\2\u0611\u0605\3\2\2\2\u0611\u0606\3\2\2\2\u0611"+ - "\u0607\3\2\2\2\u0611\u060a\3\2\2\2\u0611\u060b\3\2\2\2\u0611\u060d\3\2"+ - "\2\2\u0612\u0099\3\2\2\2\u0613\u0614\t\26\2\2\u0614\u009b\3\2\2\2\u0615"+ - "\u0616\t\27\2\2\u0616\u009d\3\2\2\2\u0617\u061a\7|\2\2\u0618\u061b\5\u00a0"+ - "Q\2\u0619\u061b\5\u00a4S\2\u061a\u0618\3\2\2\2\u061a\u0619\3\2\2\2\u061a"+ - "\u061b\3\2\2\2\u061b\u009f\3\2\2\2\u061c\u061e\5\u00a2R\2\u061d\u061f"+ - "\5\u00a6T\2\u061e\u061d\3\2\2\2\u061e\u061f\3\2\2\2\u061f\u00a1\3\2\2"+ - "\2\u0620\u0621\5\u00a8U\2\u0621\u0622\5\u00caf\2\u0622\u0624\3\2\2\2\u0623"+ - "\u0620\3\2\2\2\u0624\u0625\3\2\2\2\u0625\u0623\3\2\2\2\u0625\u0626\3\2"+ - "\2\2\u0626\u00a3\3\2\2\2\u0627\u062a\5\u00a6T\2\u0628\u062b\5\u00a2R\2"+ - "\u0629\u062b\5\u00a6T\2\u062a\u0628\3\2\2\2\u062a\u0629\3\2\2\2\u062a"+ - "\u062b\3\2\2\2\u062b\u00a5\3\2\2\2\u062c\u062d\5\u00a8U\2\u062d\u062e"+ - "\5\u00caf\2\u062e\u062f\7\u00f0\2\2\u062f\u0630\5\u00caf\2\u0630\u00a7"+ - "\3\2\2\2\u0631\u0633\t\30\2\2\u0632\u0631\3\2\2\2\u0632\u0633\3\2\2\2"+ - "\u0633\u0634\3\2\2\2\u0634\u0635\t\31\2\2\u0635\u00a9\3\2\2\2\u0636\u063a"+ - "\7a\2\2\u0637\u0638\7\17\2\2\u0638\u063a\5\u00c6d\2\u0639\u0636\3\2\2"+ - "\2\u0639\u0637\3\2\2\2\u063a\u00ab\3\2\2\2\u063b\u063c\7\27\2\2\u063c"+ - "\u063d\7\u0115\2\2\u063d\u063e\5\u00acW\2\u063e\u063f\7\u0117\2\2\u063f"+ - "\u066a\3\2\2\2\u0640\u0641\7\u0092\2\2\u0641\u0642\7\u0115\2\2\u0642\u0643"+ - "\5\u00acW\2\u0643\u0644\7\5\2\2\u0644\u0645\5\u00acW\2\u0645\u0646\7\u0117"+ - "\2\2\u0646\u066a\3\2\2\2\u0647\u064e\7\u00e4\2\2\u0648\u064a\7\u0115\2"+ - "\2\u0649\u064b\5\u00b2Z\2\u064a\u0649\3\2\2\2\u064a\u064b\3\2\2\2\u064b"+ - "\u064c\3\2\2\2\u064c\u064f\7\u0117\2\2\u064d\u064f\7\u0113\2\2\u064e\u0648"+ - "\3\2\2\2\u064e\u064d\3\2\2\2\u064f\u066a\3\2\2\2\u0650\u0651\7|\2\2\u0651"+ - "\u0654\t\32\2\2\u0652\u0653\7\u00f0\2\2\u0653\u0655\7\u0096\2\2\u0654"+ - "\u0652\3\2\2\2\u0654\u0655\3\2\2\2\u0655\u066a\3\2\2\2\u0656\u0657\7|"+ - "\2\2\u0657\u065a\t\33\2\2\u0658\u0659\7\u00f0\2\2\u0659\u065b\t\34\2\2"+ - "\u065a\u0658\3\2\2\2\u065a\u065b\3\2\2\2\u065b\u066a\3\2\2\2\u065c\u0667"+ - "\5\u00caf\2\u065d\u065e\7\4\2\2\u065e\u0663\7\u0127\2\2\u065f\u0660\7"+ - "\5\2\2\u0660\u0662\7\u0127\2\2\u0661\u065f\3\2\2\2\u0662\u0665\3\2\2\2"+ - "\u0663\u0661\3\2\2\2\u0663\u0664\3\2\2\2\u0664\u0666\3\2\2\2\u0665\u0663"+ - "\3\2\2\2\u0666\u0668\7\6\2\2\u0667\u065d\3\2\2\2\u0667\u0668\3\2\2\2\u0668"+ - "\u066a\3\2\2\2\u0669\u063b\3\2\2\2\u0669\u0640\3\2\2\2\u0669\u0647\3\2"+ - "\2\2\u0669\u0650\3\2\2\2\u0669\u0656\3\2\2\2\u0669\u065c\3\2\2\2\u066a"+ - "\u00ad\3\2\2\2\u066b\u0670\5\u00b0Y\2\u066c\u066d\7\5\2\2\u066d\u066f"+ - "\5\u00b0Y\2\u066e\u066c\3\2\2\2\u066f\u0672\3\2\2\2\u0670\u066e\3\2\2"+ - "\2\u0670\u0671\3\2\2\2\u0671\u00af\3\2\2\2\u0672\u0670\3\2\2\2\u0673\u0674"+ - "\5\u00c6d\2\u0674\u0677\5\u00acW\2\u0675\u0676\7\u009c\2\2\u0676\u0678"+ - "\7\u009d\2\2\u0677\u0675\3\2\2\2\u0677\u0678\3\2\2\2\u0678\u067a\3\2\2"+ - "\2\u0679\u067b\5\22\n\2\u067a\u0679\3\2\2\2\u067a\u067b\3\2\2\2\u067b"+ - "\u00b1\3\2\2\2\u067c\u0681\5\u00b4[\2\u067d\u067e\7\5\2\2\u067e\u0680"+ - "\5\u00b4[\2\u067f\u067d\3\2\2\2\u0680\u0683\3\2\2\2\u0681\u067f\3\2\2"+ - "\2\u0681\u0682\3\2\2\2\u0682\u00b3\3\2\2\2\u0683\u0681\3\2\2\2\u0684\u0686"+ - "\5\u00caf\2\u0685\u0687\7\r\2\2\u0686\u0685\3\2\2\2\u0686\u0687\3\2\2"+ - "\2\u0687\u0688\3\2\2\2\u0688\u068b\5\u00acW\2\u0689\u068a\7\u009c\2\2"+ - "\u068a\u068c\7\u009d\2\2\u068b\u0689\3\2\2\2\u068b\u068c\3\2\2\2\u068c"+ - "\u068e\3\2\2\2\u068d\u068f\5\22\n\2\u068e\u068d\3\2\2\2\u068e\u068f\3"+ - "\2\2\2\u068f\u00b5\3\2\2\2\u0690\u0691\7\u010a\2\2\u0691\u0692\5\u008c"+ - "G\2\u0692\u0693\7\u00ee\2\2\u0693\u0694\5\u008cG\2\u0694\u00b7\3\2\2\2"+ - "\u0695\u0696\7\u010c\2\2\u0696\u069b\5\u00ba^\2\u0697\u0698\7\5\2\2\u0698"+ - "\u069a\5\u00ba^\2\u0699\u0697\3\2\2\2\u069a\u069d\3\2\2\2\u069b\u0699"+ - "\3\2\2\2\u069b\u069c\3\2\2\2\u069c\u00b9\3\2\2\2\u069d\u069b\3\2\2\2\u069e"+ - "\u069f\5\u00c6d\2\u069f\u06a0\7\30\2\2\u06a0\u06a1\5\u00bc_\2\u06a1\u00bb"+ - "\3\2\2\2\u06a2\u06d1\5\u00c6d\2\u06a3\u06a4\7\4\2\2\u06a4\u06a5\5\u00c6"+ - "d\2\u06a5\u06a6\7\6\2\2\u06a6\u06d1\3\2\2\2\u06a7\u06ca\7\4\2\2\u06a8"+ - "\u06a9\7(\2\2\u06a9\u06aa\7 \2\2\u06aa\u06af\5\u008cG\2\u06ab\u06ac\7"+ - "\5\2\2\u06ac\u06ae\5\u008cG\2\u06ad\u06ab\3\2\2\2\u06ae\u06b1\3\2\2\2"+ - "\u06af\u06ad\3\2\2\2\u06af\u06b0\3\2\2\2\u06b0\u06cb\3\2\2\2\u06b1\u06af"+ - "\3\2\2\2\u06b2\u06b3\t\35\2\2\u06b3\u06b4\7 \2\2\u06b4\u06b9\5\u008cG"+ - "\2\u06b5\u06b6\7\5\2\2\u06b6\u06b8\5\u008cG\2\u06b7\u06b5\3\2\2\2\u06b8"+ - "\u06bb\3\2\2\2\u06b9\u06b7\3\2\2\2\u06b9\u06ba\3\2\2\2\u06ba\u06bd\3\2"+ - "\2\2\u06bb\u06b9\3\2\2\2\u06bc\u06b2\3\2\2\2\u06bc\u06bd\3\2\2\2\u06bd"+ - "\u06c8\3\2\2\2\u06be\u06bf\t\36\2\2\u06bf\u06c0\7 \2\2\u06c0\u06c5\5\66"+ - "\34\2\u06c1\u06c2\7\5\2\2\u06c2\u06c4\5\66\34\2\u06c3\u06c1\3\2\2\2\u06c4"+ - "\u06c7\3\2\2\2\u06c5\u06c3\3\2\2\2\u06c5\u06c6\3\2\2\2\u06c6\u06c9\3\2"+ - "\2\2\u06c7\u06c5\3\2\2\2\u06c8\u06be\3\2\2\2\u06c8\u06c9\3\2\2\2\u06c9"+ - "\u06cb\3\2\2\2\u06ca\u06a8\3\2\2\2\u06ca\u06bc\3\2\2\2\u06cb\u06cd\3\2"+ - "\2\2\u06cc\u06ce\5\u00be`\2\u06cd\u06cc\3\2\2\2\u06cd\u06ce\3\2\2\2\u06ce"+ - "\u06cf\3\2\2\2\u06cf\u06d1\7\6\2\2\u06d0\u06a2\3\2\2\2\u06d0\u06a3\3\2"+ - "\2\2\u06d0\u06a7\3\2\2\2\u06d1\u00bd\3\2\2\2\u06d2\u06d3\7\u00ba\2\2\u06d3"+ - "\u06e3\5\u00c0a\2\u06d4\u06d5\7\u00cf\2\2\u06d5\u06e3\5\u00c0a\2\u06d6"+ - "\u06d7\7\u00ba\2\2\u06d7\u06d8\7\34\2\2\u06d8\u06d9\5\u00c0a\2\u06d9\u06da"+ - "\7\23\2\2\u06da\u06db\5\u00c0a\2\u06db\u06e3\3\2\2\2\u06dc\u06dd\7\u00cf"+ - "\2\2\u06dd\u06de\7\34\2\2\u06de\u06df\5\u00c0a\2\u06df\u06e0\7\23\2\2"+ - "\u06e0\u06e1\5\u00c0a\2\u06e1\u06e3\3\2\2\2\u06e2\u06d2\3\2\2\2\u06e2"+ - "\u06d4\3\2\2\2\u06e2\u06d6\3\2\2\2\u06e2\u06dc\3\2\2\2\u06e3\u00bf\3\2"+ - "\2\2\u06e4\u06e5\7\u00fc\2\2\u06e5\u06ec\t\37\2\2\u06e6\u06e7\7:\2\2\u06e7"+ - "\u06ec\7\u00ce\2\2\u06e8\u06e9\5\u008cG\2\u06e9\u06ea\t\37\2\2\u06ea\u06ec"+ - "\3\2\2\2\u06eb\u06e4\3\2\2\2\u06eb\u06e6\3\2\2\2\u06eb\u06e8\3\2\2\2\u06ec"+ - "\u00c1\3\2\2\2\u06ed\u06f2\5\u00c4c\2\u06ee\u06f2\7_\2\2\u06ef\u06f2\7"+ - "\u0086\2\2\u06f0\u06f2\7\u00c8\2\2\u06f1\u06ed\3\2\2\2\u06f1\u06ee\3\2"+ - "\2\2\u06f1\u06ef\3\2\2\2\u06f1\u06f0\3\2\2\2\u06f2\u00c3\3\2\2\2\u06f3"+ - "\u06f8\5\u00caf\2\u06f4\u06f5\7\7\2\2\u06f5\u06f7\5\u00caf\2\u06f6\u06f4"+ - "\3\2\2\2\u06f7\u06fa\3\2\2\2\u06f8\u06f6\3\2\2\2\u06f8\u06f9\3\2\2\2\u06f9"+ - "\u00c5\3\2\2\2\u06fa\u06f8\3\2\2\2\u06fb\u06fc\5\u00caf\2\u06fc\u06fd"+ - "\5\u00c8e\2\u06fd\u00c7\3\2\2\2\u06fe\u06ff\7\u011a\2\2\u06ff\u0701\5"+ - "\u00caf\2\u0700\u06fe\3\2\2\2\u0701\u0702\3\2\2\2\u0702\u0700\3\2\2\2"+ - "\u0702\u0703\3\2\2\2\u0703\u0706\3\2\2\2\u0704\u0706\3\2\2\2\u0705\u0700"+ - "\3\2\2\2\u0705\u0704\3\2\2\2\u0706\u00c9\3\2\2\2\u0707\u070b\5\u00ccg"+ - "\2\u0708\u0709\6f\22\2\u0709\u070b\5\u00d4k\2\u070a\u0707\3\2\2\2\u070a"+ - "\u0708\3\2\2\2\u070b\u00cb\3\2\2\2\u070c\u0713\7\u012d\2\2\u070d\u0713"+ - "\5\u00ceh\2\u070e\u070f\6g\23\2\u070f\u0713\5\u00d2j\2\u0710\u0711\6g"+ - "\24\2\u0711\u0713\5\u00d6l\2\u0712\u070c\3\2\2\2\u0712\u070d\3\2\2\2\u0712"+ - "\u070e\3\2\2\2\u0712\u0710\3\2\2\2\u0713\u00cd\3\2\2\2\u0714\u0715\7\u012e"+ - "\2\2\u0715\u00cf\3\2\2\2\u0716\u0718\6i\25\2\u0717\u0719\7\u011a\2\2\u0718"+ - "\u0717\3\2\2\2\u0718\u0719\3\2\2\2\u0719\u071a\3\2\2\2\u071a\u0742\7\u0128"+ - "\2\2\u071b\u071d\6i\26\2\u071c\u071e\7\u011a\2\2\u071d\u071c\3\2\2\2\u071d"+ - "\u071e\3\2\2\2\u071e\u071f\3\2\2\2\u071f\u0742\7\u0129\2\2\u0720\u0722"+ - "\6i\27\2\u0721\u0723\7\u011a\2\2\u0722\u0721\3\2\2\2\u0722\u0723\3\2\2"+ - "\2\u0723\u0724\3\2\2\2\u0724\u0742\t \2\2\u0725\u0727\7\u011a\2\2\u0726"+ - "\u0725\3\2\2\2\u0726\u0727\3\2\2\2\u0727\u0728\3\2\2\2\u0728\u0742\7\u0127"+ - "\2\2\u0729\u072b\7\u011a\2\2\u072a\u0729\3\2\2\2\u072a\u072b\3\2\2\2\u072b"+ - "\u072c\3\2\2\2\u072c\u0742\7\u0124\2\2\u072d\u072f\7\u011a\2\2\u072e\u072d"+ - "\3\2\2\2\u072e\u072f\3\2\2\2\u072f\u0730\3\2\2\2\u0730\u0742\7\u0125\2"+ - "\2\u0731\u0733\7\u011a\2\2\u0732\u0731\3\2\2\2\u0732\u0733\3\2\2\2\u0733"+ - "\u0734\3\2\2\2\u0734\u0742\7\u0126\2\2\u0735\u0737\7\u011a\2\2\u0736\u0735"+ - "\3\2\2\2\u0736\u0737\3\2\2\2\u0737\u0738\3\2\2\2\u0738\u0742\7\u012b\2"+ - "\2\u0739\u073b\7\u011a\2\2\u073a\u0739\3\2\2\2\u073a\u073b\3\2\2\2\u073b"+ - "\u073c\3\2\2\2\u073c\u0742\7\u012a\2\2\u073d\u073f\7\u011a\2\2\u073e\u073d"+ - "\3\2\2\2\u073e\u073f\3\2\2\2\u073f\u0740\3\2\2\2\u0740\u0742\7\u012c\2"+ - "\2\u0741\u0716\3\2\2\2\u0741\u071b\3\2\2\2\u0741\u0720\3\2\2\2\u0741\u0726"+ - "\3\2\2\2\u0741\u072a\3\2\2\2\u0741\u072e\3\2\2\2\u0741\u0732\3\2\2\2\u0741"+ - "\u0736\3\2\2\2\u0741\u073a\3\2\2\2\u0741\u073e\3\2\2\2\u0742\u00d1\3\2"+ - "\2\2\u0743\u0744\t!\2\2\u0744\u00d3\3\2\2\2\u0745\u0746\t\"\2\2\u0746"+ - "\u00d5\3\2\2\2\u0747\u0748\t#\2\2\u0748\u00d7\3\2\2\2\u0106\u00dc\u00e4"+ - "\u00e8\u00eb\u00ef\u00f2\u00f6\u00f9\u00ff\u0107\u010c\u0118\u0124\u0129"+ - "\u0132\u013d\u0142\u0145\u015b\u015d\u0166\u016d\u0170\u0177\u017b\u0181"+ - "\u0189\u0194\u019f\u01a6\u01ac\u01b5\u01b8\u01c1\u01c4\u01cd\u01d0\u01d9"+ - "\u01dc\u01df\u01e4\u01e6\u01ef\u01f6\u01fd\u0200\u0202\u020e\u0212\u0216"+ - "\u021c\u0220\u0228\u022c\u022f\u0232\u0235\u0239\u023d\u0242\u0246\u0249"+ - "\u024c\u024f\u0253\u0258\u025c\u025f\u0262\u0265\u0267\u026d\u0274\u0279"+ - "\u027c\u027f\u0283\u028d\u0291\u0293\u0296\u029a\u02a0\u02a4\u02b1\u02b6"+ - "\u02c3\u02c8\u02d0\u02d6\u02da\u02e3\u02ed\u02fc\u0301\u0303\u0307\u0310"+ - "\u031d\u0322\u0326\u032e\u0331\u0335\u0343\u0350\u0355\u0359\u035c\u0361"+ - "\u036a\u036d\u0372\u0379\u037c\u0381\u0387\u038d\u0391\u0397\u039b\u039e"+ - "\u03a3\u03a6\u03ab\u03af\u03b2\u03b5\u03bb\u03c0\u03c5\u03d7\u03d9\u03dc"+ - "\u03e7\u03f0\u03f7\u03fb\u0403\u040b\u0411\u0419\u0425\u0428\u042e\u0432"+ - "\u0434\u043d\u0449\u044b\u0452\u0459\u045f\u0465\u0467\u046e\u0473\u0477"+ - "\u0479\u0480\u0489\u0490\u049a\u049f\u04a3\u04ac\u04b9\u04bb\u04c3\u04c5"+ - "\u04c9\u04d1\u04da\u04e0\u04e8\u04ed\u04f9\u04fe\u0501\u0507\u050b\u0510"+ - "\u0515\u051a\u0520\u0535\u0537\u0540\u0544\u054d\u0551\u0563\u0566\u056e"+ - "\u0577\u058e\u0599\u05a0\u05a3\u05ac\u05b0\u05b4\u05c0\u05d9\u05e0\u05e3"+ - "\u05f2\u05f6\u0600\u0602\u060f\u0611\u061a\u061e\u0625\u062a\u0632\u0639"+ - "\u064a\u064e\u0654\u065a\u0663\u0667\u0669\u0670\u0677\u067a\u0681\u0686"+ - "\u068b\u068e\u069b\u06af\u06b9\u06bc\u06c5\u06c8\u06ca\u06cd\u06d0\u06e2"+ - "\u06eb\u06f1\u06f8\u0702\u0705\u070a\u0712\u0718\u071d\u0722\u0726\u072a"+ - "\u072e\u0732\u0736\u073a\u073e\u0741"; - public static final ATN _ATN = - new ATNDeserializer().deserialize(_serializedATN.toCharArray()); - static { - _decisionToDFA = new DFA[_ATN.getNumberOfDecisions()]; - for (int i = 0; i < _ATN.getNumberOfDecisions(); i++) { - _decisionToDFA[i] = new DFA(_ATN.getDecisionState(i), i); - } - } -} \ No newline at end of file diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendVisitor.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendVisitor.java deleted file mode 100644 index d574af5f0c..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/gen-antlr/org/apache/amoro/spark/sql/parser/MixedFormatSqlExtendVisitor.java +++ /dev/null @@ -1,1137 +0,0 @@ -// Generated from org/apache/amoro/spark/sql/parser/MixedFormatSqlExtend.g4 by ANTLR 4.8 -package org.apache.amoro.spark.sql.parser; -import org.antlr.v4.runtime.tree.ParseTreeVisitor; - -/** - * This interface defines a complete generic visitor for a parse tree produced - * by {@link MixedFormatSqlExtendParser}. - * - * @param The return type of the visit operation. Use {@link Void} for - * operations with no return type. - */ -public interface MixedFormatSqlExtendVisitor extends ParseTreeVisitor { - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#extendStatement}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitExtendStatement(MixedFormatSqlExtendParser.ExtendStatementContext ctx); - /** - * Visit a parse tree produced by the {@code createTableWithPk} - * labeled alternative in {@link MixedFormatSqlExtendParser#statement}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitCreateTableWithPk(MixedFormatSqlExtendParser.CreateTableWithPkContext ctx); - /** - * Visit a parse tree produced by the {@code explain} - * labeled alternative in {@link MixedFormatSqlExtendParser#statement}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitExplain(MixedFormatSqlExtendParser.ExplainContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#createTableHeader}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitCreateTableHeader(MixedFormatSqlExtendParser.CreateTableHeaderContext ctx); - /** - * Visit a parse tree produced by the {@code colListWithPk} - * labeled alternative in {@link MixedFormatSqlExtendParser#colListAndPk}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitColListWithPk(MixedFormatSqlExtendParser.ColListWithPkContext ctx); - /** - * Visit a parse tree produced by the {@code colListOnlyPk} - * labeled alternative in {@link MixedFormatSqlExtendParser#colListAndPk}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitColListOnlyPk(MixedFormatSqlExtendParser.ColListOnlyPkContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#primarySpec}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPrimarySpec(MixedFormatSqlExtendParser.PrimarySpecContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#bucketSpec}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitBucketSpec(MixedFormatSqlExtendParser.BucketSpecContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#skewSpec}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSkewSpec(MixedFormatSqlExtendParser.SkewSpecContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#locationSpec}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitLocationSpec(MixedFormatSqlExtendParser.LocationSpecContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#commentSpec}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitCommentSpec(MixedFormatSqlExtendParser.CommentSpecContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#query}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitQuery(MixedFormatSqlExtendParser.QueryContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#ctes}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitCtes(MixedFormatSqlExtendParser.CtesContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#namedQuery}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitNamedQuery(MixedFormatSqlExtendParser.NamedQueryContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#tableProvider}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTableProvider(MixedFormatSqlExtendParser.TableProviderContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#createTableClauses}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitCreateTableClauses(MixedFormatSqlExtendParser.CreateTableClausesContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#tablePropertyList}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTablePropertyList(MixedFormatSqlExtendParser.TablePropertyListContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#tableProperty}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTableProperty(MixedFormatSqlExtendParser.TablePropertyContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#tablePropertyKey}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTablePropertyKey(MixedFormatSqlExtendParser.TablePropertyKeyContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#tablePropertyValue}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTablePropertyValue(MixedFormatSqlExtendParser.TablePropertyValueContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#constantList}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitConstantList(MixedFormatSqlExtendParser.ConstantListContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#nestedConstantList}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitNestedConstantList(MixedFormatSqlExtendParser.NestedConstantListContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#createFileFormat}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitCreateFileFormat(MixedFormatSqlExtendParser.CreateFileFormatContext ctx); - /** - * Visit a parse tree produced by the {@code tableFileFormat} - * labeled alternative in {@link MixedFormatSqlExtendParser#fileFormat}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTableFileFormat(MixedFormatSqlExtendParser.TableFileFormatContext ctx); - /** - * Visit a parse tree produced by the {@code genericFileFormat} - * labeled alternative in {@link MixedFormatSqlExtendParser#fileFormat}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitGenericFileFormat(MixedFormatSqlExtendParser.GenericFileFormatContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#storageHandler}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitStorageHandler(MixedFormatSqlExtendParser.StorageHandlerContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#queryOrganization}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitQueryOrganization(MixedFormatSqlExtendParser.QueryOrganizationContext ctx); - /** - * Visit a parse tree produced by the {@code queryTermDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryTerm}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitQueryTermDefault(MixedFormatSqlExtendParser.QueryTermDefaultContext ctx); - /** - * Visit a parse tree produced by the {@code setOperation} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryTerm}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSetOperation(MixedFormatSqlExtendParser.SetOperationContext ctx); - /** - * Visit a parse tree produced by the {@code queryPrimaryDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitQueryPrimaryDefault(MixedFormatSqlExtendParser.QueryPrimaryDefaultContext ctx); - /** - * Visit a parse tree produced by the {@code fromStmt} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitFromStmt(MixedFormatSqlExtendParser.FromStmtContext ctx); - /** - * Visit a parse tree produced by the {@code table} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTable(MixedFormatSqlExtendParser.TableContext ctx); - /** - * Visit a parse tree produced by the {@code inlineTableDefault1} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitInlineTableDefault1(MixedFormatSqlExtendParser.InlineTableDefault1Context ctx); - /** - * Visit a parse tree produced by the {@code subquery} - * labeled alternative in {@link MixedFormatSqlExtendParser#queryPrimary}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSubquery(MixedFormatSqlExtendParser.SubqueryContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#sortItem}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSortItem(MixedFormatSqlExtendParser.SortItemContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#fromStatement}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitFromStatement(MixedFormatSqlExtendParser.FromStatementContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#fromStatementBody}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitFromStatementBody(MixedFormatSqlExtendParser.FromStatementBodyContext ctx); - /** - * Visit a parse tree produced by the {@code transformQuerySpecification} - * labeled alternative in {@link MixedFormatSqlExtendParser#querySpecification}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTransformQuerySpecification(MixedFormatSqlExtendParser.TransformQuerySpecificationContext ctx); - /** - * Visit a parse tree produced by the {@code regularQuerySpecification} - * labeled alternative in {@link MixedFormatSqlExtendParser#querySpecification}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitRegularQuerySpecification(MixedFormatSqlExtendParser.RegularQuerySpecificationContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#transformClause}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTransformClause(MixedFormatSqlExtendParser.TransformClauseContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#selectClause}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSelectClause(MixedFormatSqlExtendParser.SelectClauseContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#whereClause}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitWhereClause(MixedFormatSqlExtendParser.WhereClauseContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#havingClause}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitHavingClause(MixedFormatSqlExtendParser.HavingClauseContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#hint}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitHint(MixedFormatSqlExtendParser.HintContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#hintStatement}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitHintStatement(MixedFormatSqlExtendParser.HintStatementContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#fromClause}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitFromClause(MixedFormatSqlExtendParser.FromClauseContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#aggregationClause}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitAggregationClause(MixedFormatSqlExtendParser.AggregationClauseContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#groupByClause}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitGroupByClause(MixedFormatSqlExtendParser.GroupByClauseContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#groupingAnalytics}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitGroupingAnalytics(MixedFormatSqlExtendParser.GroupingAnalyticsContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#groupingElement}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitGroupingElement(MixedFormatSqlExtendParser.GroupingElementContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#groupingSet}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitGroupingSet(MixedFormatSqlExtendParser.GroupingSetContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#pivotClause}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPivotClause(MixedFormatSqlExtendParser.PivotClauseContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#pivotColumn}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPivotColumn(MixedFormatSqlExtendParser.PivotColumnContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#pivotValue}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPivotValue(MixedFormatSqlExtendParser.PivotValueContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#lateralView}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitLateralView(MixedFormatSqlExtendParser.LateralViewContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#setQuantifier}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSetQuantifier(MixedFormatSqlExtendParser.SetQuantifierContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#relation}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitRelation(MixedFormatSqlExtendParser.RelationContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#joinRelation}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitJoinRelation(MixedFormatSqlExtendParser.JoinRelationContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#joinType}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitJoinType(MixedFormatSqlExtendParser.JoinTypeContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#joinCriteria}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitJoinCriteria(MixedFormatSqlExtendParser.JoinCriteriaContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#sample}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSample(MixedFormatSqlExtendParser.SampleContext ctx); - /** - * Visit a parse tree produced by the {@code sampleByPercentile} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSampleByPercentile(MixedFormatSqlExtendParser.SampleByPercentileContext ctx); - /** - * Visit a parse tree produced by the {@code sampleByRows} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSampleByRows(MixedFormatSqlExtendParser.SampleByRowsContext ctx); - /** - * Visit a parse tree produced by the {@code sampleByBucket} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSampleByBucket(MixedFormatSqlExtendParser.SampleByBucketContext ctx); - /** - * Visit a parse tree produced by the {@code sampleByBytes} - * labeled alternative in {@link MixedFormatSqlExtendParser#sampleMethod}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSampleByBytes(MixedFormatSqlExtendParser.SampleByBytesContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#identifierList}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitIdentifierList(MixedFormatSqlExtendParser.IdentifierListContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#identifierSeq}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitIdentifierSeq(MixedFormatSqlExtendParser.IdentifierSeqContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#orderedIdentifierList}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitOrderedIdentifierList(MixedFormatSqlExtendParser.OrderedIdentifierListContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#orderedIdentifier}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitOrderedIdentifier(MixedFormatSqlExtendParser.OrderedIdentifierContext ctx); - /** - * Visit a parse tree produced by the {@code tableName} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTableName(MixedFormatSqlExtendParser.TableNameContext ctx); - /** - * Visit a parse tree produced by the {@code aliasedQuery} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitAliasedQuery(MixedFormatSqlExtendParser.AliasedQueryContext ctx); - /** - * Visit a parse tree produced by the {@code aliasedRelation} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitAliasedRelation(MixedFormatSqlExtendParser.AliasedRelationContext ctx); - /** - * Visit a parse tree produced by the {@code inlineTableDefault2} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitInlineTableDefault2(MixedFormatSqlExtendParser.InlineTableDefault2Context ctx); - /** - * Visit a parse tree produced by the {@code tableValuedFunction} - * labeled alternative in {@link MixedFormatSqlExtendParser#relationPrimary}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTableValuedFunction(MixedFormatSqlExtendParser.TableValuedFunctionContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#inlineTable}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitInlineTable(MixedFormatSqlExtendParser.InlineTableContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#functionTable}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitFunctionTable(MixedFormatSqlExtendParser.FunctionTableContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#tableAlias}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTableAlias(MixedFormatSqlExtendParser.TableAliasContext ctx); - /** - * Visit a parse tree produced by the {@code rowFormatSerde} - * labeled alternative in {@link MixedFormatSqlExtendParser#rowFormat}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitRowFormatSerde(MixedFormatSqlExtendParser.RowFormatSerdeContext ctx); - /** - * Visit a parse tree produced by the {@code rowFormatDelimited} - * labeled alternative in {@link MixedFormatSqlExtendParser#rowFormat}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitRowFormatDelimited(MixedFormatSqlExtendParser.RowFormatDelimitedContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#multipartIdentifier}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitMultipartIdentifier(MixedFormatSqlExtendParser.MultipartIdentifierContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#namedExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitNamedExpression(MixedFormatSqlExtendParser.NamedExpressionContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#namedExpressionSeq}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitNamedExpressionSeq(MixedFormatSqlExtendParser.NamedExpressionSeqContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#partitionFieldList}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPartitionFieldList(MixedFormatSqlExtendParser.PartitionFieldListContext ctx); - /** - * Visit a parse tree produced by the {@code partitionTransform} - * labeled alternative in {@link MixedFormatSqlExtendParser#partitionField}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPartitionTransform(MixedFormatSqlExtendParser.PartitionTransformContext ctx); - /** - * Visit a parse tree produced by the {@code partitionColumn} - * labeled alternative in {@link MixedFormatSqlExtendParser#partitionField}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPartitionColumn(MixedFormatSqlExtendParser.PartitionColumnContext ctx); - /** - * Visit a parse tree produced by the {@code identityTransform} - * labeled alternative in {@link MixedFormatSqlExtendParser#transform}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitIdentityTransform(MixedFormatSqlExtendParser.IdentityTransformContext ctx); - /** - * Visit a parse tree produced by the {@code applyTransform} - * labeled alternative in {@link MixedFormatSqlExtendParser#transform}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitApplyTransform(MixedFormatSqlExtendParser.ApplyTransformContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#transformArgument}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTransformArgument(MixedFormatSqlExtendParser.TransformArgumentContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#expression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitExpression(MixedFormatSqlExtendParser.ExpressionContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#expressionSeq}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitExpressionSeq(MixedFormatSqlExtendParser.ExpressionSeqContext ctx); - /** - * Visit a parse tree produced by the {@code logicalNot} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitLogicalNot(MixedFormatSqlExtendParser.LogicalNotContext ctx); - /** - * Visit a parse tree produced by the {@code predicated} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPredicated(MixedFormatSqlExtendParser.PredicatedContext ctx); - /** - * Visit a parse tree produced by the {@code exists} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitExists(MixedFormatSqlExtendParser.ExistsContext ctx); - /** - * Visit a parse tree produced by the {@code logicalBinary} - * labeled alternative in {@link MixedFormatSqlExtendParser#booleanExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitLogicalBinary(MixedFormatSqlExtendParser.LogicalBinaryContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#predicate}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPredicate(MixedFormatSqlExtendParser.PredicateContext ctx); - /** - * Visit a parse tree produced by the {@code valueExpressionDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitValueExpressionDefault(MixedFormatSqlExtendParser.ValueExpressionDefaultContext ctx); - /** - * Visit a parse tree produced by the {@code comparison} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitComparison(MixedFormatSqlExtendParser.ComparisonContext ctx); - /** - * Visit a parse tree produced by the {@code arithmeticBinary} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitArithmeticBinary(MixedFormatSqlExtendParser.ArithmeticBinaryContext ctx); - /** - * Visit a parse tree produced by the {@code arithmeticUnary} - * labeled alternative in {@link MixedFormatSqlExtendParser#valueExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitArithmeticUnary(MixedFormatSqlExtendParser.ArithmeticUnaryContext ctx); - /** - * Visit a parse tree produced by the {@code struct} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitStruct(MixedFormatSqlExtendParser.StructContext ctx); - /** - * Visit a parse tree produced by the {@code dereference} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitDereference(MixedFormatSqlExtendParser.DereferenceContext ctx); - /** - * Visit a parse tree produced by the {@code simpleCase} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSimpleCase(MixedFormatSqlExtendParser.SimpleCaseContext ctx); - /** - * Visit a parse tree produced by the {@code currentLike} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitCurrentLike(MixedFormatSqlExtendParser.CurrentLikeContext ctx); - /** - * Visit a parse tree produced by the {@code columnReference} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitColumnReference(MixedFormatSqlExtendParser.ColumnReferenceContext ctx); - /** - * Visit a parse tree produced by the {@code rowConstructor} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitRowConstructor(MixedFormatSqlExtendParser.RowConstructorContext ctx); - /** - * Visit a parse tree produced by the {@code last} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitLast(MixedFormatSqlExtendParser.LastContext ctx); - /** - * Visit a parse tree produced by the {@code star} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitStar(MixedFormatSqlExtendParser.StarContext ctx); - /** - * Visit a parse tree produced by the {@code overlay} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitOverlay(MixedFormatSqlExtendParser.OverlayContext ctx); - /** - * Visit a parse tree produced by the {@code subscript} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSubscript(MixedFormatSqlExtendParser.SubscriptContext ctx); - /** - * Visit a parse tree produced by the {@code subqueryExpression} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSubqueryExpression(MixedFormatSqlExtendParser.SubqueryExpressionContext ctx); - /** - * Visit a parse tree produced by the {@code substring} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSubstring(MixedFormatSqlExtendParser.SubstringContext ctx); - /** - * Visit a parse tree produced by the {@code cast} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitCast(MixedFormatSqlExtendParser.CastContext ctx); - /** - * Visit a parse tree produced by the {@code constantDefault} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitConstantDefault(MixedFormatSqlExtendParser.ConstantDefaultContext ctx); - /** - * Visit a parse tree produced by the {@code lambda} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitLambda(MixedFormatSqlExtendParser.LambdaContext ctx); - /** - * Visit a parse tree produced by the {@code parenthesizedExpression} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitParenthesizedExpression(MixedFormatSqlExtendParser.ParenthesizedExpressionContext ctx); - /** - * Visit a parse tree produced by the {@code extract} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitExtract(MixedFormatSqlExtendParser.ExtractContext ctx); - /** - * Visit a parse tree produced by the {@code trim} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTrim(MixedFormatSqlExtendParser.TrimContext ctx); - /** - * Visit a parse tree produced by the {@code functionCall} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitFunctionCall(MixedFormatSqlExtendParser.FunctionCallContext ctx); - /** - * Visit a parse tree produced by the {@code searchedCase} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSearchedCase(MixedFormatSqlExtendParser.SearchedCaseContext ctx); - /** - * Visit a parse tree produced by the {@code position} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPosition(MixedFormatSqlExtendParser.PositionContext ctx); - /** - * Visit a parse tree produced by the {@code first} - * labeled alternative in {@link MixedFormatSqlExtendParser#primaryExpression}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitFirst(MixedFormatSqlExtendParser.FirstContext ctx); - /** - * Visit a parse tree produced by the {@code nullLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitNullLiteral(MixedFormatSqlExtendParser.NullLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code intervalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitIntervalLiteral(MixedFormatSqlExtendParser.IntervalLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code typeConstructor} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTypeConstructor(MixedFormatSqlExtendParser.TypeConstructorContext ctx); - /** - * Visit a parse tree produced by the {@code numericLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitNumericLiteral(MixedFormatSqlExtendParser.NumericLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code booleanLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitBooleanLiteral(MixedFormatSqlExtendParser.BooleanLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code stringLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#constant}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitStringLiteral(MixedFormatSqlExtendParser.StringLiteralContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#comparisonOperator}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitComparisonOperator(MixedFormatSqlExtendParser.ComparisonOperatorContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#booleanValue}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitBooleanValue(MixedFormatSqlExtendParser.BooleanValueContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#interval}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitInterval(MixedFormatSqlExtendParser.IntervalContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#errorCapturingMultiUnitsInterval}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitErrorCapturingMultiUnitsInterval(MixedFormatSqlExtendParser.ErrorCapturingMultiUnitsIntervalContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#multiUnitsInterval}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitMultiUnitsInterval(MixedFormatSqlExtendParser.MultiUnitsIntervalContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#errorCapturingUnitToUnitInterval}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitErrorCapturingUnitToUnitInterval(MixedFormatSqlExtendParser.ErrorCapturingUnitToUnitIntervalContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#unitToUnitInterval}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitUnitToUnitInterval(MixedFormatSqlExtendParser.UnitToUnitIntervalContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#intervalValue}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitIntervalValue(MixedFormatSqlExtendParser.IntervalValueContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#colPosition}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitColPosition(MixedFormatSqlExtendParser.ColPositionContext ctx); - /** - * Visit a parse tree produced by the {@code complexDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitComplexDataType(MixedFormatSqlExtendParser.ComplexDataTypeContext ctx); - /** - * Visit a parse tree produced by the {@code yearMonthIntervalDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitYearMonthIntervalDataType(MixedFormatSqlExtendParser.YearMonthIntervalDataTypeContext ctx); - /** - * Visit a parse tree produced by the {@code dayTimeIntervalDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitDayTimeIntervalDataType(MixedFormatSqlExtendParser.DayTimeIntervalDataTypeContext ctx); - /** - * Visit a parse tree produced by the {@code primitiveDataType} - * labeled alternative in {@link MixedFormatSqlExtendParser#dataType}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitPrimitiveDataType(MixedFormatSqlExtendParser.PrimitiveDataTypeContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#colTypeList}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitColTypeList(MixedFormatSqlExtendParser.ColTypeListContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#colType}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitColType(MixedFormatSqlExtendParser.ColTypeContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#complexColTypeList}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitComplexColTypeList(MixedFormatSqlExtendParser.ComplexColTypeListContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#complexColType}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitComplexColType(MixedFormatSqlExtendParser.ComplexColTypeContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#whenClause}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitWhenClause(MixedFormatSqlExtendParser.WhenClauseContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#windowClause}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitWindowClause(MixedFormatSqlExtendParser.WindowClauseContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#namedWindow}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitNamedWindow(MixedFormatSqlExtendParser.NamedWindowContext ctx); - /** - * Visit a parse tree produced by the {@code windowRef} - * labeled alternative in {@link MixedFormatSqlExtendParser#windowSpec}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitWindowRef(MixedFormatSqlExtendParser.WindowRefContext ctx); - /** - * Visit a parse tree produced by the {@code windowDef} - * labeled alternative in {@link MixedFormatSqlExtendParser#windowSpec}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitWindowDef(MixedFormatSqlExtendParser.WindowDefContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#windowFrame}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitWindowFrame(MixedFormatSqlExtendParser.WindowFrameContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#frameBound}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitFrameBound(MixedFormatSqlExtendParser.FrameBoundContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#functionName}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitFunctionName(MixedFormatSqlExtendParser.FunctionNameContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#qualifiedName}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitQualifiedName(MixedFormatSqlExtendParser.QualifiedNameContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#errorCapturingIdentifier}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitErrorCapturingIdentifier(MixedFormatSqlExtendParser.ErrorCapturingIdentifierContext ctx); - /** - * Visit a parse tree produced by the {@code errorIdent} - * labeled alternative in {@link MixedFormatSqlExtendParser#errorCapturingIdentifierExtra}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitErrorIdent(MixedFormatSqlExtendParser.ErrorIdentContext ctx); - /** - * Visit a parse tree produced by the {@code realIdent} - * labeled alternative in {@link MixedFormatSqlExtendParser#errorCapturingIdentifierExtra}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitRealIdent(MixedFormatSqlExtendParser.RealIdentContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#identifier}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitIdentifier(MixedFormatSqlExtendParser.IdentifierContext ctx); - /** - * Visit a parse tree produced by the {@code unquotedIdentifier} - * labeled alternative in {@link MixedFormatSqlExtendParser#strictIdentifier}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitUnquotedIdentifier(MixedFormatSqlExtendParser.UnquotedIdentifierContext ctx); - /** - * Visit a parse tree produced by the {@code quotedIdentifierAlternative} - * labeled alternative in {@link MixedFormatSqlExtendParser#strictIdentifier}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitQuotedIdentifierAlternative(MixedFormatSqlExtendParser.QuotedIdentifierAlternativeContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#quotedIdentifier}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitQuotedIdentifier(MixedFormatSqlExtendParser.QuotedIdentifierContext ctx); - /** - * Visit a parse tree produced by the {@code exponentLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitExponentLiteral(MixedFormatSqlExtendParser.ExponentLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code decimalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitDecimalLiteral(MixedFormatSqlExtendParser.DecimalLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code legacyDecimalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitLegacyDecimalLiteral(MixedFormatSqlExtendParser.LegacyDecimalLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code integerLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitIntegerLiteral(MixedFormatSqlExtendParser.IntegerLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code bigIntLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitBigIntLiteral(MixedFormatSqlExtendParser.BigIntLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code smallIntLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitSmallIntLiteral(MixedFormatSqlExtendParser.SmallIntLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code tinyIntLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitTinyIntLiteral(MixedFormatSqlExtendParser.TinyIntLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code doubleLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitDoubleLiteral(MixedFormatSqlExtendParser.DoubleLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code floatLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitFloatLiteral(MixedFormatSqlExtendParser.FloatLiteralContext ctx); - /** - * Visit a parse tree produced by the {@code bigDecimalLiteral} - * labeled alternative in {@link MixedFormatSqlExtendParser#number}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitBigDecimalLiteral(MixedFormatSqlExtendParser.BigDecimalLiteralContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#ansiNonReserved}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitAnsiNonReserved(MixedFormatSqlExtendParser.AnsiNonReservedContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#strictNonReserved}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitStrictNonReserved(MixedFormatSqlExtendParser.StrictNonReservedContext ctx); - /** - * Visit a parse tree produced by {@link MixedFormatSqlExtendParser#nonReserved}. - * @param ctx the parse tree - * @return the visitor result - */ - T visitNonReserved(MixedFormatSqlExtendParser.NonReservedContext ctx); -} \ No newline at end of file diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/MixedFormatSparkCatalog.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/MixedFormatSparkCatalog.java deleted file mode 100644 index ceb196f133..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/MixedFormatSparkCatalog.java +++ /dev/null @@ -1,306 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark; - -import static org.apache.amoro.spark.mixed.SparkSQLProperties.USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES; -import static org.apache.amoro.spark.mixed.SparkSQLProperties.USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES_DEFAULT; -import static org.apache.iceberg.spark.SparkSQLProperties.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE; - -import org.apache.amoro.hive.utils.CatalogUtil; -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; -import org.apache.amoro.spark.mixed.MixedSparkCatalogBase; -import org.apache.amoro.spark.mixed.MixedTableStoreType; -import org.apache.amoro.spark.table.MixedSparkTable; -import org.apache.amoro.spark.table.SparkChangeTable; -import org.apache.amoro.table.BasicUnkeyedTable; -import org.apache.amoro.table.KeyedTable; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.amoro.table.TableBuilder; -import org.apache.amoro.table.TableIdentifier; -import org.apache.amoro.table.UnkeyedTable; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Transaction; -import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; -import org.apache.spark.sql.connector.catalog.Identifier; -import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.apache.spark.sql.connector.catalog.TableChange; -import org.apache.spark.sql.connector.catalog.TableChange.ColumnChange; -import org.apache.spark.sql.connector.catalog.TableChange.RemoveProperty; -import org.apache.spark.sql.connector.catalog.TableChange.SetProperty; -import org.apache.spark.sql.connector.expressions.Transform; -import org.apache.spark.sql.types.StructType; -import scala.Option; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class MixedFormatSparkCatalog extends MixedSparkCatalogBase { - - @Override - public Table loadTable(Identifier ident) throws NoSuchTableException { - checkAndRefreshCatalogMeta(); - TableIdentifier identifier; - MixedTable table; - try { - if (isInnerTableIdentifier(ident)) { - MixedTableStoreType type = MixedTableStoreType.from(ident.name()); - identifier = buildInnerTableIdentifier(ident); - table = catalog.loadTable(identifier); - return loadInnerTable(table, type); - } else { - identifier = buildIdentifier(ident); - table = catalog.loadTable(identifier); - } - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - throw new NoSuchTableException(ident); - } - return MixedSparkTable.ofMixedTable(table, catalog, name()); - } - - private Table loadInnerTable(MixedTable table, MixedTableStoreType type) { - if (type != null) { - switch (type) { - case CHANGE: - return new SparkChangeTable( - (BasicUnkeyedTable) table.asKeyedTable().changeTable(), false); - default: - throw new IllegalArgumentException("Unknown inner table type: " + type); - } - } else { - throw new IllegalArgumentException("Table does not exist: " + table); - } - } - - @Override - public Table createTable( - Identifier ident, StructType schema, Transform[] transforms, Map properties) - throws TableAlreadyExistsException { - checkAndRefreshCatalogMeta(); - properties = Maps.newHashMap(properties); - Schema finalSchema = checkAndConvertSchema(schema, properties); - TableIdentifier identifier = buildIdentifier(ident); - TableBuilder builder = catalog.newTableBuilder(identifier, finalSchema); - PartitionSpec spec = Spark3Util.toPartitionSpec(finalSchema, transforms); - if (properties.containsKey(TableCatalog.PROP_LOCATION) - && isIdentifierLocation(properties.get(TableCatalog.PROP_LOCATION), ident)) { - properties.remove(TableCatalog.PROP_LOCATION); - } - try { - if (properties.containsKey("primary.keys")) { - PrimaryKeySpec primaryKeySpec = - PrimaryKeySpec.fromDescription(finalSchema, properties.get("primary.keys")); - properties.remove("primary.keys"); - builder - .withPartitionSpec(spec) - .withProperties(properties) - .withPrimaryKeySpec(primaryKeySpec); - } else { - builder.withPartitionSpec(spec).withProperties(properties); - } - MixedTable table = builder.create(); - return MixedSparkTable.ofMixedTable(table, catalog, name()); - } catch (AlreadyExistsException e) { - throw new TableAlreadyExistsException("Table " + ident + " already exists", Option.apply(e)); - } - } - - private Schema checkAndConvertSchema(StructType schema, Map properties) { - Schema convertSchema; - boolean useTimestampWithoutZoneInNewTables; - SparkSession sparkSession = SparkSession.active(); - if (CatalogUtil.isMixedHiveCatalog(catalog)) { - useTimestampWithoutZoneInNewTables = true; - } else { - useTimestampWithoutZoneInNewTables = - Boolean.parseBoolean( - sparkSession - .conf() - .get( - USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES, - USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES_DEFAULT)); - } - if (useTimestampWithoutZoneInNewTables) { - sparkSession.conf().set(HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, true); - convertSchema = SparkSchemaUtil.convert(schema, true); - } else { - convertSchema = SparkSchemaUtil.convert(schema, false); - } - - // schema add primary keys - if (properties.containsKey("primary.keys")) { - PrimaryKeySpec primaryKeySpec = - PrimaryKeySpec.fromDescription(convertSchema, properties.get("primary.keys")); - List primaryKeys = primaryKeySpec.fieldNames(); - Set pkSet = new HashSet<>(primaryKeys); - Set identifierFieldIds = new HashSet<>(); - List columnsWithPk = new ArrayList<>(); - convertSchema - .columns() - .forEach( - nestedField -> { - if (pkSet.contains(nestedField.name())) { - columnsWithPk.add(nestedField.asRequired()); - identifierFieldIds.add(nestedField.fieldId()); - } else { - columnsWithPk.add(nestedField); - } - }); - return new Schema(columnsWithPk, identifierFieldIds); - } - return convertSchema; - } - - @Override - public Table alterTable(Identifier ident, TableChange... changes) throws NoSuchTableException { - TableIdentifier identifier = buildIdentifier(ident); - MixedTable table; - try { - table = catalog.loadTable(identifier); - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - throw new NoSuchTableException(ident); - } - if (table.isUnkeyedTable()) { - alterUnKeyedTable(table.asUnkeyedTable(), changes); - return MixedSparkTable.ofMixedTable(table, catalog, name()); - } else if (table.isKeyedTable()) { - alterKeyedTable(table.asKeyedTable(), changes); - return MixedSparkTable.ofMixedTable(table, catalog, name()); - } - throw new UnsupportedOperationException("Unsupported alter table"); - } - - private void alterKeyedTable(KeyedTable table, TableChange... changes) { - List schemaChanges = new ArrayList<>(); - List propertyChanges = new ArrayList<>(); - for (TableChange change : changes) { - if (change instanceof ColumnChange) { - schemaChanges.add(change); - } else if (change instanceof SetProperty) { - propertyChanges.add(change); - } else if (change instanceof RemoveProperty) { - propertyChanges.add(change); - } else { - throw new UnsupportedOperationException("Cannot apply unknown table change: " + change); - } - } - commitKeyedChanges(table, schemaChanges, propertyChanges); - } - - private void commitKeyedChanges( - KeyedTable table, List schemaChanges, List propertyChanges) { - if (!schemaChanges.isEmpty()) { - Spark3Util.applySchemaChanges(table.updateSchema(), schemaChanges).commit(); - } - - if (!propertyChanges.isEmpty()) { - Spark3Util.applyPropertyChanges(table.updateProperties(), propertyChanges).commit(); - } - } - - private void alterUnKeyedTable(UnkeyedTable table, TableChange... changes) { - SetProperty setLocation = null; - SetProperty setSnapshotId = null; - SetProperty pickSnapshotId = null; - List propertyChanges = new ArrayList<>(); - List schemaChanges = new ArrayList<>(); - - for (TableChange change : changes) { - if (change instanceof SetProperty) { - SetProperty set = (SetProperty) change; - if (TableCatalog.PROP_LOCATION.equalsIgnoreCase(set.property())) { - setLocation = set; - } else if ("current-snapshot-id".equalsIgnoreCase(set.property())) { - setSnapshotId = set; - } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(set.property())) { - pickSnapshotId = set; - } else if ("sort-order".equalsIgnoreCase(set.property())) { - throw new UnsupportedOperationException( - "Cannot specify the 'sort-order' because it's a reserved table " - + "property. Please use the command 'ALTER TABLE ... WRITE ORDERED BY' to specify write sort-orders."); - } else { - propertyChanges.add(set); - } - } else if (change instanceof RemoveProperty) { - propertyChanges.add(change); - } else if (change instanceof ColumnChange) { - schemaChanges.add(change); - } else { - throw new UnsupportedOperationException("Cannot apply unknown table change: " + change); - } - } - - commitUnKeyedChanges( - table, setLocation, setSnapshotId, pickSnapshotId, propertyChanges, schemaChanges); - } - - protected void commitUnKeyedChanges( - UnkeyedTable table, - SetProperty setLocation, - SetProperty setSnapshotId, - SetProperty pickSnapshotId, - List propertyChanges, - List schemaChanges) { - // don't allow setting the snapshot and picking a commit at the same time because order is - // ambiguous and choosing - // one order leads to different results - Preconditions.checkArgument( - setSnapshotId == null || pickSnapshotId == null, - "Cannot set the current the current snapshot ID and cherry-pick snapshot changes"); - - if (setSnapshotId != null) { - long newSnapshotId = Long.parseLong(setSnapshotId.value()); - table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit(); - } - - // if updating the table snapshot, perform that update first in case it fails - if (pickSnapshotId != null) { - long newSnapshotId = Long.parseLong(pickSnapshotId.value()); - table.manageSnapshots().cherrypick(newSnapshotId).commit(); - } - - Transaction transaction = table.newTransaction(); - - if (setLocation != null) { - transaction.updateLocation().setLocation(setLocation.value()).commit(); - } - - if (!propertyChanges.isEmpty()) { - Spark3Util.applyPropertyChanges(transaction.updateProperties(), propertyChanges).commit(); - } - - if (!schemaChanges.isEmpty()) { - Spark3Util.applySchemaChanges(transaction.updateSchema(), schemaChanges).commit(); - } - - transaction.commitTransaction(); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/MixedFormatSparkSessionCatalog.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/MixedFormatSparkSessionCatalog.java deleted file mode 100644 index 19abfda8c1..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/MixedFormatSparkSessionCatalog.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark; - -import org.apache.amoro.spark.mixed.MixedSessionCatalogBase; -import org.apache.amoro.spark.mixed.MixedSparkCatalogBase; -import org.apache.spark.sql.connector.catalog.SupportsNamespaces; -import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; - -/** - * A Spark catalog that can also load non-Iceberg tables. - * - * @param CatalogPlugin class to avoid casting to TableCatalog and SupportsNamespaces. - */ -public class MixedFormatSparkSessionCatalog - extends MixedSessionCatalogBase { - - protected MixedSparkCatalogBase buildTargetCatalog( - String name, CaseInsensitiveStringMap options) { - MixedSparkCatalogBase newCatalog = new MixedFormatSparkCatalog(); - newCatalog.initialize(name, options); - return newCatalog; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/Spark32Adapter.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/Spark32Adapter.java deleted file mode 100644 index d2a7b5c499..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/Spark32Adapter.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark; - -import org.apache.amoro.spark.util.ExpressionHelper; - -public class Spark32Adapter implements SparkAdapter { - ExpressionHelper expressionHelper = new ExpressionHelper(); - - @Override - public ExpressionHelper expressions() { - return expressionHelper; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkAdapter.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkAdapter.java deleted file mode 100644 index 423b237bd0..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkAdapter.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark; - -import org.apache.amoro.spark.util.ExpressionHelper; - -/** - * This interface will provider some util or helper object to shield api differences in different - * spark versions. - */ -public interface SparkAdapter { - - /** - * A helper object to help build spark expressions {@link - * org.apache.spark.sql.connector.expressions.Expression}, and provider a covert method to help - * covert {@link org.apache.spark.sql.connector.expressions.Expression} to {@link - * org.apache.spark.sql.catalyst.expressions.Expression} - * - * @return expression helper object - */ - ExpressionHelper expressions(); -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkAdapterLoader.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkAdapterLoader.java deleted file mode 100644 index 4cd193aae6..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkAdapterLoader.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark; - -/** - * A util class to load spark adapter via spark version. This util class will move to spark3-common - * module if multi spark modules added. - */ -public class SparkAdapterLoader { - private static final SparkAdapter adapter = new Spark32Adapter(); - - /** - * This method will implement as SPI if multi spark modules added. - * - * @return A SparkAdapter objects - */ - public static SparkAdapter getOrLoad() { - return adapter; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkInternalRowCastWrapper.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkInternalRowCastWrapper.java deleted file mode 100644 index f5a6aa4782..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkInternalRowCastWrapper.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark; - -import org.apache.amoro.data.ChangeAction; -import org.apache.amoro.spark.sql.utils.ProjectingInternalRow; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.catalyst.util.ArrayData; -import org.apache.spark.sql.catalyst.util.MapData; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.unsafe.types.CalendarInterval; -import org.apache.spark.unsafe.types.UTF8String; -import scala.collection.Seq; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; - -/** cast internal row to upsert internal row */ -public class SparkInternalRowCastWrapper extends GenericInternalRow { - private final InternalRow row; - private final StructType schema; - private ChangeAction changeAction = ChangeAction.INSERT; - private List dataTypeList; - - public SparkInternalRowCastWrapper( - InternalRow row, ChangeAction changeAction, StructType schema) { - this.row = row; - this.changeAction = changeAction; - if (row instanceof ProjectingInternalRow) { - this.schema = ((ProjectingInternalRow) row).schema(); - } else { - this.schema = schema; - } - } - - public StructType getSchema() { - return this.schema; - } - - @Override - public Object genericGet(int ordinal) { - return row.get(ordinal, schema.apply(ordinal).dataType()); - } - - @Override - public Seq toSeq(Seq fieldTypes) { - return super.toSeq(fieldTypes); - } - - @Override - public int numFields() { - return schema.size() / 2; - } - - @Override - public void setNullAt(int i) { - super.setNullAt(i); - } - - @Override - public void update(int i, Object value) { - super.update(i, value); - } - - @Override - public boolean isNullAt(int ordinal) { - dataTypeList = - Arrays.stream(schema.fields()).map(StructField::dataType).collect(Collectors.toList()); - return row.get(ordinal, dataTypeList.get(ordinal)) == null; - } - - @Override - public Object get(int pos, DataType dt) { - return row.get(pos, dt); - } - - @Override - public boolean getBoolean(int ordinal) { - return super.getBoolean(ordinal); - } - - @Override - public byte getByte(int ordinal) { - return super.getByte(ordinal); - } - - @Override - public short getShort(int ordinal) { - return super.getShort(ordinal); - } - - @Override - public int getInt(int ordinal) { - return super.getInt(ordinal); - } - - @Override - public long getLong(int ordinal) { - return super.getLong(ordinal); - } - - @Override - public float getFloat(int ordinal) { - return super.getFloat(ordinal); - } - - @Override - public double getDouble(int ordinal) { - return super.getDouble(ordinal); - } - - @Override - public Decimal getDecimal(int ordinal, int precision, int scale) { - return super.getDecimal(ordinal, precision, scale); - } - - @Override - public UTF8String getUTF8String(int ordinal) { - return super.getUTF8String(ordinal); - } - - @Override - public byte[] getBinary(int ordinal) { - return super.getBinary(ordinal); - } - - @Override - public ArrayData getArray(int ordinal) { - return super.getArray(ordinal); - } - - @Override - public CalendarInterval getInterval(int ordinal) { - return super.getInterval(ordinal); - } - - @Override - public MapData getMap(int ordinal) { - return super.getMap(ordinal); - } - - @Override - public InternalRow getStruct(int ordinal, int numFields) { - return super.getStruct(ordinal, numFields); - } - - public InternalRow getRow() { - return this.row; - } - - public ChangeAction getChangeAction() { - return changeAction; - } - - @Override - public String toString() { - return super.toString(); - } - - @Override - public GenericInternalRow copy() { - return super.copy(); - } - - @Override - public boolean equals(Object o) { - return super.equals(o); - } - - @Override - public int hashCode() { - return super.hashCode(); - } - - @Override - public Object[] values() { - return super.values(); - } - - public InternalRow setFileOffset(Long fileOffset) { - List dataTypeList = - Arrays.stream(schema.fields()).map(StructField::dataType).collect(Collectors.toList()); - List objectSeq = new ArrayList<>(dataTypeList.size() + 1); - row.toSeq(schema).toStream().foreach(objectSeq::add); - objectSeq.add(fileOffset); - return new GenericInternalRow(objectSeq.toArray()); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkInternalRowWrapper.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkInternalRowWrapper.java deleted file mode 100644 index 118b5c03f3..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkInternalRowWrapper.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark; - -import org.apache.iceberg.StructLike; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.BinaryType; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.DecimalType; -import org.apache.spark.sql.types.StringType; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -import java.nio.ByteBuffer; -import java.util.function.BiFunction; -import java.util.stream.Stream; - -/** - * this class copied from iceberg org.apache.iceberg.spark.source.InternalRowWrapper for - * InternalRowWrapper is not public class - */ -public class SparkInternalRowWrapper implements StructLike { - private final DataType[] types; - private final BiFunction[] getters; - private InternalRow row = null; - - public SparkInternalRowWrapper(StructType rowType) { - this.types = Stream.of(rowType.fields()).map(StructField::dataType).toArray(DataType[]::new); - this.getters = Stream.of(types).map(SparkInternalRowWrapper::getter).toArray(BiFunction[]::new); - } - - @Override - public int size() { - return types.length; - } - - @Override - public T get(int pos, Class javaClass) { - if (row.isNullAt(pos)) { - return null; - } else if (getters[pos] != null) { - return javaClass.cast(getters[pos].apply(row, pos)); - } - - return javaClass.cast(row.get(pos, types[pos])); - } - - @Override - public void set(int pos, T value) { - row.update(pos, value); - } - - public SparkInternalRowWrapper wrap(InternalRow internalRow) { - this.row = internalRow; - return this; - } - - private static BiFunction getter(DataType type) { - if (type instanceof StringType) { - return (row, pos) -> row.getUTF8String(pos).toString(); - } else if (type instanceof DecimalType) { - DecimalType decimal = (DecimalType) type; - return (row, pos) -> - row.getDecimal(pos, decimal.precision(), decimal.scale()).toJavaBigDecimal(); - } else if (type instanceof BinaryType) { - return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); - } else if (type instanceof StructType) { - StructType structType = (StructType) type; - SparkInternalRowWrapper nestedWrapper = new SparkInternalRowWrapper(structType); - return (row, pos) -> nestedWrapper.wrap(row.getStruct(pos, structType.size())); - } - - return null; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkUnifiedCatalog.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkUnifiedCatalog.java deleted file mode 100644 index 4dd464d554..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkUnifiedCatalog.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark; - -import org.apache.amoro.TableFormat; -import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException; -import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; -import org.apache.spark.sql.connector.catalog.FunctionCatalog; -import org.apache.spark.sql.connector.catalog.Identifier; -import org.apache.spark.sql.connector.catalog.SupportsNamespaces; -import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; -import org.apache.spark.sql.connector.iceberg.catalog.ProcedureCatalog; - -public class SparkUnifiedCatalog extends SparkUnifiedCatalogBase - implements TableCatalog, SupportsNamespaces, ProcedureCatalog, FunctionCatalog { - /** - * List the functions in a namespace from the catalog. - * - *

If there are no functions in the namespace, implementations should return an empty array. - * - * @param namespace a multi-part namespace - * @return an array of Identifiers for functions - * @throws NoSuchNamespaceException If the namespace does not exist (optional). - */ - @Override - public Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceException { - TableCatalog tableCatalog = tableCatalog(TableFormat.ICEBERG); - if (tableCatalog instanceof FunctionCatalog) { - return ((FunctionCatalog) tableCatalog).listFunctions(namespace); - } - throw new NoSuchNamespaceException(namespace); - } - - /** - * Load a function by {@link Identifier identifier} from the catalog. - * - * @param ident a function identifier - * @return an unbound function instance - * @throws NoSuchFunctionException If the function doesn't exist - */ - @Override - public UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionException { - - TableCatalog tableCatalog = tableCatalog(TableFormat.ICEBERG); - if (tableCatalog instanceof FunctionCatalog) { - return ((FunctionCatalog) tableCatalog).loadFunction(ident); - } - throw new NoSuchFunctionException(ident); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkUnifiedSessionCatalog.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkUnifiedSessionCatalog.java deleted file mode 100644 index 2fab4b9bf1..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/SparkUnifiedSessionCatalog.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark; - -import org.apache.spark.sql.connector.catalog.FunctionCatalog; -import org.apache.spark.sql.connector.catalog.SupportsNamespaces; -import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; - -/** @Auth: hzwangtao6 @Time: 2024/5/24 14:04 @Description: */ -public class SparkUnifiedSessionCatalog< - T extends TableCatalog & SupportsNamespaces & FunctionCatalog> - extends SparkUnifiedSessionCatalogBase { - - @Override - protected SparkUnifiedCatalogBase createUnifiedCatalog( - String name, CaseInsensitiveStringMap options) { - return new SparkUnifiedCatalog(); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/command/MigrateToMixedFormatCommand.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/command/MigrateToMixedFormatCommand.java deleted file mode 100644 index 8785cbf0ff..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/command/MigrateToMixedFormatCommand.java +++ /dev/null @@ -1,349 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.command; - -import org.apache.amoro.shade.guava32.com.google.common.base.Joiner; -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableList; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; -import org.apache.amoro.spark.MixedFormatSparkCatalog; -import org.apache.amoro.spark.MixedFormatSparkSessionCatalog; -import org.apache.amoro.spark.table.MixedSparkTable; -import org.apache.amoro.spark.table.UnkeyedSparkTable; -import org.apache.amoro.spark.util.MixedFormatSparkUtils; -import org.apache.amoro.table.UnkeyedTable; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.MetricsConfig; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.TableMigrationUtil; -import org.apache.iceberg.hadoop.Util; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.SparkTableUtil; -import org.apache.spark.sql.AnalysisException; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.TableIdentifier; -import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException; -import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; -import org.apache.spark.sql.catalyst.catalog.CatalogTable; -import org.apache.spark.sql.connector.catalog.CatalogManager; -import org.apache.spark.sql.connector.catalog.Identifier; -import org.apache.spark.sql.connector.catalog.SupportsNamespaces; -import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.apache.spark.sql.connector.catalog.V1Table; -import org.apache.spark.sql.connector.expressions.Transform; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import scala.Option; -import scala.Some; - -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -/** - * migrate a v1 table to mixed-format table. will reuse file in v1 table , but delete metadata in - * session catalog - */ -public class MigrateToMixedFormatCommand implements MixedFormatSparkCommand { - private static final Logger LOG = LoggerFactory.getLogger(MigrateToMixedFormatCommand.class); - - private static final String V1TABLE_BACKUP_SUFFIX = "_BAK_MIXED_"; - protected static final List EXCLUDED_PROPERTIES = - ImmutableList.of("path", "transient_lastDdlTime", "serialization.format"); - - private static final StructType OUTPUT_TYPE = - new StructType( - new StructField[] { - new StructField("partition", DataTypes.StringType, false, Metadata.empty()), - new StructField("file_counts", DataTypes.IntegerType, false, Metadata.empty()) - }); - - private final SparkSession spark; - private final TableCatalog sourceCatalog; - private final Identifier sourceIdentifier; - private final Identifier backupV1TableIdentifier; - private final TableCatalog targetCatalog; - private final Identifier targetIdentifier; - - protected MigrateToMixedFormatCommand( - TableCatalog sourceCatalog, - Identifier sourceIdentifier, - TableCatalog catalog, - Identifier identifier, - SparkSession spark) { - this.spark = spark; - this.sourceCatalog = sourceCatalog; - this.targetCatalog = catalog; - this.targetIdentifier = identifier; - this.sourceIdentifier = sourceIdentifier; - String backupName = sourceIdentifier.name(); - backupV1TableIdentifier = Identifier.of(sourceIdentifier.namespace(), backupName); - } - - @Override - public String name() { - return "MigrateToMixedFormatTable"; - } - - @Override - public StructType outputType() { - return OUTPUT_TYPE; - } - - @Override - public Row[] execute() throws AnalysisException { - List dataFiles; - TableIdentifier ident; - PartitionSpec spec; - Schema schema; - LOG.info( - "start to migrate {} to {}, using temp backup table {}", - sourceIdentifier, - targetIdentifier, - backupV1TableIdentifier); - V1Table sourceTable = loadV1Table(sourceCatalog, backupV1TableIdentifier); - ident = - new TableIdentifier( - backupV1TableIdentifier.name(), Some.apply(backupV1TableIdentifier.namespace()[0])); - dataFiles = loadDataFiles(ident); - UnkeyedTable table = createUnkeyedTable(sourceTable); - - spec = table.spec(); - - AppendFiles appendFiles = table.newAppend(); - dataFiles.forEach(appendFiles::appendFile); - appendFiles.commit(); - - LOG.info( - "migrate table {} finished, remove metadata of backup {} table", - targetIdentifier, - backupV1TableIdentifier); - - if (PartitionSpec.unpartitioned().equals(spec)) { - return new Row[] {RowFactory.create("ALL", dataFiles.size())}; - } - - Map> partitions = Maps.newHashMap(); - dataFiles.forEach( - d -> { - String partition = spec.partitionToPath(d.partition()); - List df = partitions.computeIfAbsent(partition, p -> Lists.newArrayList()); - df.add(d); - }); - return partitions.keySet().stream() - .sorted() - .map(p -> RowFactory.create(p, partitions.get(p).size())) - .toArray(Row[]::new); - } - - private List loadDataFiles(TableIdentifier ident) throws AnalysisException { - PartitionSpec spec = - SparkSchemaUtil.specForTable(spark, ident.database().get() + "." + ident.table()); - - if (spec.equals(PartitionSpec.unpartitioned())) { - return listUnPartitionedSparkTable(spark, ident); - } else { - List sparkPartitions = - SparkTableUtil.getPartitions(spark, ident, Maps.newHashMap()); - Preconditions.checkArgument( - !sparkPartitions.isEmpty(), "Cannot find any partitions in table %s", ident); - return listPartitionDataFiles(spark, sparkPartitions, spec); - } - } - - private UnkeyedTable createUnkeyedTable(V1Table sourceTable) - throws TableAlreadyExistsException, NoSuchNamespaceException { - Map properties = Maps.newHashMap(); - properties.putAll(sourceTable.properties()); - EXCLUDED_PROPERTIES.forEach(properties::remove); - properties.put(TableCatalog.PROP_PROVIDER, "arctic"); - properties.put("migrated", "true"); - - StructType schema = sourceTable.schema(); - Transform[] partitions = sourceTable.partitioning(); - boolean threw = true; - Table table = null; - try { - table = targetCatalog.createTable(targetIdentifier, schema, partitions, properties); - if (table instanceof UnkeyedSparkTable) { - threw = false; - return ((UnkeyedSparkTable) table).table(); - } else if (table instanceof MixedSparkTable) { - threw = false; - return ((MixedSparkTable) table).table().asUnkeyedTable(); - } - throw new IllegalStateException("target table must be un-keyed table"); - } finally { - if (threw && table != null) { - try { - targetCatalog.dropTable(targetIdentifier); - } catch (Exception e) { - LOG.warn("error when rollback table", e); - } - } - } - } - - private static V1Table loadV1Table(TableCatalog catalog, Identifier identifier) - throws NoSuchTableException { - Table table = catalog.loadTable(identifier); - Preconditions.checkArgument(table instanceof V1Table, "source table must be V1Table"); - return (V1Table) table; - } - - private static List listUnPartitionedSparkTable( - SparkSession spark, TableIdentifier sourceTableIdent) - throws NoSuchDatabaseException, NoSuchTableException { - CatalogTable sourceTable = spark.sessionState().catalog().getTableMetadata(sourceTableIdent); - Option format = - sourceTable.storage().serde().nonEmpty() - ? sourceTable.storage().serde() - : sourceTable.provider(); - Preconditions.checkArgument(format.nonEmpty(), "Could not determine table format"); - - Map partition = Collections.emptyMap(); - PartitionSpec spec = PartitionSpec.unpartitioned(); - Configuration conf = spark.sessionState().newHadoopConf(); - MetricsConfig metricsConfig = MetricsConfig.getDefault(); - return TableMigrationUtil.listPartition( - partition, - Util.uriToString(sourceTable.location()), - format.get(), - spec, - conf, - metricsConfig, - null); - } - - private static List listPartitionDataFiles( - SparkSession spark, List partitions, PartitionSpec spec) { - - Configuration conf = spark.sessionState().newHadoopConf(); - MetricsConfig metricsConfig = MetricsConfig.getDefault(); - - return partitions.stream() - .map( - p -> - TableMigrationUtil.listPartition( - p.getValues(), p.getUri(), p.getFormat(), spec, conf, metricsConfig, null)) - .flatMap(Collection::stream) - .collect(Collectors.toList()); - } - - public static Builder newBuilder(SparkSession spark) { - return new Builder(spark); - } - - public static class Builder { - - List source; - List target; - - SparkSession spark; - - private Builder(SparkSession spark) { - this.spark = spark; - } - - public Builder withSource(List source) { - this.source = source; - return this; - } - - public Builder withTarget(List target) { - this.target = target; - return this; - } - - public MigrateToMixedFormatCommand build() throws NoSuchTableException { - MixedFormatSparkUtils.TableCatalogAndIdentifier tableCatalogAndIdentifier = - MixedFormatSparkUtils.tableCatalogAndIdentifier(spark, source); - TableCatalog sourceCatalog = tableCatalogAndIdentifier.catalog(); - Identifier sourceTableIdentifier = tableCatalogAndIdentifier.identifier(); - - checkSourceCatalogAndTable(sourceCatalog, sourceTableIdentifier); - - tableCatalogAndIdentifier = MixedFormatSparkUtils.tableCatalogAndIdentifier(spark, target); - TableCatalog targetCatalog = tableCatalogAndIdentifier.catalog(); - Identifier targetTableIdentifier = tableCatalogAndIdentifier.identifier(); - - checkTargetCatalog(targetCatalog); - checkTargetTable(targetCatalog, targetTableIdentifier); - - return new MigrateToMixedFormatCommand( - sourceCatalog, sourceTableIdentifier, targetCatalog, targetTableIdentifier, spark); - } - - private void checkSourceCatalogAndTable(TableCatalog catalog, Identifier identifier) - throws NoSuchTableException { - Preconditions.checkArgument( - catalog.name().equalsIgnoreCase(CatalogManager.SESSION_CATALOG_NAME()), - "source table must in session catalog, current table is %s", - catalog.name()); - - Preconditions.checkArgument( - catalog.tableExists(identifier), - "source table %s does not exist in catalog %s", - Joiner.on(".").join(identifier.namespace()), - catalog.name()); - loadV1Table(catalog, identifier); - } - - private void checkTargetCatalog(TableCatalog catalog) { - Preconditions.checkArgument( - catalog instanceof MixedFormatSparkCatalog - || catalog instanceof MixedFormatSparkSessionCatalog, - "target catalog must be %s", - MixedFormatSparkCatalog.class.getName()); - } - - private void checkTargetTable(TableCatalog catalog, Identifier identifier) { - Preconditions.checkArgument( - catalog instanceof SupportsNamespaces, "The target catalog must support namespace"); - Preconditions.checkArgument( - ((SupportsNamespaces) catalog).namespaceExists(identifier.namespace()), - "database %s does not exist in catalog %s", - Joiner.on(".").join(identifier.namespace()), - catalog.name()); - - List nameParts = Lists.newArrayList(identifier.namespace()); - nameParts.add(identifier.name()); - Preconditions.checkArgument( - !catalog.tableExists(identifier), - "target table %s already exist in catalog %s", - Joiner.on(".").join(nameParts), - catalog.name()); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/command/MixedFormatSparkCommand.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/command/MixedFormatSparkCommand.java deleted file mode 100644 index 53817f8e18..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/command/MixedFormatSparkCommand.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.command; - -import org.apache.spark.sql.AnalysisException; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.types.StructType; - -public interface MixedFormatSparkCommand { - - String name(); - - StructType outputType(); - - Row[] execute() throws AnalysisException; - - default String execInfo() { - return ""; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/InternalRowFileAppenderFactory.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/InternalRowFileAppenderFactory.java deleted file mode 100644 index d384ea57f0..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/InternalRowFileAppenderFactory.java +++ /dev/null @@ -1,372 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.io; - -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.MetricsConfig; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.deletes.EqualityDeleteWriter; -import org.apache.iceberg.deletes.PositionDeleteWriter; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.io.DataWriter; -import org.apache.iceberg.io.DeleteSchemaUtil; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.AdaptHiveParquet; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.data.AdaptHiveSparkParquetWriters; -import org.apache.iceberg.spark.data.SparkAvroWriter; -import org.apache.iceberg.spark.data.SparkOrcWriter; -import org.apache.iceberg.spark.data.SparkParquetWriters; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.unsafe.types.UTF8String; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Map; - -/** - * This InternalRowFileAppenderFactory file is forked from - * org.apache.iceberg.spark.source.SparkAppenderFactory for SparkAppenderFactory is package-private - * class - */ -public class InternalRowFileAppenderFactory implements FileAppenderFactory { - - private final Map properties; - private final Schema writeSchema; - private final StructType dsSchema; - private final PartitionSpec spec; - private final int[] equalityFieldIds; - private final Schema eqDeleteRowSchema; - private final Schema posDeleteRowSchema; - - private StructType eqDeleteSparkType = null; - private StructType posDeleteSparkType = null; - private final boolean writeHive; - - public static Builder builderFor(Table table, Schema writeSchema, StructType dsSchema) { - return new Builder(table, writeSchema, dsSchema); - } - - protected InternalRowFileAppenderFactory( - Map properties, - Schema writeSchema, - StructType dsSchema, - PartitionSpec spec, - int[] equalityFieldIds, - Schema eqDeleteRowSchema, - Schema posDeleteRowSchema, - boolean writeHive) { - this.properties = properties; - this.writeSchema = writeSchema; - this.dsSchema = dsSchema; - this.spec = spec; - this.equalityFieldIds = equalityFieldIds; - this.eqDeleteRowSchema = eqDeleteRowSchema; - this.posDeleteRowSchema = posDeleteRowSchema; - this.writeHive = writeHive; - } - - public static class Builder { - private final Table table; - private final Schema writeSchema; - private final StructType dsSchema; - private PartitionSpec spec; - private int[] equalityFieldIds; - private Schema eqDeleteRowSchema; - private Schema posDeleteRowSchema; - private boolean writeHive; - - Builder(Table table, Schema writeSchema, StructType dsSchema) { - this.table = table; - this.spec = table.spec(); - this.writeSchema = writeSchema; - this.dsSchema = dsSchema; - } - - public Builder spec(PartitionSpec newSpec) { - this.spec = newSpec; - return this; - } - - public Builder equalityFieldIds(int[] newEqualityFieldIds) { - this.equalityFieldIds = newEqualityFieldIds; - return this; - } - - public Builder eqDeleteRowSchema(Schema newEqDeleteRowSchema) { - this.eqDeleteRowSchema = newEqDeleteRowSchema; - return this; - } - - public Builder posDelRowSchema(Schema newPosDelRowSchema) { - this.posDeleteRowSchema = newPosDelRowSchema; - return this; - } - - public Builder writeHive(boolean writeHive) { - this.writeHive = writeHive; - return this; - } - - public InternalRowFileAppenderFactory build() { - Preconditions.checkNotNull(table, "Table must not be null"); - Preconditions.checkNotNull(writeSchema, "Write Schema must not be null"); - Preconditions.checkNotNull(dsSchema, "DS Schema must not be null"); - if (equalityFieldIds != null) { - Preconditions.checkNotNull( - eqDeleteRowSchema, - "Equality Field Ids and Equality Delete Row Schema" + " must be set together"); - } - if (eqDeleteRowSchema != null) { - Preconditions.checkNotNull( - equalityFieldIds, - "Equality Field Ids and Equality Delete Row Schema" + " must be set together"); - } - - return new InternalRowFileAppenderFactory( - table.properties(), - writeSchema, - dsSchema, - spec, - equalityFieldIds, - eqDeleteRowSchema, - posDeleteRowSchema, - writeHive); - } - } - - private StructType lazyEqDeleteSparkType() { - if (eqDeleteSparkType == null) { - Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null"); - this.eqDeleteSparkType = SparkSchemaUtil.convert(eqDeleteRowSchema); - } - return eqDeleteSparkType; - } - - private StructType lazyPosDeleteSparkType() { - if (posDeleteSparkType == null) { - Preconditions.checkNotNull( - posDeleteRowSchema, "Position delete row schema shouldn't be null"); - this.posDeleteSparkType = SparkSchemaUtil.convert(posDeleteRowSchema); - } - return posDeleteSparkType; - } - - // todo control whether need adapt hive parquet - @Override - public FileAppender newAppender(OutputFile file, FileFormat fileFormat) { - MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties); - try { - switch (fileFormat) { - case PARQUET: - if (writeHive) { - return AdaptHiveParquet.write(file) - .createWriterFunc( - msgType -> AdaptHiveSparkParquetWriters.buildWriter(dsSchema, msgType)) - .setAll(properties) - .metricsConfig(metricsConfig) - .schema(writeSchema) - .overwrite() - .build(); - } else { - return Parquet.write(file) - .createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dsSchema, msgType)) - .setAll(properties) - .metricsConfig(metricsConfig) - .schema(writeSchema) - .overwrite() - .build(); - } - - case AVRO: - return Avro.write(file) - .createWriterFunc(ignored -> new SparkAvroWriter(dsSchema)) - .setAll(properties) - .schema(writeSchema) - .overwrite() - .build(); - - case ORC: - return ORC.write(file) - .createWriterFunc(SparkOrcWriter::new) - .setAll(properties) - .metricsConfig(metricsConfig) - .schema(writeSchema) - .overwrite() - .build(); - - default: - throw new UnsupportedOperationException("Cannot write unknown format: " + fileFormat); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public DataWriter newDataWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - return new DataWriter<>( - newAppender(file.encryptingOutputFile(), format), - format, - file.encryptingOutputFile().location(), - spec, - partition, - file.keyMetadata()); - } - - @Override - public EqualityDeleteWriter newEqDeleteWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - Preconditions.checkState( - equalityFieldIds != null && equalityFieldIds.length > 0, - "Equality field ids shouldn't be null or empty when creating equality-delete writer"); - Preconditions.checkNotNull( - eqDeleteRowSchema, - "Equality delete row schema shouldn't be null when creating equality-delete writer"); - - try { - switch (format) { - case PARQUET: - if (writeHive) { - return AdaptHiveParquet.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc( - msgType -> - AdaptHiveSparkParquetWriters.buildWriter(lazyEqDeleteSparkType(), msgType)) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - } else { - return Parquet.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc( - msgType -> SparkParquetWriters.buildWriter(lazyEqDeleteSparkType(), msgType)) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - } - case AVRO: - return Avro.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc(ignored -> new SparkAvroWriter(lazyEqDeleteSparkType())) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - case ORC: - return ORC.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc(SparkOrcWriter::new) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write equality-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException("Failed to create new equality delete writer", e); - } - } - - @Override - public PositionDeleteWriter newPosDeleteWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - try { - switch (format) { - case PARQUET: - StructType sparkPosDeleteSchema = - SparkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - if (writeHive) { - return AdaptHiveParquet.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc( - msgType -> - AdaptHiveSparkParquetWriters.buildWriter(sparkPosDeleteSchema, msgType)) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .transformPaths(path -> UTF8String.fromString(path.toString())) - .buildPositionWriter(); - } else { - return Parquet.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc( - msgType -> SparkParquetWriters.buildWriter(sparkPosDeleteSchema, msgType)) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .transformPaths(path -> UTF8String.fromString(path.toString())) - .buildPositionWriter(); - } - case AVRO: - return Avro.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc(ignored -> new SparkAvroWriter(lazyPosDeleteSparkType())) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .buildPositionWriter(); - - case ORC: - return ORC.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc(SparkOrcWriter::new) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .buildPositionWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write pos-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException("Failed to create new equality delete writer", e); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/SparkBaseTaskWriter.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/SparkBaseTaskWriter.java deleted file mode 100644 index 01f899f1a6..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/SparkBaseTaskWriter.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.io; - -import org.apache.amoro.io.AuthenticatedFileIO; -import org.apache.amoro.io.writer.BaseTaskWriter; -import org.apache.amoro.io.writer.OutputFileFactory; -import org.apache.amoro.spark.SparkInternalRowWrapper; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructType; - -public class SparkBaseTaskWriter extends BaseTaskWriter { - - private final StructType structType; - - public SparkBaseTaskWriter( - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory outputFileFactory, - AuthenticatedFileIO io, - long targetFileSize, - long mask, - Schema schema, - PartitionSpec spec, - PrimaryKeySpec primaryKeySpec, - boolean orderedWriter) { - super( - format, - appenderFactory, - outputFileFactory, - io, - targetFileSize, - mask, - schema, - spec, - primaryKeySpec, - orderedWriter); - this.structType = SparkSchemaUtil.convert(schema); - } - - @Override - protected StructLike asStructLike(InternalRow data) { - return new SparkInternalRowWrapper(structType).wrap(data); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/SparkChangeTaskWriter.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/SparkChangeTaskWriter.java deleted file mode 100644 index 67eea9c043..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/SparkChangeTaskWriter.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.io; - -import org.apache.amoro.data.ChangeAction; -import org.apache.amoro.io.AuthenticatedFileIO; -import org.apache.amoro.io.writer.ChangeTaskWriter; -import org.apache.amoro.io.writer.OutputFileFactory; -import org.apache.amoro.spark.SparkInternalRowCastWrapper; -import org.apache.amoro.spark.SparkInternalRowWrapper; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.spark.sql.catalyst.InternalRow; - -/** change task writer */ -public class SparkChangeTaskWriter extends ChangeTaskWriter { - private final Schema schema; - - protected SparkChangeTaskWriter( - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory outputFileFactory, - AuthenticatedFileIO io, - long targetFileSize, - long mask, - Schema schema, - PartitionSpec spec, - PrimaryKeySpec primaryKeySpec, - boolean orderedWriter) { - super( - format, - appenderFactory, - outputFileFactory, - io, - targetFileSize, - mask, - schema, - spec, - primaryKeySpec, - orderedWriter); - this.schema = schema; - } - - @Override - protected StructLike asStructLike(InternalRow data) { - return new SparkInternalRowWrapper(SparkSchemaUtil.convert(schema)).wrap(data); - } - - @Override - protected InternalRow appendMetaColumns(InternalRow data, Long fileOffset) { - SparkInternalRowCastWrapper row = (SparkInternalRowCastWrapper) data; - return row.setFileOffset(fileOffset); - } - - @Override - protected ChangeAction action(InternalRow data) { - SparkInternalRowCastWrapper row = (SparkInternalRowCastWrapper) data; - return row.getChangeAction(); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/TaskWriters.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/TaskWriters.java deleted file mode 100644 index 25d8797efc..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/TaskWriters.java +++ /dev/null @@ -1,295 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.io; - -import org.apache.amoro.hive.io.writer.AdaptHiveOutputFileFactory; -import org.apache.amoro.hive.table.SupportHive; -import org.apache.amoro.io.writer.ChangeTaskWriter; -import org.apache.amoro.io.writer.CommonOutputFileFactory; -import org.apache.amoro.io.writer.OutputFileFactory; -import org.apache.amoro.properties.HiveTableProperties; -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.amoro.table.KeyedTable; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.amoro.table.TableProperties; -import org.apache.amoro.table.UnkeyedTable; -import org.apache.amoro.utils.SchemaUtil; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructType; - -import java.util.Locale; - -public class TaskWriters { - private final MixedTable table; - private Long transactionId; - private int partitionId = 0; - private long taskId = 0; - private StructType dsSchema; - private String hiveSubdirectory; - private boolean orderedWriter = false; - - private final boolean isHiveTable; - private final FileFormat fileFormat; - private final long fileSize; - private final long mask; - - protected TaskWriters(MixedTable table) { - this.table = table; - this.isHiveTable = table instanceof SupportHive; - - this.fileFormat = - FileFormat.valueOf( - (table - .properties() - .getOrDefault( - TableProperties.BASE_FILE_FORMAT, TableProperties.BASE_FILE_FORMAT_DEFAULT) - .toUpperCase(Locale.ENGLISH))); - this.fileSize = - PropertyUtil.propertyAsLong( - table.properties(), - TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, - TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); - this.mask = - PropertyUtil.propertyAsLong( - table.properties(), - TableProperties.BASE_FILE_INDEX_HASH_BUCKET, - TableProperties.BASE_FILE_INDEX_HASH_BUCKET_DEFAULT) - - 1; - } - - public static TaskWriters of(MixedTable table) { - return new TaskWriters(table); - } - - public TaskWriters withTransactionId(Long transactionId) { - this.transactionId = transactionId; - return this; - } - - public TaskWriters withPartitionId(int partitionId) { - this.partitionId = partitionId; - return this; - } - - public TaskWriters withTaskId(long taskId) { - this.taskId = taskId; - return this; - } - - public TaskWriters withDataSourceSchema(StructType dsSchema) { - this.dsSchema = dsSchema; - return this; - } - - public TaskWriters withHiveSubdirectory(String hiveSubdirectory) { - this.hiveSubdirectory = hiveSubdirectory; - return this; - } - - public TaskWriters withOrderedWriter(boolean orderedWriter) { - this.orderedWriter = orderedWriter; - return this; - } - - public TaskWriter newBaseWriter(boolean isOverwrite) { - preconditions(); - - String baseLocation; - EncryptionManager encryptionManager; - Schema schema; - PrimaryKeySpec primaryKeySpec = null; - Table icebergTable; - - if (table.isKeyedTable()) { - KeyedTable keyedTable = table.asKeyedTable(); - baseLocation = keyedTable.baseLocation(); - encryptionManager = keyedTable.baseTable().encryption(); - schema = keyedTable.baseTable().schema(); - primaryKeySpec = keyedTable.primaryKeySpec(); - icebergTable = keyedTable.baseTable(); - } else { - UnkeyedTable table = this.table.asUnkeyedTable(); - baseLocation = table.location(); - encryptionManager = table.encryption(); - schema = table.schema(); - icebergTable = table; - } - - FileAppenderFactory appenderFactory = - InternalRowFileAppenderFactory.builderFor(icebergTable, schema, dsSchema) - .writeHive(isHiveTable) - .build(); - boolean hiveConsistentWrite = - PropertyUtil.propertyAsBoolean( - table.properties(), - HiveTableProperties.HIVE_CONSISTENT_WRITE_ENABLED, - HiveTableProperties.HIVE_CONSISTENT_WRITE_ENABLED_DEFAULT); - OutputFileFactory outputFileFactory; - if (isHiveTable && isOverwrite) { - outputFileFactory = - new AdaptHiveOutputFileFactory( - ((SupportHive) table).hiveLocation(), - table.spec(), - fileFormat, - table.io(), - encryptionManager, - partitionId, - taskId, - transactionId, - hiveSubdirectory, - hiveConsistentWrite); - } else { - outputFileFactory = - new CommonOutputFileFactory( - baseLocation, - table.spec(), - fileFormat, - table.io(), - encryptionManager, - partitionId, - taskId, - transactionId); - } - - return new SparkBaseTaskWriter( - fileFormat, - appenderFactory, - outputFileFactory, - table.io(), - fileSize, - mask, - schema, - table.spec(), - primaryKeySpec, - orderedWriter); - } - - public ChangeTaskWriter newChangeWriter() { - preconditions(); - String changeLocation; - EncryptionManager encryptionManager; - Schema schema; - PrimaryKeySpec primaryKeySpec = null; - Table icebergTable; - - if (table.isKeyedTable()) { - KeyedTable keyedTable = table.asKeyedTable(); - changeLocation = keyedTable.changeLocation(); - encryptionManager = keyedTable.changeTable().encryption(); - schema = SchemaUtil.changeWriteSchema(keyedTable.changeTable().schema()); - primaryKeySpec = keyedTable.primaryKeySpec(); - icebergTable = keyedTable.baseTable(); - } else { - throw new UnsupportedOperationException("Unkeyed table does not support change writer"); - } - FileAppenderFactory appenderFactory = - InternalRowFileAppenderFactory.builderFor( - icebergTable, schema, SparkSchemaUtil.convert(schema)) - .writeHive(isHiveTable) - .build(); - - OutputFileFactory outputFileFactory; - outputFileFactory = - new CommonOutputFileFactory( - changeLocation, - table.spec(), - fileFormat, - table.io(), - encryptionManager, - partitionId, - taskId, - transactionId); - - return new SparkChangeTaskWriter( - fileFormat, - appenderFactory, - outputFileFactory, - table.io(), - fileSize, - mask, - schema, - table.spec(), - primaryKeySpec, - orderedWriter); - } - - public TaskWriter newUnkeyedUpsertWriter() { - preconditions(); - Schema schema = table.schema(); - InternalRowFileAppenderFactory build = - new InternalRowFileAppenderFactory.Builder(table.asUnkeyedTable(), schema, dsSchema) - .build(); - long fileSizeBytes = - PropertyUtil.propertyAsLong( - table.properties(), - TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, - TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); - long mask = - PropertyUtil.propertyAsLong( - table.properties(), - TableProperties.BASE_FILE_INDEX_HASH_BUCKET, - TableProperties.BASE_FILE_INDEX_HASH_BUCKET_DEFAULT) - - 1; - CommonOutputFileFactory commonOutputFileFactory = - new CommonOutputFileFactory( - table.location(), - table.spec(), - fileFormat, - table.io(), - table.asUnkeyedTable().encryption(), - partitionId, - taskId, - transactionId); - SparkBaseTaskWriter sparkBaseTaskWriter = - new SparkBaseTaskWriter( - fileFormat, - build, - commonOutputFileFactory, - table.io(), - fileSizeBytes, - mask, - schema, - table.spec(), - null, - orderedWriter); - return new UnkeyedUpsertSparkWriter<>( - table, build, commonOutputFileFactory, fileFormat, schema, sparkBaseTaskWriter); - } - - private void preconditions() { - if (table.isKeyedTable()) { - Preconditions.checkState(transactionId != null, "Transaction id is not set for KeyedTable"); - } else { - Preconditions.checkState( - transactionId == null, "Transaction id should be null for UnkeyedTable"); - } - Preconditions.checkState(partitionId >= 0, "Partition id is not set"); - Preconditions.checkState(taskId >= 0, "Task id is not set"); - Preconditions.checkState(dsSchema != null, "Data source schema is not set"); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/UnkeyedUpsertSparkWriter.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/UnkeyedUpsertSparkWriter.java deleted file mode 100644 index 969ee9bb8b..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/io/UnkeyedUpsertSparkWriter.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.io; - -import static org.apache.spark.sql.types.DataTypes.IntegerType; -import static org.apache.spark.sql.types.DataTypes.StringType; - -import org.apache.amoro.data.ChangeAction; -import org.apache.amoro.io.writer.OutputFileFactory; -import org.apache.amoro.io.writer.SortedPosDeleteWriter; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.SparkInternalRowCastWrapper; -import org.apache.amoro.spark.SparkInternalRowWrapper; -import org.apache.amoro.table.MixedTable; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.spark.sql.catalyst.InternalRow; - -import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class UnkeyedUpsertSparkWriter implements TaskWriter { - - private final List completedDeleteFiles = Lists.newArrayList(); - private final List completedDataFiles = Lists.newArrayList(); - - private final FileAppenderFactory appenderFactory; - private final OutputFileFactory fileFactory; - private final FileFormat format; - private final Schema schema; - private final MixedTable table; - private final SparkBaseTaskWriter writer; - private final Map> writerMap = new HashMap<>(); - private boolean closed = false; - - public UnkeyedUpsertSparkWriter( - MixedTable table, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileFormat format, - Schema schema, - SparkBaseTaskWriter writer) { - this.table = table; - this.appenderFactory = appenderFactory; - this.fileFactory = fileFactory; - this.format = format; - this.schema = schema; - this.writer = writer; - } - - @Override - public void write(T row) throws IOException { - if (closed) { - throw new IllegalStateException( - "Pos-delete writer for table " + table.id().toString() + " already closed"); - } - - SparkInternalRowCastWrapper internalRow = (SparkInternalRowCastWrapper) row; - StructLike structLike = - new SparkInternalRowWrapper(SparkSchemaUtil.convert(schema)).wrap(internalRow.getRow()); - PartitionKey partitionKey = new PartitionKey(table.spec(), schema); - partitionKey.partition(structLike); - if (writerMap.get(partitionKey) == null) { - SortedPosDeleteWriter writer = - new SortedPosDeleteWriter<>( - appenderFactory, fileFactory, table.io(), format, partitionKey); - writerMap.putIfAbsent(partitionKey, writer); - } - if (internalRow.getChangeAction() == ChangeAction.DELETE) { - SortedPosDeleteWriter deleteWriter = writerMap.get(partitionKey); - int numFields = internalRow.getRow().numFields(); - Object file = internalRow.getRow().get(numFields - 2, StringType); - Object pos = internalRow.getRow().get(numFields - 1, IntegerType); - deleteWriter.delete(file.toString(), Long.parseLong(pos.toString()), null); - } else { - this.writer.write(internalRow.getRow()); - } - } - - @Override - public void abort() throws IOException { - close(); - } - - @Override - public WriteResult complete() throws IOException { - for (Map.Entry> entry : writerMap.entrySet()) { - completedDeleteFiles.addAll(entry.getValue().complete()); - } - close(); - completedDataFiles.addAll(Arrays.asList(writer.complete().dataFiles())); - return WriteResult.builder() - .addDeleteFiles(completedDeleteFiles) - .addDataFiles(completedDataFiles) - .build(); - } - - @Override - public void close() throws IOException { - this.closed = true; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/KeyedSparkBatchScan.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/KeyedSparkBatchScan.java deleted file mode 100644 index 7f576e741b..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/KeyedSparkBatchScan.java +++ /dev/null @@ -1,272 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.reader; - -import org.apache.amoro.io.AuthenticatedFileIO; -import org.apache.amoro.scan.CombinedScanTask; -import org.apache.amoro.scan.KeyedTableScan; -import org.apache.amoro.scan.KeyedTableScanTask; -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.util.Stats; -import org.apache.amoro.table.KeyedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.connector.read.Batch; -import org.apache.spark.sql.connector.read.InputPartition; -import org.apache.spark.sql.connector.read.PartitionReader; -import org.apache.spark.sql.connector.read.PartitionReaderFactory; -import org.apache.spark.sql.connector.read.Scan; -import org.apache.spark.sql.connector.read.Statistics; -import org.apache.spark.sql.connector.read.SupportsReportStatistics; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.Serializable; -import java.io.UncheckedIOException; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; - -public class KeyedSparkBatchScan implements Scan, Batch, SupportsReportStatistics { - private static final Logger LOG = LoggerFactory.getLogger(KeyedSparkBatchScan.class); - - private final KeyedTable table; - private final boolean caseSensitive; - private final Schema expectedSchema; - private final List filterExpressions; - private StructType readSchema = null; - private List tasks = null; - - KeyedSparkBatchScan( - KeyedTable table, - boolean caseSensitive, - Schema expectedSchema, - List filters, - CaseInsensitiveStringMap options) { - Preconditions.checkNotNull(table, "table must not be null"); - Preconditions.checkNotNull(expectedSchema, "expectedSchema must not be null"); - Preconditions.checkNotNull(filters, "filters must not be null"); - - this.table = table; - this.caseSensitive = caseSensitive; - this.expectedSchema = expectedSchema; - this.filterExpressions = filters; - } - - @Override - public Batch toBatch() { - return this; - } - - @Override - public StructType readSchema() { - if (readSchema == null) { - this.readSchema = SparkSchemaUtil.convert(expectedSchema); - } - return readSchema; - } - - @Override - public InputPartition[] planInputPartitions() { - List scanTasks = tasks(); - MixedFormatInputPartition[] readTasks = new MixedFormatInputPartition[scanTasks.size()]; - for (int i = 0; i < scanTasks.size(); i++) { - readTasks[i] = - new MixedFormatInputPartition(scanTasks.get(i), table, expectedSchema, caseSensitive); - } - return readTasks; - } - - @Override - public PartitionReaderFactory createReaderFactory() { - return new ReaderFactory(); - } - - @Override - public Statistics estimateStatistics() { - long sizeInBytes = 0L; - long numRows = 0L; - - for (CombinedScanTask combinedScanTask : tasks()) { - for (KeyedTableScanTask fileScanTask : combinedScanTask.tasks()) { - - sizeInBytes += fileScanTask.cost(); - numRows += fileScanTask.recordCount(); - } - } - - return new Stats(sizeInBytes, numRows); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - KeyedSparkBatchScan that = (KeyedSparkBatchScan) o; - return table.id().equals(that.table.id()) - && readSchema().equals(that.readSchema()) - && // compare Spark schemas to ignore field ids - filterExpressions.toString().equals(that.filterExpressions.toString()); - } - - @Override - public int hashCode() { - return Objects.hash(table.id(), readSchema()); - } - - private List tasks() { - if (tasks == null) { - KeyedTableScan scan = table.newScan(); - - for (Expression filter : filterExpressions) { - scan = scan.filter(filter); - } - long startTime = System.currentTimeMillis(); - LOG.info("mor statistics plan task start"); - try (CloseableIterable tasksIterable = scan.planTasks()) { - this.tasks = Lists.newArrayList(tasksIterable); - LOG.info( - "mor statistics plan task end, cost time {}, tasks num {}", - System.currentTimeMillis() - startTime, - tasks.size()); - } catch (IOException e) { - throw new UncheckedIOException("Failed to close table scan: %s", e); - } - } - return tasks; - } - - @Override - public String description() { - String filters = - filterExpressions.stream().map(Spark3Util::describe).collect(Collectors.joining(", ")); - return String.format("%s [filters=%s]", table, filters); - } - - @Override - public String toString() { - return String.format( - "IcebergScan(table=%s, type=%s, filters=%s, caseSensitive=%s)", - table, expectedSchema.asStruct(), filterExpressions, caseSensitive); - } - - private static class ReaderFactory implements PartitionReaderFactory { - @Override - public PartitionReader createReader(InputPartition partition) { - if (partition instanceof MixedFormatInputPartition) { - return new RowReader((MixedFormatInputPartition) partition); - } else { - throw new UnsupportedOperationException("Incorrect input partition type: " + partition); - } - } - } - - private static class RowReader implements PartitionReader { - - SparkKeyedDataReader reader; - Iterator scanTasks; - KeyedTableScanTask currentScanTask; - CloseableIterator currentIterator = CloseableIterator.empty(); - InternalRow current; - - RowReader(MixedFormatInputPartition task) { - reader = - new SparkKeyedDataReader( - task.io, - task.tableSchema, - task.expectedSchema, - task.keySpec, - task.nameMapping, - task.caseSensitive); - scanTasks = task.combinedScanTask.tasks().iterator(); - } - - @Override - public boolean next() throws IOException { - while (true) { - if (currentIterator.hasNext()) { - this.current = currentIterator.next(); - return true; - } else if (scanTasks.hasNext()) { - this.currentIterator.close(); - this.currentScanTask = scanTasks.next(); - this.currentIterator = reader.readData(this.currentScanTask); - } else { - this.currentIterator.close(); - return false; - } - } - } - - @Override - public InternalRow get() { - return this.current; - } - - @Override - public void close() throws IOException { - this.currentIterator.close(); - while (scanTasks.hasNext()) { - scanTasks.next(); - } - } - } - - private static class MixedFormatInputPartition implements InputPartition, Serializable { - final CombinedScanTask combinedScanTask; - final AuthenticatedFileIO io; - final boolean caseSensitive; - final Schema expectedSchema; - final Schema tableSchema; - final PrimaryKeySpec keySpec; - final String nameMapping; - - MixedFormatInputPartition( - CombinedScanTask combinedScanTask, - KeyedTable table, - Schema expectedSchema, - boolean caseSensitive) { - this.combinedScanTask = combinedScanTask; - this.expectedSchema = expectedSchema; - this.tableSchema = table.schema(); - this.caseSensitive = caseSensitive; - this.io = table.io(); - this.keySpec = table.primaryKeySpec(); - this.nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkKeyedDataReader.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkKeyedDataReader.java deleted file mode 100644 index 4ecc04d6e4..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkKeyedDataReader.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.reader; - -import org.apache.amoro.hive.io.reader.AbstractAdaptHiveKeyedDataReader; -import org.apache.amoro.io.AuthenticatedFileIO; -import org.apache.amoro.spark.SparkInternalRowWrapper; -import org.apache.amoro.spark.util.MixedFormatSparkUtils; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.orc.OrcRowReader; -import org.apache.iceberg.parquet.ParquetValueReader; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.data.SparkOrcReader; -import org.apache.orc.TypeDescription; -import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructType; - -import java.util.Map; -import java.util.function.Function; - -public class SparkKeyedDataReader extends AbstractAdaptHiveKeyedDataReader { - - public SparkKeyedDataReader( - AuthenticatedFileIO fileIO, - Schema tableSchema, - Schema projectedSchema, - PrimaryKeySpec primaryKeySpec, - String nameMapping, - boolean caseSensitive) { - super( - fileIO, - tableSchema, - projectedSchema, - primaryKeySpec, - nameMapping, - caseSensitive, - MixedFormatSparkUtils::convertConstant, - true); - } - - @Override - protected Function> getParquetReaderFunction( - Schema projectSchema, Map idToConstant) { - return fileSchema -> SparkParquetReaders.buildReader(projectSchema, fileSchema, idToConstant); - } - - @Override - protected Function> getOrcReaderFunction( - Schema projectSchema, Map idToConstant) { - return fileSchema -> new SparkOrcReader(projectSchema, fileSchema, idToConstant); - } - - @Override - protected Function> toStructLikeFunction() { - return schema -> { - final StructType structType = SparkSchemaUtil.convert(schema); - SparkInternalRowWrapper wrapper = new SparkInternalRowWrapper(structType); - return row -> wrapper.wrap(row); - }; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkParquetReaders.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkParquetReaders.java deleted file mode 100644 index 46f3e6f4fa..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkParquetReaders.java +++ /dev/null @@ -1,765 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.reader; - -import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableList; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.parquet.ParquetSchemaUtil; -import org.apache.iceberg.parquet.ParquetValueReader; -import org.apache.iceberg.parquet.ParquetValueReaders; -import org.apache.iceberg.parquet.TypeWithSchemaVisitor; -import org.apache.iceberg.types.Types; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.DecimalMetadata; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; -import org.apache.spark.sql.catalyst.util.ArrayData; -import org.apache.spark.sql.catalyst.util.GenericArrayData; -import org.apache.spark.sql.catalyst.util.MapData; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.unsafe.types.CalendarInterval; -import org.apache.spark.unsafe.types.UTF8String; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -/** - * Copy from org.apache.iceberg.spark.data.SparkParquetReaders to adapt string type writen by impala - */ -public class SparkParquetReaders { - private SparkParquetReaders() {} - - @SuppressWarnings("unchecked") - public static ParquetValueReader buildReader( - Schema expectedSchema, MessageType fileSchema, Map idToConstant) { - if (ParquetSchemaUtil.hasIds(fileSchema)) { - return (ParquetValueReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema, idToConstant)); - } else { - return (ParquetValueReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), - fileSchema, - new FallbackReadBuilder(fileSchema, idToConstant)); - } - } - - private static class FallbackReadBuilder extends ReadBuilder { - FallbackReadBuilder(MessageType type, Map idToConstant) { - super(type, idToConstant); - } - - @Override - public ParquetValueReader message( - Types.StructType expected, MessageType message, List> fieldReaders) { - // the top level matches by ID, but the remaining IDs are missing - return super.struct(expected, message, fieldReaders); - } - - @Override - public ParquetValueReader struct( - Types.StructType ignored, GroupType struct, List> fieldReaders) { - // the expected struct is ignored because nested fields are never found when the - List> newFields = - Lists.newArrayListWithExpectedSize(fieldReaders.size()); - List types = Lists.newArrayListWithExpectedSize(fieldReaders.size()); - List fields = struct.getFields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i); - int fieldD = type().getMaxDefinitionLevel(path(fieldType.getName())) - 1; - newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); - types.add(fieldType); - } - - return new InternalRowReader(types, newFields); - } - } - - private static class ReadBuilder extends TypeWithSchemaVisitor> { - private final MessageType type; - private final Map idToConstant; - - ReadBuilder(MessageType type, Map idToConstant) { - this.type = type; - this.idToConstant = idToConstant; - } - - @Override - public ParquetValueReader message( - Types.StructType expected, MessageType message, List> fieldReaders) { - return struct(expected, message.asGroupType(), fieldReaders); - } - - @Override - public ParquetValueReader struct( - Types.StructType expected, GroupType struct, List> fieldReaders) { - // match the expected struct's order - Map> readersById = Maps.newHashMap(); - Map typesById = Maps.newHashMap(); - List fields = struct.getFields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i); - int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1; - if (fieldType.getId() != null) { - int id = fieldType.getId().intValue(); - readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); - typesById.put(id, fieldType); - } - } - - List expectedFields = - expected != null ? expected.fields() : ImmutableList.of(); - List> reorderedFields = - Lists.newArrayListWithExpectedSize(expectedFields.size()); - List types = Lists.newArrayListWithExpectedSize(expectedFields.size()); - for (Types.NestedField field : expectedFields) { - int id = field.fieldId(); - if (idToConstant.containsKey(id)) { - // containsKey is used because the constant may be null - reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id))); - types.add(null); - } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { - reorderedFields.add(ParquetValueReaders.position()); - types.add(null); - } else if (id == MetadataColumns.IS_DELETED.fieldId()) { - reorderedFields.add(ParquetValueReaders.constant(false)); - types.add(null); - } else { - ParquetValueReader reader = readersById.get(id); - if (reader != null) { - reorderedFields.add(reader); - types.add(typesById.get(id)); - } else { - reorderedFields.add(ParquetValueReaders.nulls()); - types.add(null); - } - } - } - - return new InternalRowReader(types, reorderedFields); - } - - @Override - public ParquetValueReader list( - Types.ListType expectedList, GroupType array, ParquetValueReader elementReader) { - GroupType repeated = array.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1; - int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1; - - Type elementType = repeated.getType(0); - int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1; - - return new ArrayReader<>( - repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader)); - } - - @Override - public ParquetValueReader map( - Types.MapType expectedMap, - GroupType map, - ParquetValueReader keyReader, - ParquetValueReader valueReader) { - GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1; - int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1; - - Type keyType = repeatedKeyValue.getType(0); - int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1; - Type valueType = repeatedKeyValue.getType(1); - int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1; - - return new MapReader<>( - repeatedD, - repeatedR, - ParquetValueReaders.option(keyType, keyD, keyReader), - ParquetValueReaders.option(valueType, valueD, valueReader)); - } - - @Override - public ParquetValueReader primitive( - org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { - ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return new StringReader(desc); - case INT_8: - case INT_16: - case INT_32: - if (expected != null && expected.typeId() == Types.LongType.get().typeId()) { - return new ParquetValueReaders.IntAsLongReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader(desc); - } - case DATE: - case INT_64: - case TIMESTAMP_MICROS: - return new ParquetValueReaders.UnboxedReader<>(desc); - case TIMESTAMP_MILLIS: - return new TimestampMillisReader(desc); - case DECIMAL: - DecimalMetadata decimal = primitive.getDecimalMetadata(); - switch (primitive.getPrimitiveTypeName()) { - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return new BinaryDecimalReader(desc, decimal.getScale()); - case INT64: - return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT32: - return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return new ParquetValueReaders.ByteArrayReader(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } - } - - switch (primitive.getPrimitiveTypeName()) { - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - // Change for mixed-format table ⬇ - if (expected == Types.StringType.get()) { - return new StringReader(desc); - } else { - return new ParquetValueReaders.ByteArrayReader(desc); - } - // Change for mixed-format table ⬆ - case INT32: - if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) { - return new ParquetValueReaders.IntAsLongReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case FLOAT: - if (expected != null - && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) { - return new ParquetValueReaders.FloatAsDoubleReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case BOOLEAN: - case INT64: - case DOUBLE: - return new ParquetValueReaders.UnboxedReader<>(desc); - case INT96: - // Impala & Spark used to write timestamps as INT96 without a logical type. For backwards - // compatibility we try to read INT96 as timestamps. - return new TimestampInt96Reader(desc); - default: - throw new UnsupportedOperationException("Unsupported type: " + primitive); - } - } - - protected MessageType type() { - return type; - } - } - - private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader { - private final int scale; - - BinaryDecimalReader(ColumnDescriptor desc, int scale) { - super(desc); - this.scale = scale; - } - - @Override - public Decimal read(Decimal ignored) { - Binary binary = column.nextBinary(); - return Decimal.fromDecimal(new BigDecimal(new BigInteger(binary.getBytes()), scale)); - } - } - - private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public Decimal read(Decimal ignored) { - return Decimal.apply(column.nextInteger(), precision, scale); - } - } - - private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - LongDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public Decimal read(Decimal ignored) { - return Decimal.apply(column.nextLong(), precision, scale); - } - } - - private static class TimestampMillisReader extends ParquetValueReaders.UnboxedReader { - TimestampMillisReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public Long read(Long ignored) { - return readLong(); - } - - @Override - public long readLong() { - return 1000 * column.nextLong(); - } - } - - private static class TimestampInt96Reader extends ParquetValueReaders.UnboxedReader { - private static final long UNIX_EPOCH_JULIAN = 2_440_588L; - - TimestampInt96Reader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public Long read(Long ignored) { - return readLong(); - } - - @Override - public long readLong() { - final ByteBuffer byteBuffer = - column.nextBinary().toByteBuffer().order(ByteOrder.LITTLE_ENDIAN); - final long timeOfDayNanos = byteBuffer.getLong(); - final int julianDay = byteBuffer.getInt(); - - return TimeUnit.DAYS.toMicros(julianDay - UNIX_EPOCH_JULIAN) - + TimeUnit.NANOSECONDS.toMicros(timeOfDayNanos); - } - } - - private static class StringReader extends ParquetValueReaders.PrimitiveReader { - StringReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public UTF8String read(UTF8String ignored) { - Binary binary = column.nextBinary(); - ByteBuffer buffer = binary.toByteBuffer(); - if (buffer.hasArray()) { - return UTF8String.fromBytes( - buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); - } else { - return UTF8String.fromBytes(binary.getBytes()); - } - } - } - - private static class ArrayReader - extends ParquetValueReaders.RepeatedReader { - private int readPos = 0; - private int writePos = 0; - - ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader reader) { - super(definitionLevel, repetitionLevel, reader); - } - - @Override - @SuppressWarnings("unchecked") - protected ReusableArrayData newListData(ArrayData reuse) { - this.readPos = 0; - this.writePos = 0; - - if (reuse instanceof ReusableArrayData) { - return (ReusableArrayData) reuse; - } else { - return new ReusableArrayData(); - } - } - - @Override - @SuppressWarnings("unchecked") - protected E getElement(ReusableArrayData list) { - E value = null; - if (readPos < list.capacity()) { - value = (E) list.values[readPos]; - } - - readPos += 1; - - return value; - } - - @Override - protected void addElement(ReusableArrayData reused, E element) { - if (writePos >= reused.capacity()) { - reused.grow(); - } - - reused.values[writePos] = element; - - writePos += 1; - } - - @Override - protected ArrayData buildList(ReusableArrayData list) { - list.setNumElements(writePos); - return list; - } - } - - private static class MapReader - extends ParquetValueReaders.RepeatedKeyValueReader { - private int readPos = 0; - private int writePos = 0; - - private final ParquetValueReaders.ReusableEntry entry = - new ParquetValueReaders.ReusableEntry<>(); - private final ParquetValueReaders.ReusableEntry nullEntry = - new ParquetValueReaders.ReusableEntry<>(); - - MapReader( - int definitionLevel, - int repetitionLevel, - ParquetValueReader keyReader, - ParquetValueReader valueReader) { - super(definitionLevel, repetitionLevel, keyReader, valueReader); - } - - @Override - @SuppressWarnings("unchecked") - protected ReusableMapData newMapData(MapData reuse) { - this.readPos = 0; - this.writePos = 0; - - if (reuse instanceof ReusableMapData) { - return (ReusableMapData) reuse; - } else { - return new ReusableMapData(); - } - } - - @Override - @SuppressWarnings("unchecked") - protected Map.Entry getPair(ReusableMapData map) { - Map.Entry kv = nullEntry; - if (readPos < map.capacity()) { - entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]); - kv = entry; - } - - readPos += 1; - - return kv; - } - - @Override - protected void addPair(ReusableMapData map, K key, V value) { - if (writePos >= map.capacity()) { - map.grow(); - } - - map.keys.values[writePos] = key; - map.values.values[writePos] = value; - - writePos += 1; - } - - @Override - protected MapData buildMap(ReusableMapData map) { - map.setNumElements(writePos); - return map; - } - } - - private static class InternalRowReader - extends ParquetValueReaders.StructReader { - private final int numFields; - - InternalRowReader(List types, List> readers) { - super(types, readers); - this.numFields = readers.size(); - } - - @Override - protected GenericInternalRow newStructData(InternalRow reuse) { - if (reuse instanceof GenericInternalRow) { - return (GenericInternalRow) reuse; - } else { - return new GenericInternalRow(numFields); - } - } - - @Override - protected Object getField(GenericInternalRow intermediate, int pos) { - return intermediate.genericGet(pos); - } - - @Override - protected InternalRow buildStruct(GenericInternalRow struct) { - return struct; - } - - @Override - protected void set(GenericInternalRow row, int pos, Object value) { - row.update(pos, value); - } - - @Override - protected void setNull(GenericInternalRow row, int pos) { - row.setNullAt(pos); - } - - @Override - protected void setBoolean(GenericInternalRow row, int pos, boolean value) { - row.setBoolean(pos, value); - } - - @Override - protected void setInteger(GenericInternalRow row, int pos, int value) { - row.setInt(pos, value); - } - - @Override - protected void setLong(GenericInternalRow row, int pos, long value) { - row.setLong(pos, value); - } - - @Override - protected void setFloat(GenericInternalRow row, int pos, float value) { - row.setFloat(pos, value); - } - - @Override - protected void setDouble(GenericInternalRow row, int pos, double value) { - row.setDouble(pos, value); - } - } - - private static class ReusableMapData extends MapData { - private final ReusableArrayData keys; - private final ReusableArrayData values; - private int numElements; - - private ReusableMapData() { - this.keys = new ReusableArrayData(); - this.values = new ReusableArrayData(); - } - - private void grow() { - keys.grow(); - values.grow(); - } - - private int capacity() { - return keys.capacity(); - } - - public void setNumElements(int numElements) { - this.numElements = numElements; - keys.setNumElements(numElements); - values.setNumElements(numElements); - } - - @Override - public int numElements() { - return numElements; - } - - @Override - public MapData copy() { - return new ArrayBasedMapData(keyArray().copy(), valueArray().copy()); - } - - @Override - public ReusableArrayData keyArray() { - return keys; - } - - @Override - public ReusableArrayData valueArray() { - return values; - } - } - - private static class ReusableArrayData extends ArrayData { - private static final Object[] EMPTY = new Object[0]; - - private Object[] values = EMPTY; - private int numElements = 0; - - private void grow() { - if (values.length == 0) { - this.values = new Object[20]; - } else { - Object[] old = values; - this.values = new Object[old.length << 2]; - // copy the old array in case it has values that can be reused - System.arraycopy(old, 0, values, 0, old.length); - } - } - - private int capacity() { - return values.length; - } - - public void setNumElements(int numElements) { - this.numElements = numElements; - } - - @Override - public Object get(int ordinal, DataType dataType) { - return values[ordinal]; - } - - @Override - public int numElements() { - return numElements; - } - - @Override - public ArrayData copy() { - return new GenericArrayData(array()); - } - - @Override - public Object[] array() { - return Arrays.copyOfRange(values, 0, numElements); - } - - @Override - public void setNullAt(int i) { - values[i] = null; - } - - @Override - public void update(int ordinal, Object value) { - values[ordinal] = value; - } - - @Override - public boolean isNullAt(int ordinal) { - return null == values[ordinal]; - } - - @Override - public boolean getBoolean(int ordinal) { - return (boolean) values[ordinal]; - } - - @Override - public byte getByte(int ordinal) { - return (byte) values[ordinal]; - } - - @Override - public short getShort(int ordinal) { - return (short) values[ordinal]; - } - - @Override - public int getInt(int ordinal) { - return (int) values[ordinal]; - } - - @Override - public long getLong(int ordinal) { - return (long) values[ordinal]; - } - - @Override - public float getFloat(int ordinal) { - return (float) values[ordinal]; - } - - @Override - public double getDouble(int ordinal) { - return (double) values[ordinal]; - } - - @Override - public Decimal getDecimal(int ordinal, int precision, int scale) { - return (Decimal) values[ordinal]; - } - - @Override - public UTF8String getUTF8String(int ordinal) { - return (UTF8String) values[ordinal]; - } - - @Override - public byte[] getBinary(int ordinal) { - return (byte[]) values[ordinal]; - } - - @Override - public CalendarInterval getInterval(int ordinal) { - return (CalendarInterval) values[ordinal]; - } - - @Override - public InternalRow getStruct(int ordinal, int numFields) { - return (InternalRow) values[ordinal]; - } - - @Override - public ArrayData getArray(int ordinal) { - return (ArrayData) values[ordinal]; - } - - @Override - public MapData getMap(int ordinal) { - return (MapData) values[ordinal]; - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkScanBuilder.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkScanBuilder.java deleted file mode 100644 index 9ab15a3918..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkScanBuilder.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.reader; - -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.table.SupportsExtendIdentColumns; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.UnkeyedTable; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.expressions.Binder; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.spark.SparkFilters; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.connector.read.Scan; -import org.apache.spark.sql.connector.read.ScanBuilder; -import org.apache.spark.sql.connector.read.SupportsPushDownFilters; -import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns; -import org.apache.spark.sql.sources.Filter; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; - -import java.util.List; -import java.util.stream.Collectors; - -public class SparkScanBuilder - implements ScanBuilder, - SupportsExtendIdentColumns, - SupportsPushDownFilters, - SupportsPushDownRequiredColumns { - private static final Filter[] NO_FILTERS = new Filter[0]; - - private final MixedTable table; - private final CaseInsensitiveStringMap options; - - private final List metaColumns = Lists.newArrayList(); - - private Schema schema = null; - private StructType requestedProjection; - private final boolean caseSensitive; - private List filterExpressions = Lists.newArrayList(); - private Filter[] pushedFilters = NO_FILTERS; - - public SparkScanBuilder(SparkSession spark, MixedTable table, CaseInsensitiveStringMap options) { - this.table = table; - this.options = options; - this.caseSensitive = Boolean.parseBoolean(spark.conf().get("spark.sql.caseSensitive")); - } - - public SparkScanBuilder( - SparkSession spark, MixedTable table, CaseInsensitiveStringMap options, Schema schema) { - this.table = table; - this.options = options; - this.schema = schema; - this.caseSensitive = Boolean.parseBoolean(spark.conf().get("spark.sql.caseSensitive")); - } - - private Schema lazySchemaWithRowIdent() { - if (schema == null) { - if (requestedProjection != null) { - // the projection should include all columns that will be returned, - // including those only used in filters - this.schema = - SparkSchemaUtil.prune( - table.schema(), requestedProjection, filterExpression(), caseSensitive); - } else { - this.schema = table.schema(); - } - } - - UnkeyedTable icebergTable; - if (table.isUnkeyedTable()) { - icebergTable = table.asUnkeyedTable(); - } else { - icebergTable = table.asKeyedTable().baseTable(); - } - // metadata columns - List fields = - metaColumns.stream() - .distinct() - .map(column -> MetadataColumns.metadataColumn(icebergTable, column)) - .collect(Collectors.toList()); - if (fields.size() == 1) { - return schema; - } - Schema meta = new Schema(fields); - - return TypeUtil.join(schema, meta); - } - - private Expression filterExpression() { - if (!filterExpressions.isEmpty()) { - return filterExpressions.stream().reduce(Expressions.alwaysTrue(), Expressions::and); - } - return Expressions.alwaysTrue(); - } - - @Override - public Filter[] pushFilters(Filter[] filters) { - List expressions = Lists.newArrayListWithExpectedSize(filters.length); - List pushed = Lists.newArrayListWithExpectedSize(filters.length); - - for (Filter filter : filters) { - Expression expr = SparkFilters.convert(filter); - if (expr != null) { - try { - Binder.bind(table.schema().asStruct(), expr, caseSensitive); - expressions.add(expr); - pushed.add(filter); - } catch (ValidationException e) { - // binding to the table schema failed, so this expression cannot be pushed down - } - } - } - - this.filterExpressions = expressions; - this.pushedFilters = pushed.toArray(new Filter[0]); - - // Spark doesn't support residuals per task, so return all filters - // to get Spark to handle record-level filtering - return filters; - } - - @Override - public Filter[] pushedFilters() { - return pushedFilters; - } - - @Override - public void pruneColumns(StructType requestedSchema) { - this.requestedProjection = requestedSchema; - } - - @Override - public Scan build() { - if (table.isKeyedTable()) { - return new KeyedSparkBatchScan( - table.asKeyedTable(), - caseSensitive, - lazySchemaWithRowIdent(), - filterExpressions, - options); - } else if (table.isUnkeyedTable()) { - return new UnkeyedSparkBatchScan( - table.asUnkeyedTable(), - caseSensitive, - lazySchemaWithRowIdent(), - filterExpressions, - options); - } else { - throw new IllegalStateException( - "Unable to build scan for table: " + table.id().toString() + ", unknown table " + "type"); - } - } - - @Override - public SupportsExtendIdentColumns withIdentifierColumns() { - if (table.isUnkeyedTable()) { - this.metaColumns.addAll(UnkeyedSparkBatchScan.ROW_ID_COLUMNS); - } else if (table.isKeyedTable()) { - this.metaColumns.addAll(table.asKeyedTable().primaryKeySpec().fieldNames()); - } - return this; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkUnkeyedDataReader.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkUnkeyedDataReader.java deleted file mode 100644 index f366306396..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/SparkUnkeyedDataReader.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.reader; - -import org.apache.amoro.hive.io.reader.AbstractAdaptHiveUnkeyedDataReader; -import org.apache.amoro.io.AuthenticatedFileIO; -import org.apache.amoro.spark.SparkInternalRowWrapper; -import org.apache.amoro.spark.util.MixedFormatSparkUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.orc.OrcRowReader; -import org.apache.iceberg.parquet.ParquetValueReader; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.data.SparkOrcReader; -import org.apache.orc.TypeDescription; -import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructType; - -import java.util.Map; -import java.util.function.Function; - -public class SparkUnkeyedDataReader extends AbstractAdaptHiveUnkeyedDataReader { - - public SparkUnkeyedDataReader( - AuthenticatedFileIO fileIO, - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive) { - super( - fileIO, - tableSchema, - projectedSchema, - nameMapping, - caseSensitive, - MixedFormatSparkUtils::convertConstant, - true); - } - - @Override - protected Function> getParquetReaderFunction( - Schema projectedSchema, Map idToConstant) { - return fileSchema -> SparkParquetReaders.buildReader(projectedSchema, fileSchema, idToConstant); - } - - @Override - protected Function> getOrcReaderFunction( - Schema projectSchema, Map idToConstant) { - return fileSchema -> new SparkOrcReader(projectSchema, fileSchema, idToConstant); - } - - @Override - protected Function> toStructLikeFunction() { - return schema -> { - final StructType structType = SparkSchemaUtil.convert(schema); - SparkInternalRowWrapper wrapper = new SparkInternalRowWrapper(structType); - return row -> wrapper.wrap(row); - }; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/UnkeyedSparkBatchScan.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/UnkeyedSparkBatchScan.java deleted file mode 100644 index 7bb2931c8e..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/reader/UnkeyedSparkBatchScan.java +++ /dev/null @@ -1,280 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.reader; - -import org.apache.amoro.io.AuthenticatedFileIO; -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.util.Stats; -import org.apache.amoro.table.UnkeyedTable; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SnapshotSummary; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.connector.read.Batch; -import org.apache.spark.sql.connector.read.InputPartition; -import org.apache.spark.sql.connector.read.PartitionReader; -import org.apache.spark.sql.connector.read.PartitionReaderFactory; -import org.apache.spark.sql.connector.read.Scan; -import org.apache.spark.sql.connector.read.Statistics; -import org.apache.spark.sql.connector.read.SupportsReportStatistics; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.Serializable; -import java.io.UncheckedIOException; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; - -public class UnkeyedSparkBatchScan implements Scan, Batch, SupportsReportStatistics { - private static final Logger LOG = LoggerFactory.getLogger(UnkeyedSparkBatchScan.class); - - public static final String FILE_NAME_COL = "_file"; - public static final String ROW_POS_COL = "_pos"; - public static final List ROW_ID_COLUMNS = Lists.newArrayList(FILE_NAME_COL, ROW_POS_COL); - - private final UnkeyedTable table; - private final boolean caseSensitive; - private final Schema expectedSchema; - private final List filterExpressions; - - private StructType readSchema = null; - private List tasks = null; - - UnkeyedSparkBatchScan( - UnkeyedTable table, - boolean caseSensitive, - Schema expectedSchema, - List filters, - CaseInsensitiveStringMap options) { - Preconditions.checkNotNull(table, "table must not be null"); - Preconditions.checkNotNull(expectedSchema, "expectedSchema must not be null"); - Preconditions.checkNotNull(filters, "filters must not be null"); - - this.table = table; - this.caseSensitive = caseSensitive; - this.expectedSchema = expectedSchema; - this.filterExpressions = filters; - } - - @Override - public InputPartition[] planInputPartitions() { - List scanTasks = tasks(); - MixedFormatInputPartition[] readTasks = new MixedFormatInputPartition[scanTasks.size()]; - for (int i = 0; i < scanTasks.size(); i++) { - readTasks[i] = - new MixedFormatInputPartition(scanTasks.get(i), table, expectedSchema, caseSensitive); - } - return readTasks; - } - - @Override - public PartitionReaderFactory createReaderFactory() { - return new ReaderFactory(); - } - - @Override - public Statistics estimateStatistics() { - if (table.currentSnapshot() == null) { - return new Stats(0L, 0L); - } - if (!table.spec().isUnpartitioned() && filterExpressions.isEmpty()) { - LOG.debug("using table metadata to estimate table statistics"); - long totalRecords = - PropertyUtil.propertyAsLong( - table.currentSnapshot().summary(), - SnapshotSummary.TOTAL_RECORDS_PROP, - Long.MAX_VALUE); - return new Stats(SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords); - } - - long numRows = 0L; - - for (CombinedScanTask task : tasks()) { - for (FileScanTask file : task.files()) { - double fractionOfFileScanned = ((double) file.length()) / file.file().fileSizeInBytes(); - numRows += (fractionOfFileScanned * file.file().recordCount()); - } - } - long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), numRows); - return new Stats(sizeInBytes, numRows); - } - - @Override - public StructType readSchema() { - if (readSchema == null) { - this.readSchema = SparkSchemaUtil.convert(expectedSchema); - } - return readSchema; - } - - @Override - public Batch toBatch() { - return this; - } - - private List tasks() { - if (tasks == null) { - TableScan scan = table.newScan(); - - for (Expression filter : filterExpressions) { - scan = scan.filter(filter); - } - long startTime = System.currentTimeMillis(); - LOG.info("mor statistics plan task start"); - try (CloseableIterable tasksIterable = scan.planTasks()) { - this.tasks = Lists.newArrayList(tasksIterable); - LOG.info( - "mor statistics plan task end, cost time {}, tasks num {}", - System.currentTimeMillis() - startTime, - tasks.size()); - } catch (IOException e) { - throw new UncheckedIOException("Failed to close table scan: %s", e); - } - } - return tasks; - } - - private static class ReaderFactory implements PartitionReaderFactory { - @Override - public PartitionReader createReader(InputPartition partition) { - if (partition instanceof MixedFormatInputPartition) { - return new RowReader((MixedFormatInputPartition) partition); - } else { - throw new UnsupportedOperationException("Incorrect input partition type: " + partition); - } - } - } - - private static class RowReader implements PartitionReader { - - SparkUnkeyedDataReader reader; - Iterator scanTasks; - FileScanTask currentScanTask; - CloseableIterator currentIterator = CloseableIterator.empty(); - InternalRow current; - - RowReader(MixedFormatInputPartition task) { - reader = - new SparkUnkeyedDataReader( - task.io, task.tableSchema, task.expectedSchema, task.nameMapping, task.caseSensitive); - scanTasks = task.combinedScanTask.files().iterator(); - } - - @Override - public boolean next() throws IOException { - while (true) { - if (currentIterator.hasNext()) { - this.current = currentIterator.next(); - return true; - } else if (scanTasks.hasNext()) { - this.currentIterator.close(); - this.currentScanTask = scanTasks.next(); - this.currentIterator = reader.readData(this.currentScanTask).iterator(); - } else { - this.currentIterator.close(); - return false; - } - } - } - - @Override - public InternalRow get() { - return this.current; - } - - @Override - public void close() throws IOException { - this.currentIterator.close(); - while (scanTasks.hasNext()) { - scanTasks.next(); - } - } - } - - private static class MixedFormatInputPartition implements InputPartition, Serializable { - final CombinedScanTask combinedScanTask; - final AuthenticatedFileIO io; - final boolean caseSensitive; - final Schema expectedSchema; - final Schema tableSchema; - final String nameMapping; - - MixedFormatInputPartition( - CombinedScanTask combinedScanTask, - UnkeyedTable table, - Schema expectedSchema, - boolean caseSensitive) { - this.combinedScanTask = combinedScanTask; - this.expectedSchema = expectedSchema; - this.tableSchema = table.schema(); - this.caseSensitive = caseSensitive; - this.io = table.io(); - this.nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - UnkeyedSparkBatchScan that = (UnkeyedSparkBatchScan) o; - return table.id().equals(that.table.id()) - && readSchema().equals(that.readSchema()) - && filterExpressions.toString().equals(that.filterExpressions.toString()); - } - - @Override - public int hashCode() { - return Objects.hash(table.id(), readSchema()); - } - - @Override - public String description() { - String filters = - filterExpressions.stream().map(Spark3Util::describe).collect(Collectors.joining(", ")); - return String.format("%s [filters=%s]", table, filters); - } - - @Override - public String toString() { - return String.format( - "IcebergScan(table=%s, type=%s, filters=%s, caseSensitive=%s)", - table, expectedSchema.asStruct(), filterExpressions, caseSensitive); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/sql/connector/expressions/FileIndexBucket.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/sql/connector/expressions/FileIndexBucket.java deleted file mode 100644 index 2db5b35d9b..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/sql/connector/expressions/FileIndexBucket.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.connector.expressions; - -import org.apache.amoro.data.PrimaryKeyData; -import org.apache.amoro.shade.guava32.com.google.common.base.Joiner; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.iceberg.Schema; -import org.apache.spark.sql.connector.expressions.Expression; -import org.apache.spark.sql.connector.expressions.Expressions; -import org.apache.spark.sql.connector.expressions.NamedReference; -import org.apache.spark.sql.connector.expressions.Transform; - -import java.util.Arrays; -import java.util.List; - -public class FileIndexBucket implements Transform { - - private final Schema schema; - private final PrimaryKeyData primaryKeyData; - private final int mask; - private final String[] primaryKeyColumns; - - public FileIndexBucket(Schema schema, PrimaryKeySpec keySpec, int mask) { - this.primaryKeyData = new PrimaryKeyData(keySpec, schema); - this.mask = mask; - this.primaryKeyColumns = - keySpec.fields().stream() - .map(PrimaryKeySpec.PrimaryKeyField::fieldName) - .toArray(String[]::new); - this.schema = schema; - } - - public Schema schema() { - return this.schema; - } - - public PrimaryKeyData primaryKeyData() { - return this.primaryKeyData; - } - - public int mask() { - return this.mask; - } - - @Override - public String name() { - return "FileIndexBucket"; - } - - @Override - public NamedReference[] references() { - return Arrays.stream(this.primaryKeyColumns) - .map(Expressions::column) - .toArray(NamedReference[]::new); - } - - @Override - public Expression[] arguments() { - List arguments = Lists.newArrayList(); - arguments.add(Expressions.literal(mask)); - arguments.addAll(Arrays.asList(references())); - return arguments.toArray(new Expression[0]); - } - - @Override - public String describe() { - String columns = Joiner.on(", ").join(primaryKeyColumns); - return "FileIndexBucket(" + mask + ", " + columns + ")"; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/MixedSparkTable.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/MixedSparkTable.java deleted file mode 100644 index 0c740e426c..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/MixedSparkTable.java +++ /dev/null @@ -1,240 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.table; - -import org.apache.amoro.hive.table.SupportHive; -import org.apache.amoro.mixed.MixedFormatCatalog; -import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableMap; -import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableSet; -import org.apache.amoro.shade.guava32.com.google.common.collect.Sets; -import org.apache.amoro.spark.reader.SparkScanBuilder; -import org.apache.amoro.spark.util.MixedFormatSparkUtils; -import org.apache.amoro.spark.writer.MixedFormatSparkWriteBuilder; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.TableProperties; -import org.apache.iceberg.Schema; -import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException; -import org.apache.spark.sql.catalyst.analysis.PartitionAlreadyExistsException; -import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement; -import org.apache.spark.sql.connector.catalog.SupportsRead; -import org.apache.spark.sql.connector.catalog.SupportsWrite; -import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.connector.catalog.TableCapability; -import org.apache.spark.sql.connector.expressions.Transform; -import org.apache.spark.sql.connector.read.ScanBuilder; -import org.apache.spark.sql.connector.write.LogicalWriteInfo; -import org.apache.spark.sql.connector.write.WriteBuilder; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; - -import java.util.Map; -import java.util.Set; - -public class MixedSparkTable - implements Table, - SupportsRead, - SupportsWrite, - SupportsRowLevelOperator, - SupportsPartitionManagement { - private static final Set RESERVED_PROPERTIES = - Sets.newHashSet("provider", "format", "current-snapshot-id"); - private static final Set CAPABILITIES = - ImmutableSet.of( - TableCapability.BATCH_READ, - TableCapability.BATCH_WRITE, - TableCapability.STREAMING_WRITE, - TableCapability.OVERWRITE_BY_FILTER, - TableCapability.OVERWRITE_DYNAMIC); - - private final MixedTable mixedTable; - private final String sparkCatalogName; - private StructType lazyTableSchema = null; - private SparkSession lazySpark = null; - private final MixedFormatCatalog catalog; - - public static Table ofMixedTable( - MixedTable table, MixedFormatCatalog catalog, String sparkCatalogName) { - if (table.isUnkeyedTable()) { - if (!(table instanceof SupportHive)) { - return new UnkeyedSparkTable(table.asUnkeyedTable(), false, sparkCatalogName); - } - } - return new MixedSparkTable(table, catalog, sparkCatalogName); - } - - public MixedSparkTable( - MixedTable mixedTable, MixedFormatCatalog catalog, String sparkCatalogName) { - this.mixedTable = mixedTable; - this.sparkCatalogName = sparkCatalogName; - this.catalog = catalog; - } - - private SparkSession sparkSession() { - if (lazySpark == null) { - this.lazySpark = SparkSession.active(); - } - - return lazySpark; - } - - public MixedTable table() { - return mixedTable; - } - - @Override - public String name() { - return sparkCatalogName - + "." - + mixedTable.id().getDatabase() - + "." - + mixedTable.id().getTableName(); - } - - @Override - public StructType schema() { - if (lazyTableSchema == null) { - Schema tableSchema = mixedTable.schema(); - this.lazyTableSchema = SparkSchemaUtil.convert(tableSchema); - } - - return lazyTableSchema; - } - - @Override - public Transform[] partitioning() { - return Spark3Util.toTransforms(mixedTable.spec()); - } - - @Override - public Map properties() { - ImmutableMap.Builder propsBuilder = ImmutableMap.builder(); - - if (!mixedTable.properties().containsKey(TableProperties.BASE_FILE_FORMAT)) { - propsBuilder.put(TableProperties.BASE_FILE_FORMAT, TableProperties.BASE_FILE_FORMAT_DEFAULT); - } - - if (!mixedTable.properties().containsKey(TableProperties.DELTA_FILE_FORMAT)) { - propsBuilder.put( - TableProperties.DELTA_FILE_FORMAT, - mixedTable - .properties() - .getOrDefault( - TableProperties.CHANGE_FILE_FORMAT, TableProperties.CHANGE_FILE_FORMAT_DEFAULT)); - } - propsBuilder.put("provider", MixedFormatSparkUtils.mixedTableProvider(table())); - mixedTable.properties().entrySet().stream() - .filter(entry -> !RESERVED_PROPERTIES.contains(entry.getKey())) - .forEach(propsBuilder::put); - - return propsBuilder.build(); - } - - @Override - public Set capabilities() { - return CAPABILITIES; - } - - @Override - public String toString() { - return mixedTable.toString(); - } - - @Override - public boolean equals(Object other) { - if (this == other) { - return true; - } else if (other == null || getClass() != other.getClass()) { - return false; - } - - // use only name in order to correctly invalidate Spark cache - MixedSparkTable that = (MixedSparkTable) other; - return mixedTable.id().equals(that.mixedTable.id()); - } - - @Override - public int hashCode() { - // use only name in order to correctly invalidate Spark cache - return mixedTable.id().hashCode(); - } - - @Override - public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { - return new SparkScanBuilder(sparkSession(), mixedTable, options); - } - - @Override - public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { - return new MixedFormatSparkWriteBuilder(mixedTable, info, catalog); - } - - @Override - public SupportsExtendIdentColumns newUpsertScanBuilder(CaseInsensitiveStringMap options) { - return new SparkScanBuilder(sparkSession(), mixedTable, options); - } - - @Override - public boolean requireAdditionIdentifierColumns() { - return true; - } - - @Override - public boolean appendAsUpsert() { - return mixedTable.isKeyedTable() - && Boolean.parseBoolean( - mixedTable.properties().getOrDefault(TableProperties.UPSERT_ENABLED, "false")); - } - - @Override - public StructType partitionSchema() { - return SparkSchemaUtil.convert(new Schema(table().spec().partitionType().fields())); - } - - @Override - public void createPartition(InternalRow ident, Map properties) - throws PartitionAlreadyExistsException, UnsupportedOperationException { - throw new UnsupportedOperationException("not supported create partition"); - } - - @Override - public boolean dropPartition(InternalRow ident) { - return false; - } - - @Override - public void replacePartitionMetadata(InternalRow ident, Map properties) - throws NoSuchPartitionException, UnsupportedOperationException { - throw new UnsupportedOperationException("not supported replace partition"); - } - - @Override - public Map loadPartitionMetadata(InternalRow ident) - throws UnsupportedOperationException { - return null; - } - - @Override - public InternalRow[] listPartitionIdentifiers(String[] names, InternalRow ident) { - return new InternalRow[0]; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/SparkChangeTable.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/SparkChangeTable.java deleted file mode 100644 index d8696ba3be..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/SparkChangeTable.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.table; - -import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableSet; -import org.apache.amoro.spark.reader.SparkScanBuilder; -import org.apache.amoro.table.BasicUnkeyedTable; -import org.apache.amoro.table.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.source.SparkTable; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.connector.catalog.TableCapability; -import org.apache.spark.sql.connector.read.ScanBuilder; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; - -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; - -public class SparkChangeTable extends SparkTable { - - private final BasicUnkeyedTable basicUnkeyedTable; - - private SparkSession lazySpark = null; - - private static final Set CAPABILITIES = - ImmutableSet.of(TableCapability.BATCH_READ); - - public SparkChangeTable(BasicUnkeyedTable basicUnkeyedTable, boolean refreshEagerly) { - super(basicUnkeyedTable, refreshEagerly); - this.basicUnkeyedTable = basicUnkeyedTable; - } - - private SparkSession sparkSession() { - if (lazySpark == null) { - this.lazySpark = SparkSession.active(); - } - - return lazySpark; - } - - public Set capabilities() { - return CAPABILITIES; - } - - @Override - public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { - return new SparkScanBuilder( - sparkSession(), basicUnkeyedTable, options, buildSchema(basicUnkeyedTable)); - } - - public Schema buildSchema(BasicUnkeyedTable table) { - Schema schema = table.schema(); - List columns = schema.columns().stream().collect(Collectors.toList()); - columns.add(MetadataColumns.TRANSACTION_ID_FILED); - columns.add(MetadataColumns.FILE_OFFSET_FILED); - columns.add(MetadataColumns.CHANGE_ACTION_FIELD); - return new Schema(columns); - } - - @Override - public StructType schema() { - return SparkSchemaUtil.convert(buildSchema(basicUnkeyedTable)); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/SupportsExtendIdentColumns.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/SupportsExtendIdentColumns.java deleted file mode 100644 index 7e374530d7..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/SupportsExtendIdentColumns.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.table; - -import org.apache.spark.sql.connector.read.ScanBuilder; - -public interface SupportsExtendIdentColumns extends ScanBuilder { - - SupportsExtendIdentColumns withIdentifierColumns(); -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/SupportsRowLevelOperator.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/SupportsRowLevelOperator.java deleted file mode 100644 index e7163a2e3e..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/SupportsRowLevelOperator.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.table; - -import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; - -/** - * A mix-in interface of {@link org.apache.spark.sql.connector.catalog.Table}, to indicate that can - * handle update or delete by upsert. - */ -public interface SupportsRowLevelOperator extends Table { - - /** - * Returns support extend columns scan builder - * - * @param options - * @return - */ - SupportsExtendIdentColumns newUpsertScanBuilder(CaseInsensitiveStringMap options); - - boolean requireAdditionIdentifierColumns(); - - /** - * will table handle insert as upsert - * - * @return true if table require insert as upsert - */ - boolean appendAsUpsert(); -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/UnkeyedSparkTable.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/UnkeyedSparkTable.java deleted file mode 100644 index 244b3b5b71..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/table/UnkeyedSparkTable.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.table; - -import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; -import org.apache.amoro.spark.util.MixedFormatSparkUtils; -import org.apache.amoro.table.UnkeyedTable; -import org.apache.iceberg.Schema; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.source.SparkTable; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException; -import org.apache.spark.sql.catalyst.analysis.PartitionAlreadyExistsException; -import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement; -import org.apache.spark.sql.types.StructType; - -import java.util.Map; - -public class UnkeyedSparkTable extends SparkTable implements SupportsPartitionManagement { - private final UnkeyedTable unkeyedTable; - private final String sparkCatalogName; - - public UnkeyedSparkTable( - UnkeyedTable unkeyedTable, boolean refreshEagerly, String sparkCatalogName) { - super(unkeyedTable, refreshEagerly); - this.unkeyedTable = unkeyedTable; - this.sparkCatalogName = sparkCatalogName; - } - - @Override - public UnkeyedTable table() { - return unkeyedTable; - } - - @Override - public String name() { - return sparkCatalogName - + "." - + unkeyedTable.id().getDatabase() - + "." - + unkeyedTable.id().getTableName(); - } - - @Override - public Map properties() { - Map properties = Maps.newHashMap(); - properties.putAll(super.properties()); - properties.put("provider", MixedFormatSparkUtils.mixedTableProvider(table())); - return properties; - } - - @Override - public StructType partitionSchema() { - return SparkSchemaUtil.convert(new Schema(table().spec().partitionType().fields())); - } - - @Override - public void createPartition(InternalRow ident, Map properties) - throws PartitionAlreadyExistsException, UnsupportedOperationException { - throw new UnsupportedOperationException("not supported create partition"); - } - - @Override - public boolean dropPartition(InternalRow ident) { - return false; - } - - @Override - public void replacePartitionMetadata(InternalRow ident, Map properties) - throws NoSuchPartitionException, UnsupportedOperationException { - throw new UnsupportedOperationException("not supported replace partition"); - } - - @Override - public Map loadPartitionMetadata(InternalRow ident) - throws UnsupportedOperationException { - return null; - } - - @Override - public InternalRow[] listPartitionIdentifiers(String[] names, InternalRow ident) { - return new InternalRow[0]; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/DistributionAndOrderingUtil.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/DistributionAndOrderingUtil.java deleted file mode 100644 index 4b6335ab73..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/DistributionAndOrderingUtil.java +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.util; - -import static org.apache.iceberg.spark.Spark3Util.toTransforms; - -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.shade.guava32.com.google.common.collect.ObjectArrays; -import org.apache.amoro.spark.SparkAdapterLoader; -import org.apache.amoro.spark.sql.connector.expressions.FileIndexBucket; -import org.apache.amoro.table.DistributionHashMode; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.amoro.table.TableProperties; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.transforms.SortOrderVisitor; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.spark.sql.connector.expressions.Expression; -import org.apache.spark.sql.connector.expressions.Expressions; -import org.apache.spark.sql.connector.expressions.NamedReference; -import org.apache.spark.sql.connector.expressions.Transform; - -import java.util.Arrays; -import java.util.List; - -public class DistributionAndOrderingUtil { - - private static final ExpressionHelper expressionHelper = - SparkAdapterLoader.getOrLoad().expressions(); - - private static final NamedReference SPEC_ID = Expressions.column(MetadataColumns.SPEC_ID.name()); - private static final NamedReference PARTITION = - Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME); - private static final NamedReference FILE_PATH = - Expressions.column(MetadataColumns.FILE_PATH.name()); - private static final NamedReference ROW_POSITION = - Expressions.column(MetadataColumns.ROW_POSITION.name()); - - private static final Expression SPEC_ID_ORDER = expressionHelper.sort(SPEC_ID, true); - private static final Expression PARTITION_ORDER = expressionHelper.sort(PARTITION, true); - private static final Expression FILE_PATH_ORDER = expressionHelper.sort(FILE_PATH, true); - private static final Expression ROW_POSITION_ORDER = expressionHelper.sort(ROW_POSITION, true); - private static final Expression[] METADATA_ORDERS = - new Expression[] {PARTITION_ORDER, FILE_PATH_ORDER, ROW_POSITION_ORDER}; - - /** - * Build a list of {@link org.apache.spark.sql.connector.expressions.Expression} indicate how to - * shuffle incoming data before writing. The result of this method will convert to a list of - * {@link org.apache.spark.sql.catalyst.expressions.Expression} which will be used by a {@link - * org.apache.spark.sql.catalyst.plans.logical.RepartitionByExpression} operator. - * - * @param table the mixed-format table to write - * @param writeBase write to base store - * @return array of expressions indicate how to shuffle incoming data. - */ - public static Expression[] buildTableRequiredDistribution(MixedTable table, boolean writeBase) { - DistributionHashMode distributionHashMode = - DistributionHashMode.autoSelect(table.isKeyedTable(), !table.spec().isUnpartitioned()); - - List distributionExpressions = Lists.newArrayList(); - - if (distributionHashMode.isSupportPartition()) { - distributionExpressions.addAll(Arrays.asList(toTransforms(table.spec()))); - } - - if (distributionHashMode.isSupportPrimaryKey()) { - Transform transform = - toTransformsFromPrimary(table, table.asKeyedTable().primaryKeySpec(), writeBase); - distributionExpressions.add(transform); - } - - return distributionExpressions.toArray(new Expression[0]); - } - - private static Transform toTransformsFromPrimary( - MixedTable table, PrimaryKeySpec primaryKeySpec, boolean writeBase) { - int numBucket = - PropertyUtil.propertyAsInt( - table.properties(), - TableProperties.BASE_FILE_INDEX_HASH_BUCKET, - TableProperties.BASE_FILE_INDEX_HASH_BUCKET_DEFAULT); - - if (!writeBase) { - numBucket = - PropertyUtil.propertyAsInt( - table.properties(), - TableProperties.CHANGE_FILE_INDEX_HASH_BUCKET, - TableProperties.CHANGE_FILE_INDEX_HASH_BUCKET_DEFAULT); - } - return new FileIndexBucket(table.schema(), primaryKeySpec, numBucket - 1); - } - - /** - * Build a list of {@link org.apache.spark.sql.connector.expressions.Expression} to indicate how - * the incoming data will be sorted before write. The result of this method will covert to {@link - * org.apache.spark.sql.catalyst.expressions.Expression} list and be used for a local sort by add - * an {@link org.apache.spark.sql.catalyst.plans.logical.Sort} operator for in-coming data. - * - * @param table the mixed-format table to write to - * @param rowLevelOperation is this writing is an row-level-operation or a batch overwrite. - * @param writeBase is this writing happened in base store. - * @return array of expression to indicate how incoming data will be sorted. - */ - public static Expression[] buildTableRequiredSortOrder( - MixedTable table, boolean rowLevelOperation, boolean writeBase) { - Schema schema = table.schema(); - PartitionSpec partitionSpec = table.spec(); - PrimaryKeySpec keySpec = PrimaryKeySpec.noPrimaryKey(); - if (table.isKeyedTable()) { - keySpec = table.asKeyedTable().primaryKeySpec(); - } - boolean withMetaColumn = table.isUnkeyedTable() && rowLevelOperation; - - if (partitionSpec.isUnpartitioned() && !keySpec.primaryKeyExisted() && !withMetaColumn) { - return new Expression[0]; - } - - SortOrder.Builder builder = SortOrder.builderFor(schema); - if (partitionSpec.isPartitioned()) { - for (PartitionField field : partitionSpec.fields()) { - String sourceName = schema.findColumnName(field.sourceId()); - builder.asc( - org.apache.iceberg.expressions.Expressions.transform(sourceName, field.transform())); - } - } - SortOrder sortOrder = builder.build(); - List converted = - SortOrderVisitor.visit(sortOrder, new SortOrderToSpark(expressionHelper)); - - if (keySpec.primaryKeyExisted()) { - Transform fileIndexBucket = toTransformsFromPrimary(table, keySpec, writeBase); - converted.add(expressionHelper.sort(fileIndexBucket, true)); - } - - Expression[] orders = converted.toArray(new Expression[0]); - - if (withMetaColumn) { - orders = ObjectArrays.concat(orders, METADATA_ORDERS, Expression.class); - } - return orders; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/ExpressionHelper.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/ExpressionHelper.java deleted file mode 100644 index 628ec2500d..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/ExpressionHelper.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.util; - -import org.apache.spark.sql.amoro.catalyst.MixedFormatSpark32Helper; -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; -import org.apache.spark.sql.connector.expressions.Expression; -import org.apache.spark.sql.connector.expressions.Expressions; -import org.apache.spark.sql.connector.expressions.NullOrdering; -import org.apache.spark.sql.connector.expressions.SortDirection; -import org.apache.spark.sql.connector.expressions.SortOrder; -import org.apache.spark.sql.connector.expressions.Transform; - -public class ExpressionHelper { - - public Transform bucket(int numBuckets, String... columns) { - return Expressions.bucket(numBuckets, columns); - } - - public Transform truncate(String column, int width) { - return Expressions.apply("truncate", Expressions.column(column), Expressions.literal(width)); - } - - public Expression sort(final Expression expr, boolean ascending) { - final SortDirection direction = ascending ? SortDirection.ASCENDING : SortDirection.DESCENDING; - final NullOrdering nullOrdering = - ascending ? NullOrdering.NULLS_FIRST : NullOrdering.NULLS_LAST; - return new SortOrder() { - @Override - public Expression expression() { - return expr; - } - - @Override - public SortDirection direction() { - return direction; - } - - @Override - public NullOrdering nullOrdering() { - return nullOrdering; - } - - @Override - public String describe() { - return String.format("%s %s %s", expr.describe(), direction, nullOrdering); - } - }; - } - - public org.apache.spark.sql.catalyst.expressions.Expression toCatalyst( - Expression expr, LogicalPlan plan) { - return MixedFormatSpark32Helper.toCatalyst(expr, plan); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/MixedFormatSparkUtils.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/MixedFormatSparkUtils.java deleted file mode 100644 index 52a4b90b3b..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/MixedFormatSparkUtils.java +++ /dev/null @@ -1,195 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.util; - -import static org.apache.amoro.table.TableProperties.WRITE_DISTRIBUTION_MODE; -import static org.apache.amoro.table.TableProperties.WRITE_DISTRIBUTION_MODE_DEFAULT; -import static org.apache.iceberg.spark.Spark3Util.toTransforms; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.shade.guava32.com.google.common.base.Joiner; -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.amoro.spark.table.MixedSparkTable; -import org.apache.amoro.table.DistributionHashMode; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.amoro.table.TableProperties; -import org.apache.avro.generic.GenericData; -import org.apache.avro.util.Utf8; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.util.ByteBuffers; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.connector.catalog.CatalogPlugin; -import org.apache.spark.sql.connector.catalog.Identifier; -import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.apache.spark.sql.connector.distributions.ClusteredDistribution; -import org.apache.spark.sql.connector.distributions.Distributions; -import org.apache.spark.sql.connector.expressions.Expressions; -import org.apache.spark.sql.connector.expressions.Transform; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.unsafe.types.UTF8String; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Locale; -import java.util.Objects; - -public class MixedFormatSparkUtils { - private static final Logger LOG = LoggerFactory.getLogger(MixedFormatSparkUtils.class); - - public static TableCatalogAndIdentifier tableCatalogAndIdentifier( - SparkSession spark, List nameParts) { - Spark3Util.CatalogAndIdentifier catalogAndIdentifier = - Spark3Util.catalogAndIdentifier( - spark, nameParts, spark.sessionState().catalogManager().currentCatalog()); - CatalogPlugin catalog = catalogAndIdentifier.catalog(); - Preconditions.checkArgument( - catalog instanceof TableCatalog, - "Cannot resolver name-parts %s to catalog and identifier, %s is not a table catalog", - Joiner.on(',').join(nameParts), - catalog.name()); - return new TableCatalogAndIdentifier((TableCatalog) catalog, catalogAndIdentifier.identifier()); - } - - public static class TableCatalogAndIdentifier { - TableCatalog tableCatalog; - Identifier identifier; - - public TableCatalogAndIdentifier(TableCatalog tableCatalog, Identifier identifier) { - this.tableCatalog = tableCatalog; - this.identifier = identifier; - } - - public TableCatalog catalog() { - return this.tableCatalog; - } - - public Identifier identifier() { - return this.identifier; - } - } - - public static ClusteredDistribution buildRequiredDistribution(MixedSparkTable mixedSparkTable) { - // Fallback to use distribution mode parsed from table properties . - String modeName = - PropertyUtil.propertyAsString( - mixedSparkTable.properties(), WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_DEFAULT); - DistributionMode writeMode = DistributionMode.fromName(modeName); - switch (writeMode) { - case NONE: - return null; - - case HASH: - DistributionHashMode distributionHashMode = - DistributionHashMode.valueOfDesc( - mixedSparkTable - .properties() - .getOrDefault( - TableProperties.WRITE_DISTRIBUTION_HASH_MODE, - TableProperties.WRITE_DISTRIBUTION_HASH_MODE_DEFAULT)); - List transforms = new ArrayList<>(); - if (DistributionHashMode.AUTO.equals(distributionHashMode)) { - distributionHashMode = - DistributionHashMode.autoSelect( - mixedSparkTable.table().isKeyedTable(), - !mixedSparkTable.table().spec().isUnpartitioned()); - } - if (distributionHashMode.isSupportPrimaryKey()) { - Transform transform = - toTransformsFromPrimary( - mixedSparkTable, mixedSparkTable.table().asKeyedTable().primaryKeySpec()); - transforms.add(transform); - if (distributionHashMode.isSupportPartition()) { - transforms.addAll(Arrays.asList(toTransforms(mixedSparkTable.table().spec()))); - } - return Distributions.clustered( - transforms.stream().filter(Objects::nonNull).toArray(Transform[]::new)); - } else { - if (distributionHashMode.isSupportPartition()) { - return Distributions.clustered(toTransforms(mixedSparkTable.table().spec())); - } else { - return null; - } - } - - case RANGE: - LOG.warn( - "Fallback to use 'none' distribution mode, because {}={} is not supported in spark now", - WRITE_DISTRIBUTION_MODE, - DistributionMode.RANGE.modeName()); - return null; - - default: - throw new RuntimeException("Unrecognized write.distribution-mode: " + writeMode); - } - } - - private static Transform toTransformsFromPrimary( - MixedSparkTable mixedSparkTable, PrimaryKeySpec primaryKeySpec) { - int numBucket = - PropertyUtil.propertyAsInt( - mixedSparkTable.properties(), - TableProperties.BASE_FILE_INDEX_HASH_BUCKET, - TableProperties.BASE_FILE_INDEX_HASH_BUCKET_DEFAULT); - return Expressions.bucket(numBucket, primaryKeySpec.fieldNames().get(0)); - } - - public static Object convertConstant(Type type, Object value) { - if (value == null) { - return null; - } - - switch (type.typeId()) { - case DECIMAL: - return Decimal.apply((BigDecimal) value); - case STRING: - if (value instanceof Utf8) { - Utf8 utf8 = (Utf8) value; - return UTF8String.fromBytes(utf8.getBytes(), 0, utf8.getByteLength()); - } - return UTF8String.fromString(value.toString()); - case FIXED: - if (value instanceof byte[]) { - return value; - } else if (value instanceof GenericData.Fixed) { - return ((GenericData.Fixed) value).bytes(); - } - return ByteBuffers.toByteArray((ByteBuffer) value); - case BINARY: - return ByteBuffers.toByteArray((ByteBuffer) value); - default: - } - return value; - } - - public static String mixedTableProvider(MixedTable table) { - if (table.format().in(TableFormat.MIXED_ICEBERG, TableFormat.MIXED_HIVE)) { - return table.format().name().toLowerCase(Locale.ROOT); - } - throw new IllegalArgumentException("Not a mixed-format table:" + table.format()); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/SortOrderToSpark.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/SortOrderToSpark.java deleted file mode 100644 index ded36da33c..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/SortOrderToSpark.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.util; - -import org.apache.iceberg.NullOrder; -import org.apache.iceberg.SortDirection; -import org.apache.iceberg.transforms.SortOrderVisitor; -import org.apache.spark.sql.connector.expressions.Expression; -import org.apache.spark.sql.connector.expressions.Expressions; - -public class SortOrderToSpark implements SortOrderVisitor { - final ExpressionHelper expressionHelper; - - public SortOrderToSpark(ExpressionHelper expressionHelper) { - this.expressionHelper = expressionHelper; - } - - @Override - public Expression field( - String sourceName, int sourceId, SortDirection direction, NullOrder nullOrder) { - return expressionHelper.sort(Expressions.column(sourceName), ascending(direction)); - } - - @Override - public Expression bucket( - String sourceName, int sourceId, int width, SortDirection direction, NullOrder nullOrder) { - return expressionHelper.sort(Expressions.bucket(width, sourceName), ascending(direction)); - } - - @Override - public Expression truncate( - String sourceName, int sourceId, int width, SortDirection direction, NullOrder nullOrder) { - return expressionHelper.sort( - expressionHelper.truncate(sourceName, width), ascending(direction)); - } - - @Override - public Expression year( - String sourceName, int sourceId, SortDirection direction, NullOrder nullOrder) { - return expressionHelper.sort(Expressions.years(sourceName), ascending(direction)); - } - - @Override - public Expression month( - String sourceName, int sourceId, SortDirection direction, NullOrder nullOrder) { - return expressionHelper.sort(Expressions.months(sourceName), ascending(direction)); - } - - @Override - public Expression day( - String sourceName, int sourceId, SortDirection direction, NullOrder nullOrder) { - return expressionHelper.sort(Expressions.days(sourceName), ascending(direction)); - } - - @Override - public Expression hour( - String sourceName, int sourceId, SortDirection direction, NullOrder nullOrder) { - return expressionHelper.sort(Expressions.hours(sourceName), ascending(direction)); - } - - private boolean ascending(SortDirection direction) { - return direction == SortDirection.ASC; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/Stats.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/Stats.java deleted file mode 100644 index a7d76d92c4..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/util/Stats.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.util; - -import org.apache.spark.sql.connector.read.Statistics; - -import java.util.OptionalLong; - -public class Stats implements Statistics { - private final OptionalLong sizeInBytes; - private final OptionalLong numRows; - - public Stats(long sizeInBytes, long numRows) { - this.sizeInBytes = OptionalLong.of(sizeInBytes); - this.numRows = OptionalLong.of(numRows); - } - - @Override - public OptionalLong sizeInBytes() { - return sizeInBytes; - } - - @Override - public OptionalLong numRows() { - return numRows; - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/KeyedSparkBatchWrite.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/KeyedSparkBatchWrite.java deleted file mode 100644 index af333c3968..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/KeyedSparkBatchWrite.java +++ /dev/null @@ -1,346 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.writer; - -import static org.apache.amoro.hive.op.UpdateHiveFiles.DELETE_UNTRACKED_HIVE_FILE; -import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; -import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; -import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; -import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; -import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; -import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; -import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; -import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; - -import org.apache.amoro.api.BlockableOperation; -import org.apache.amoro.api.OperationConflictException; -import org.apache.amoro.hive.utils.HiveTableUtil; -import org.apache.amoro.mixed.MixedFormatCatalog; -import org.apache.amoro.op.OverwriteBaseFiles; -import org.apache.amoro.op.RewritePartitions; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.io.TaskWriters; -import org.apache.amoro.spark.sql.utils.RowDeltaUtils; -import org.apache.amoro.table.KeyedTable; -import org.apache.amoro.table.blocker.Blocker; -import org.apache.amoro.table.blocker.TableBlockerManager; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.Tasks; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.connector.write.BatchWrite; -import org.apache.spark.sql.connector.write.DataWriter; -import org.apache.spark.sql.connector.write.DataWriterFactory; -import org.apache.spark.sql.connector.write.LogicalWriteInfo; -import org.apache.spark.sql.connector.write.PhysicalWriteInfo; -import org.apache.spark.sql.connector.write.Write; -import org.apache.spark.sql.connector.write.WriterCommitMessage; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class KeyedSparkBatchWrite implements MixedFormatSparkWriteBuilder.MixedFormatWrite, Write { - private final KeyedTable table; - private final StructType dsSchema; - - private final long txId; - private final String hiveSubdirectory; - - private final boolean orderedWriter; - private final MixedFormatCatalog catalog; - - KeyedSparkBatchWrite(KeyedTable table, LogicalWriteInfo info, MixedFormatCatalog catalog) { - this.table = table; - this.dsSchema = info.schema(); - this.txId = table.beginTransaction(null); - this.hiveSubdirectory = HiveTableUtil.newHiveSubdirectory(this.txId); - this.orderedWriter = - Boolean.parseBoolean( - info.options().getOrDefault("writer.distributed-and-ordered", "false")); - this.catalog = catalog; - } - - @Override - public BatchWrite asBatchAppend() { - return new AppendWrite(); - } - - @Override - public BatchWrite asDynamicOverwrite() { - return new DynamicOverwrite(); - } - - @Override - public BatchWrite asOverwriteByFilter(Expression overwriteExpr) { - return new OverwriteByFilter(overwriteExpr); - } - - @Override - public BatchWrite asUpsertWrite() { - return new UpsertWrite(); - } - - private abstract class BaseBatchWrite implements BatchWrite { - - protected TableBlockerManager tableBlockerManager; - protected Blocker block; - - @Override - public void abort(WriterCommitMessage[] messages) { - try { - Map props = table.properties(); - Tasks.foreach(WriteTaskCommit.files(messages)) - .retry( - PropertyUtil.propertyAsInt(props, COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT)) - .exponentialBackoff( - PropertyUtil.propertyAsInt( - props, COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT), - PropertyUtil.propertyAsInt( - props, COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT), - PropertyUtil.propertyAsInt( - props, COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), - 2.0 /* exponential */) - .throwFailureWhenFinished() - .run( - file -> { - table.io().deleteFile(file.path().toString()); - }); - } finally { - tableBlockerManager.release(block); - } - } - - public void checkBlocker(TableBlockerManager tableBlockerManager) { - List blockerIds = - tableBlockerManager.getBlockers().stream() - .map(Blocker::blockerId) - .collect(Collectors.toList()); - if (!blockerIds.contains(block.blockerId())) { - throw new IllegalStateException("block is not in blockerManager"); - } - } - - public void getBlocker() { - this.tableBlockerManager = catalog.getTableBlockerManager(table.id()); - ArrayList operations = Lists.newArrayList(); - operations.add(BlockableOperation.BATCH_WRITE); - operations.add(BlockableOperation.OPTIMIZE); - try { - this.block = tableBlockerManager.block(operations); - } catch (OperationConflictException e) { - throw new IllegalStateException( - "failed to block table " + table.id() + " with " + operations, e); - } - } - } - - private class AppendWrite extends BaseBatchWrite { - - @Override - public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - getBlocker(); - return new ChangeWriteFactory(table, dsSchema, txId, orderedWriter); - } - - @Override - public void commit(WriterCommitMessage[] messages) { - checkBlocker(tableBlockerManager); - AppendFiles append = table.changeTable().newAppend(); - for (DataFile file : WriteTaskCommit.files(messages)) { - append.appendFile(file); - } - append.commit(); - tableBlockerManager.release(block); - } - } - - private class DynamicOverwrite extends BaseBatchWrite { - - @Override - public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - getBlocker(); - return new BaseWriterFactory(table, dsSchema, txId, hiveSubdirectory, orderedWriter); - } - - @Override - public void commit(WriterCommitMessage[] messages) { - checkBlocker(tableBlockerManager); - RewritePartitions rewritePartitions = table.newRewritePartitions(); - rewritePartitions.updateOptimizedSequenceDynamically(txId); - - for (DataFile file : WriteTaskCommit.files(messages)) { - rewritePartitions.addDataFile(file); - } - rewritePartitions.commit(); - tableBlockerManager.release(block); - } - } - - private class OverwriteByFilter extends BaseBatchWrite { - private final Expression overwriteExpr; - - private OverwriteByFilter(Expression overwriteExpr) { - this.overwriteExpr = overwriteExpr; - } - - @Override - public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - getBlocker(); - return new BaseWriterFactory(table, dsSchema, txId, hiveSubdirectory, orderedWriter); - } - - @Override - public void commit(WriterCommitMessage[] messages) { - checkBlocker(tableBlockerManager); - OverwriteBaseFiles overwriteBaseFiles = table.newOverwriteBaseFiles(); - overwriteBaseFiles.overwriteByRowFilter(overwriteExpr); - overwriteBaseFiles.updateOptimizedSequenceDynamically(txId); - overwriteBaseFiles.set(DELETE_UNTRACKED_HIVE_FILE, "true"); - - for (DataFile file : WriteTaskCommit.files(messages)) { - overwriteBaseFiles.addFile(file); - } - overwriteBaseFiles.commit(); - tableBlockerManager.release(block); - } - } - - private class UpsertWrite extends BaseBatchWrite { - - @Override - public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - getBlocker(); - return new UpsertChangeFactory(table, dsSchema, txId, orderedWriter); - } - - @Override - public void commit(WriterCommitMessage[] messages) { - checkBlocker(tableBlockerManager); - AppendFiles append = table.changeTable().newAppend(); - for (DataFile file : WriteTaskCommit.files(messages)) { - append.appendFile(file); - } - append.commit(); - tableBlockerManager.release(block); - } - } - - private abstract static class AbstractWriterFactory implements DataWriterFactory, Serializable { - protected final KeyedTable table; - protected final StructType dsSchema; - protected final Long transactionId; - protected final boolean orderedWrite; - - AbstractWriterFactory( - KeyedTable table, StructType dsSchema, Long transactionId, boolean orderedWrite) { - this.table = table; - this.dsSchema = dsSchema; - this.transactionId = transactionId; - this.orderedWrite = orderedWrite; - } - - public TaskWriter newWriter(int partitionId, long taskId, StructType schema) { - return TaskWriters.of(table) - .withTransactionId(transactionId) - .withPartitionId(partitionId) - .withTaskId(taskId) - .withDataSourceSchema(schema) - .withOrderedWriter(orderedWrite) - .newChangeWriter(); - } - } - - private static class BaseWriterFactory extends AbstractWriterFactory { - - protected final String hiveSubdirectory; - - BaseWriterFactory( - KeyedTable table, - StructType dsSchema, - Long transactionId, - String hiveSubdirectory, - boolean orderedWrite) { - super(table, dsSchema, transactionId, orderedWrite); - this.hiveSubdirectory = hiveSubdirectory; - } - - @Override - public DataWriter createWriter(int partitionId, long taskId) { - TaskWriters writerBuilder = - TaskWriters.of(table) - .withTransactionId(transactionId) - .withPartitionId(partitionId) - .withTaskId(taskId) - .withDataSourceSchema(dsSchema) - .withOrderedWriter(orderedWrite) - .withHiveSubdirectory(hiveSubdirectory); - - TaskWriter writer = writerBuilder.newBaseWriter(true); - return new SimpleInternalRowDataWriter(writer); - } - } - - private static class ChangeWriteFactory extends AbstractWriterFactory { - - ChangeWriteFactory( - KeyedTable table, StructType dsSchema, long transactionId, boolean orderedWrite) { - super(table, dsSchema, transactionId, orderedWrite); - } - - @Override - public DataWriter createWriter(int partitionId, long taskId) { - return new SimpleRowLevelDataWriter( - newWriter(partitionId, taskId, dsSchema), - newWriter(partitionId, taskId, dsSchema), - dsSchema, - true); - } - } - - private static class UpsertChangeFactory extends AbstractWriterFactory { - - UpsertChangeFactory( - KeyedTable table, StructType dsSchema, long transactionId, boolean orderedWrite) { - super(table, dsSchema, transactionId, orderedWrite); - } - - @Override - public DataWriter createWriter(int partitionId, long taskId) { - StructType schema = - new StructType( - Arrays.stream(dsSchema.fields()) - .filter(field -> !field.name().equals(RowDeltaUtils.OPERATION_COLUMN())) - .toArray(StructField[]::new)); - return new SimpleRowLevelDataWriter( - newWriter(partitionId, taskId, schema), - newWriter(partitionId, taskId, schema), - dsSchema, - true); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/MixedFormatSparkWriteBuilder.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/MixedFormatSparkWriteBuilder.java deleted file mode 100644 index c749cd2d4f..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/MixedFormatSparkWriteBuilder.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.writer; - -import org.apache.amoro.mixed.MixedFormatCatalog; -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.amoro.table.MixedTable; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.spark.SparkFilters; -import org.apache.spark.sql.connector.write.BatchWrite; -import org.apache.spark.sql.connector.write.LogicalWriteInfo; -import org.apache.spark.sql.connector.write.SupportsDynamicOverwrite; -import org.apache.spark.sql.connector.write.SupportsOverwrite; -import org.apache.spark.sql.connector.write.Write; -import org.apache.spark.sql.connector.write.WriteBuilder; -import org.apache.spark.sql.sources.Filter; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; - -public class MixedFormatSparkWriteBuilder - implements WriteBuilder, SupportsDynamicOverwrite, SupportsOverwrite { - - public interface MixedFormatWrite { - - BatchWrite asBatchAppend(); - - BatchWrite asDynamicOverwrite(); - - BatchWrite asOverwriteByFilter(Expression overwriteExpr); - - BatchWrite asUpsertWrite(); - } - - protected final CaseInsensitiveStringMap options; - - protected Expression overwriteExpr = null; - - private WriteMode writeMode = WriteMode.APPEND; - // private final MixedFormatWrite write; - private final MixedTable table; - private final LogicalWriteInfo info; - private final MixedFormatCatalog catalog; - - public MixedFormatSparkWriteBuilder( - MixedTable table, LogicalWriteInfo info, MixedFormatCatalog catalog) { - this.options = info.options(); - if (options.containsKey(WriteMode.WRITE_MODE_KEY)) { - this.writeMode = WriteMode.getWriteMode(options.get(WriteMode.WRITE_MODE_KEY)); - } - this.table = table; - this.info = info; - this.catalog = catalog; - } - - @Override - public WriteBuilder overwriteDynamicPartitions() { - Preconditions.checkState( - overwriteExpr == null, "Cannot overwrite dynamically and by filter: %s", overwriteExpr); - writeMode = WriteMode.OVERWRITE_DYNAMIC; - return this; - } - - @Override - public WriteBuilder overwrite(Filter[] filters) { - this.overwriteExpr = SparkFilters.convert(filters); - String overwriteMode = options.getOrDefault("overwrite-mode", "null"); - if (overwriteExpr == Expressions.alwaysTrue() && "dynamic".equals(overwriteMode)) { - writeMode = WriteMode.OVERWRITE_DYNAMIC; - } else { - writeMode = WriteMode.OVERWRITE_BY_FILTER; - } - return this; - } - - @Override - public Write build() { - if (table.isKeyedTable()) { - return new KeyedSparkBatchWrite(table.asKeyedTable(), info, catalog) { - @Override - public BatchWrite toBatch() { - switch (writeMode) { - case APPEND: - return asBatchAppend(); - case OVERWRITE_BY_FILTER: - return asOverwriteByFilter(overwriteExpr); - case OVERWRITE_DYNAMIC: - return asDynamicOverwrite(); - case UPSERT: - return asUpsertWrite(); - default: - throw new UnsupportedOperationException("unsupported write mode: " + writeMode); - } - } - }; - } else { - return new UnkeyedSparkBatchWrite(table.asUnkeyedTable(), info, catalog) { - @Override - public BatchWrite toBatch() { - switch (writeMode) { - case APPEND: - return asBatchAppend(); - case OVERWRITE_BY_FILTER: - return asOverwriteByFilter(overwriteExpr); - case OVERWRITE_DYNAMIC: - return asDynamicOverwrite(); - case UPSERT: - return asUpsertWrite(); - default: - throw new UnsupportedOperationException("unsupported write mode: " + writeMode); - } - } - }; - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/RowLevelWriter.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/RowLevelWriter.java deleted file mode 100644 index fee3174fd2..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/RowLevelWriter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.writer; - -import org.apache.spark.sql.connector.write.DataWriter; - -import java.io.IOException; - -/** A data writer responsible for writing a delta of rows. */ -public interface RowLevelWriter extends DataWriter { - void delete(T row) throws IOException; - - void update(T updateBefore, T updateAfter) throws IOException; - - void insert(T row) throws IOException; - - @Override - default void write(T row) throws IOException { - insert(row); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/SimpleInternalRowDataWriter.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/SimpleInternalRowDataWriter.java deleted file mode 100644 index 7db3da725e..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/SimpleInternalRowDataWriter.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.writer; - -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.connector.write.DataWriter; -import org.apache.spark.sql.connector.write.WriterCommitMessage; - -import java.io.IOException; - -public class SimpleInternalRowDataWriter implements DataWriter { - final TaskWriter writer; - - public SimpleInternalRowDataWriter(TaskWriter writer) { - this.writer = writer; - } - - @Override - public void write(InternalRow record) throws IOException { - writer.write(record); - } - - @Override - public WriterCommitMessage commit() throws IOException { - WriteResult result = writer.complete(); - return new WriteTaskCommit.Builder() - .addDataFiles(result.dataFiles()) - .addDeleteFiles(result.deleteFiles()) - .build(); - } - - @Override - public void abort() throws IOException { - if (this.writer != null) { - this.writer.abort(); - } - } - - @Override - public void close() throws IOException { - if (this.writer != null) { - writer.close(); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/SimpleRowLevelDataWriter.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/SimpleRowLevelDataWriter.java deleted file mode 100644 index 4690c010b7..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/SimpleRowLevelDataWriter.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.writer; - -import org.apache.amoro.data.ChangeAction; -import org.apache.amoro.spark.SparkInternalRowCastWrapper; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.connector.write.WriterCommitMessage; -import org.apache.spark.sql.types.StructType; - -import java.io.IOException; - -public class SimpleRowLevelDataWriter implements RowLevelWriter { - private final TaskWriter insertWriter; - private final TaskWriter deleteWrite; - - private final StructType schema; - private final boolean isKeyedTable; - - public SimpleRowLevelDataWriter( - TaskWriter insertWriter, - TaskWriter deleteWrite, - StructType schema, - boolean isKeyedTable) { - this.insertWriter = insertWriter; - this.deleteWrite = deleteWrite; - this.schema = schema; - this.isKeyedTable = isKeyedTable; - } - - @Override - public void delete(InternalRow row) throws IOException { - deleteWrite.write(new SparkInternalRowCastWrapper(row, ChangeAction.DELETE, schema)); - } - - @Override - public void update(InternalRow updateBefore, InternalRow updateAfter) throws IOException { - SparkInternalRowCastWrapper delete; - SparkInternalRowCastWrapper insert; - if (isKeyedTable) { - delete = new SparkInternalRowCastWrapper(updateBefore, ChangeAction.UPDATE_BEFORE, schema); - insert = new SparkInternalRowCastWrapper(updateAfter, ChangeAction.UPDATE_AFTER, schema); - } else { - delete = new SparkInternalRowCastWrapper(updateBefore, ChangeAction.DELETE, schema); - insert = new SparkInternalRowCastWrapper(updateAfter, ChangeAction.INSERT, schema); - } - if (!rowIsAllNull(delete)) { - insertWriter.write(delete); - } - insertWriter.write(insert); - } - - private boolean rowIsAllNull(SparkInternalRowCastWrapper row) { - boolean isAllNull = true; - for (int i = 0; i < row.getSchema().size(); i++) { - if (!row.getRow().isNullAt(i)) { - isAllNull = false; - } - } - return isAllNull; - } - - @Override - public void insert(InternalRow row) throws IOException { - insertWriter.write(new SparkInternalRowCastWrapper(row, ChangeAction.INSERT, schema)); - } - - @Override - public WriterCommitMessage commit() throws IOException { - WriteResult insert = this.insertWriter.complete(); - WriteResult delete = this.deleteWrite.complete(); - return new WriteTaskCommit.Builder() - .addDataFiles(insert.dataFiles()) - .addDataFiles(delete.dataFiles()) - .addDeleteFiles(insert.deleteFiles()) - .addDeleteFiles(delete.deleteFiles()) - .build(); - } - - @Override - public void abort() throws IOException { - if (this.insertWriter != null) { - this.insertWriter.abort(); - } - if (this.deleteWrite != null) { - this.deleteWrite.abort(); - } - } - - @Override - public void close() throws IOException { - if (this.insertWriter != null) { - this.insertWriter.close(); - } - if (this.deleteWrite != null) { - this.deleteWrite.close(); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/UnkeyedSparkBatchWrite.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/UnkeyedSparkBatchWrite.java deleted file mode 100644 index 1e84972578..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/UnkeyedSparkBatchWrite.java +++ /dev/null @@ -1,326 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.writer; - -import static org.apache.amoro.hive.op.UpdateHiveFiles.DELETE_UNTRACKED_HIVE_FILE; -import static org.apache.amoro.spark.writer.WriteTaskCommit.files; -import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; -import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; -import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; -import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; -import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; -import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; -import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; -import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; - -import org.apache.amoro.api.BlockableOperation; -import org.apache.amoro.api.OperationConflictException; -import org.apache.amoro.hive.utils.HiveTableUtil; -import org.apache.amoro.mixed.MixedFormatCatalog; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.io.TaskWriters; -import org.apache.amoro.table.UnkeyedTable; -import org.apache.amoro.table.blocker.Blocker; -import org.apache.amoro.table.blocker.TableBlockerManager; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.OverwriteFiles; -import org.apache.iceberg.ReplacePartitions; -import org.apache.iceberg.RowDelta; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.Tasks; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.connector.write.BatchWrite; -import org.apache.spark.sql.connector.write.DataWriter; -import org.apache.spark.sql.connector.write.DataWriterFactory; -import org.apache.spark.sql.connector.write.LogicalWriteInfo; -import org.apache.spark.sql.connector.write.PhysicalWriteInfo; -import org.apache.spark.sql.connector.write.Write; -import org.apache.spark.sql.connector.write.WriterCommitMessage; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class UnkeyedSparkBatchWrite - implements MixedFormatSparkWriteBuilder.MixedFormatWrite, Write { - - private final UnkeyedTable table; - private final StructType dsSchema; - private final String hiveSubdirectory = HiveTableUtil.newHiveSubdirectory(); - - private final boolean orderedWriter; - - private final MixedFormatCatalog catalog; - - public UnkeyedSparkBatchWrite( - UnkeyedTable table, LogicalWriteInfo info, MixedFormatCatalog catalog) { - this.table = table; - this.dsSchema = info.schema(); - this.orderedWriter = - Boolean.parseBoolean( - info.options().getOrDefault("writer.distributed-and-ordered", "false")); - this.catalog = catalog; - } - - @Override - public BatchWrite asBatchAppend() { - return new AppendWrite(); - } - - @Override - public BatchWrite asDynamicOverwrite() { - return new DynamicOverwrite(); - } - - @Override - public BatchWrite asOverwriteByFilter(Expression overwriteExpr) { - return new OverwriteByFilter(overwriteExpr); - } - - @Override - public BatchWrite asUpsertWrite() { - return new UpsertWrite(); - } - - private abstract class BaseBatchWrite implements BatchWrite { - - protected TableBlockerManager tableBlockerManager; - protected Blocker block; - - @Override - public void abort(WriterCommitMessage[] messages) { - try { - Map props = table.properties(); - Tasks.foreach(files(messages)) - .retry( - PropertyUtil.propertyAsInt(props, COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT)) - .exponentialBackoff( - PropertyUtil.propertyAsInt( - props, COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT), - PropertyUtil.propertyAsInt( - props, COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT), - PropertyUtil.propertyAsInt( - props, COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), - 2.0 /* exponential */) - .throwFailureWhenFinished() - .run( - file -> { - table.io().deleteFile(file.path().toString()); - }); - } finally { - tableBlockerManager.release(block); - } - } - - public void checkBlocker(TableBlockerManager tableBlockerManager) { - List blockerIds = - tableBlockerManager.getBlockers().stream() - .map(Blocker::blockerId) - .collect(Collectors.toList()); - if (!blockerIds.contains(block.blockerId())) { - throw new IllegalStateException("block is not in blockerManager"); - } - } - - public void getBlocker() { - this.tableBlockerManager = catalog.getTableBlockerManager(table.id()); - ArrayList operations = Lists.newArrayList(); - operations.add(BlockableOperation.BATCH_WRITE); - operations.add(BlockableOperation.OPTIMIZE); - try { - this.block = tableBlockerManager.block(operations); - } catch (OperationConflictException e) { - throw new IllegalStateException( - "failed to block table " + table.id() + " with " + operations, e); - } - } - } - - private class AppendWrite extends BaseBatchWrite { - - @Override - public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - getBlocker(); - return new WriterFactory(table, dsSchema, false, null, orderedWriter); - } - - @Override - public void commit(WriterCommitMessage[] messages) { - checkBlocker(tableBlockerManager); - AppendFiles appendFiles = table.newAppend(); - for (DataFile file : files(messages)) { - appendFiles.appendFile(file); - } - appendFiles.commit(); - tableBlockerManager.release(block); - } - } - - private class DynamicOverwrite extends BaseBatchWrite { - - @Override - public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - getBlocker(); - return new WriterFactory(table, dsSchema, true, hiveSubdirectory, orderedWriter); - } - - @Override - public void commit(WriterCommitMessage[] messages) { - checkBlocker(tableBlockerManager); - ReplacePartitions replacePartitions = table.newReplacePartitions(); - for (DataFile file : files(messages)) { - replacePartitions.addFile(file); - } - replacePartitions.commit(); - tableBlockerManager.release(block); - } - } - - private class OverwriteByFilter extends BaseBatchWrite { - private final Expression overwriteExpr; - - private OverwriteByFilter(Expression overwriteExpr) { - this.overwriteExpr = overwriteExpr; - } - - @Override - public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - getBlocker(); - return new WriterFactory(table, dsSchema, true, hiveSubdirectory, orderedWriter); - } - - @Override - public void commit(WriterCommitMessage[] messages) { - checkBlocker(tableBlockerManager); - OverwriteFiles overwriteFiles = table.newOverwrite(); - overwriteFiles.overwriteByRowFilter(overwriteExpr); - overwriteFiles.set(DELETE_UNTRACKED_HIVE_FILE, "true"); - for (DataFile file : files(messages)) { - overwriteFiles.addFile(file); - } - overwriteFiles.commit(); - tableBlockerManager.release(block); - } - } - - private class UpsertWrite extends BaseBatchWrite { - @Override - public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - getBlocker(); - return new DeltaUpsertWriteFactory(table, dsSchema, orderedWriter); - } - - @Override - public void commit(WriterCommitMessage[] messages) { - checkBlocker(tableBlockerManager); - RowDelta rowDelta = table.newRowDelta(); - if (WriteTaskCommit.deleteFiles(messages).iterator().hasNext()) { - for (DeleteFile file : WriteTaskCommit.deleteFiles(messages)) { - rowDelta.addDeletes(file); - } - } - if (WriteTaskCommit.files(messages).iterator().hasNext()) { - for (DataFile file : WriteTaskCommit.files(messages)) { - rowDelta.addRows(file); - } - } - rowDelta.commit(); - tableBlockerManager.release(block); - } - } - - private static class WriterFactory implements DataWriterFactory, Serializable { - protected final UnkeyedTable table; - protected final StructType dsSchema; - - protected final String hiveSubdirectory; - - protected final boolean isOverwrite; - protected final boolean orderedWriter; - - WriterFactory( - UnkeyedTable table, - StructType dsSchema, - boolean isOverwrite, - String hiveSubdirectory, - boolean orderedWrite) { - this.table = table; - this.dsSchema = dsSchema; - this.isOverwrite = isOverwrite; - this.hiveSubdirectory = hiveSubdirectory; - this.orderedWriter = orderedWrite; - } - - @Override - public DataWriter createWriter(int partitionId, long taskId) { - TaskWriters builder = - TaskWriters.of(table) - .withPartitionId(partitionId) - .withTaskId(taskId) - .withOrderedWriter(orderedWriter) - .withDataSourceSchema(dsSchema) - .withHiveSubdirectory(hiveSubdirectory); - - TaskWriter writer = builder.newBaseWriter(this.isOverwrite); - return new SimpleInternalRowDataWriter(writer); - } - - public TaskWriter newWriter(int partitionId, long taskId, StructType schema) { - return TaskWriters.of(table) - .withPartitionId(partitionId) - .withTaskId(taskId) - .withDataSourceSchema(schema) - .newUnkeyedUpsertWriter(); - } - } - - private static class DeltaUpsertWriteFactory extends WriterFactory { - - DeltaUpsertWriteFactory(UnkeyedTable table, StructType dsSchema, boolean ordredWriter) { - super(table, dsSchema, false, null, ordredWriter); - } - - @Override - public DataWriter createWriter(int partitionId, long taskId) { - StructType schema = - new StructType( - Arrays.stream(dsSchema.fields()) - .filter( - f -> - !f.name().equals("_file") - && !f.name().equals("_pos") - && !f.name().equals("_upsert_op")) - .toArray(StructField[]::new)); - return new SimpleRowLevelDataWriter( - newWriter(partitionId, taskId, schema), - newWriter(partitionId, taskId, schema), - dsSchema, - table.isKeyedTable()); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/WriteMode.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/WriteMode.java deleted file mode 100644 index 1e680f0d0a..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/WriteMode.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.writer; - -public enum WriteMode { - OVERWRITE_BY_FILTER("overwrite-by-filter"), - OVERWRITE_DYNAMIC("overwrite-dynamic"), - APPEND("append"), - UPSERT("upsert"); - - public static final String WRITE_MODE_KEY = "write-mode"; - - public final String mode; - - WriteMode(String mode) { - this.mode = mode; - } - - public static WriteMode getWriteMode(String mode) { - for (WriteMode m : values()) { - if (m.mode.equalsIgnoreCase(mode)) { - return m; - } - } - throw new IllegalArgumentException("Invalid write mode: " + mode); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/WriteTaskCommit.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/WriteTaskCommit.java deleted file mode 100644 index bf1b6cf016..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/amoro/spark/writer/WriteTaskCommit.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.writer; - -import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableList; -import org.apache.amoro.shade.guava32.com.google.common.collect.Iterables; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.spark.sql.connector.write.WriterCommitMessage; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -public class WriteTaskCommit implements WriterCommitMessage { - private final DataFile[] dataFiles; - private final DeleteFile[] deleteFiles; - - WriteTaskCommit(List dataFiles, List deleteFiles) { - this.dataFiles = dataFiles.toArray(new DataFile[0]); - this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); - } - - DataFile[] files() { - return dataFiles; - } - - public static Iterable files(WriterCommitMessage[] messages) { - if (messages.length > 0) { - return Iterables.concat( - Iterables.transform( - Arrays.asList(messages), - message -> - message != null - ? ImmutableList.copyOf(((WriteTaskCommit) message).files()) - : ImmutableList.of())); - } - return ImmutableList.of(); - } - - DeleteFile[] deleteFiles() { - return deleteFiles; - } - - public static Iterable deleteFiles(WriterCommitMessage[] messages) { - if (messages.length > 0) { - return Iterables.concat( - Iterables.transform( - Arrays.asList(messages), - message -> - message != null - ? ImmutableList.copyOf(((WriteTaskCommit) message).deleteFiles()) - : ImmutableList.of())); - } - return ImmutableList.of(); - } - - public static class Builder { - private final List dataFiles; - private final List deleteFiles; - - Builder() { - this.dataFiles = new ArrayList<>(); - this.deleteFiles = new ArrayList<>(); - } - - public Builder add(WriteTaskCommit result) { - addDataFiles(result.dataFiles); - addDeleteFiles(result.deleteFiles); - return this; - } - - public Builder addAll(Iterable results) { - results.forEach(this::add); - return this; - } - - public Builder addDataFiles(DataFile... files) { - Collections.addAll(dataFiles, files); - return this; - } - - public Builder addDataFiles(Iterable files) { - Iterables.addAll(dataFiles, files); - return this; - } - - public Builder addDeleteFiles(DeleteFile... files) { - Collections.addAll(deleteFiles, files); - return this; - } - - public Builder addDeleteFiles(Iterable files) { - Iterables.addAll(deleteFiles, files); - return this; - } - - public WriteTaskCommit build() { - return new WriteTaskCommit(dataFiles, deleteFiles); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/iceberg/spark/data/AdaptHiveParquetWithSparkSchemaVisitor.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/iceberg/spark/data/AdaptHiveParquetWithSparkSchemaVisitor.java deleted file mode 100644 index ecfb62e0b5..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/iceberg/spark/data/AdaptHiveParquetWithSparkSchemaVisitor.java +++ /dev/null @@ -1,236 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.iceberg.spark.data; - -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.OriginalType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; -import org.apache.parquet.schema.Type.Repetition; -import org.apache.spark.sql.types.ArrayType; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.MapType; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -import java.util.Deque; -import java.util.List; - -/** - * Copy from iceberg {@link ParquetWithSparkSchemaVisitor} to change some code, see annotation - * "Change for mixed-format table" - */ -public class AdaptHiveParquetWithSparkSchemaVisitor { - private final Deque fieldNames = Lists.newLinkedList(); - - public static T visit( - DataType sType, Type type, AdaptHiveParquetWithSparkSchemaVisitor visitor) { - Preconditions.checkArgument(sType != null, "Invalid DataType: null"); - if (type instanceof MessageType) { - Preconditions.checkArgument( - sType instanceof StructType, "Invalid struct: %s is not a struct", sType); - StructType struct = (StructType) sType; - return visitor.message( - struct, (MessageType) type, visitFields(struct, type.asGroupType(), visitor)); - - } else if (type.isPrimitive()) { - return visitor.primitive(sType, type.asPrimitiveType()); - - } else { - // if not a primitive, the typeId must be a group - GroupType group = type.asGroupType(); - OriginalType annotation = group.getOriginalType(); - if (annotation != null) { - switch (annotation) { - case LIST: - Preconditions.checkArgument( - !group.isRepetition(Repetition.REPEATED), - "Invalid list: top-level group is repeated: %s", - group); - Preconditions.checkArgument( - group.getFieldCount() == 1, - "Invalid list: does not contain single repeated field: %s", - group); - - GroupType repeatedElement = group.getFields().get(0).asGroupType(); - Preconditions.checkArgument( - repeatedElement.isRepetition(Repetition.REPEATED), - "Invalid list: inner group is not repeated"); - Preconditions.checkArgument( - repeatedElement.getFieldCount() <= 1, - "Invalid list: repeated group is not a single field: %s", - group); - - Preconditions.checkArgument( - sType instanceof ArrayType, "Invalid list: %s is not an array", sType); - ArrayType array = (ArrayType) sType; - StructField element = - new StructField( - "element", array.elementType(), array.containsNull(), Metadata.empty()); - - visitor.fieldNames.push(repeatedElement.getName()); - try { - T elementResult = null; - if (repeatedElement.getFieldCount() > 0) { - elementResult = visitField(element, repeatedElement.getType(0), visitor); - } - - return visitor.list(array, group, elementResult); - - } finally { - visitor.fieldNames.pop(); - } - - case MAP: - Preconditions.checkArgument( - !group.isRepetition(Repetition.REPEATED), - "Invalid map: top-level group is repeated: %s", - group); - Preconditions.checkArgument( - group.getFieldCount() == 1, - "Invalid map: does not contain single repeated field: %s", - group); - - GroupType repeatedKeyValue = group.getType(0).asGroupType(); - Preconditions.checkArgument( - repeatedKeyValue.isRepetition(Repetition.REPEATED), - "Invalid map: inner group is not repeated"); - Preconditions.checkArgument( - repeatedKeyValue.getFieldCount() <= 2, - "Invalid map: repeated group does not have 2 fields"); - - Preconditions.checkArgument( - sType instanceof MapType, "Invalid map: %s is not a map", sType); - MapType map = (MapType) sType; - StructField keyField = new StructField("key", map.keyType(), false, Metadata.empty()); - StructField valueField = - new StructField( - "value", map.valueType(), map.valueContainsNull(), Metadata.empty()); - - visitor.fieldNames.push(repeatedKeyValue.getName()); - try { - T keyResult = null; - T valueResult = null; - switch (repeatedKeyValue.getFieldCount()) { - case 2: - // if there are 2 fields, both key and value are projected - keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor); - valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor); - break; - case 1: - // if there is just one, use the name to determine what it is - Type keyOrValue = repeatedKeyValue.getType(0); - if (keyOrValue.getName().equalsIgnoreCase("key")) { - keyResult = visitField(keyField, keyOrValue, visitor); - // value result remains null - } else { - valueResult = visitField(valueField, keyOrValue, visitor); - // key result remains null - } - break; - default: - // both results will remain null - } - - return visitor.map(map, group, keyResult, valueResult); - - } finally { - visitor.fieldNames.pop(); - } - - default: - } - } - - Preconditions.checkArgument( - sType instanceof StructType, "Invalid struct: %s is not a struct", sType); - StructType struct = (StructType) sType; - return visitor.struct(struct, group, visitFields(struct, group, visitor)); - } - } - - private static T visitField( - StructField sField, Type field, AdaptHiveParquetWithSparkSchemaVisitor visitor) { - visitor.fieldNames.push(field.getName()); - try { - return visit(sField.dataType(), field, visitor); - } finally { - visitor.fieldNames.pop(); - } - } - - private static List visitFields( - StructType struct, GroupType group, AdaptHiveParquetWithSparkSchemaVisitor visitor) { - StructField[] sFields = struct.fields(); - Preconditions.checkArgument( - sFields.length == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); - List results = Lists.newArrayListWithExpectedSize(group.getFieldCount()); - for (int i = 0; i < sFields.length; i += 1) { - Type field = group.getFields().get(i); - StructField sField = sFields[i]; - - // Change for mixed-format table ⬇ - // Preconditions.checkArgument(field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.name())), - // "Structs do not match: field %s != %s", field.getName(), sField.name()); - Preconditions.checkArgument( - field.getName().equals(sField.name()), - "Structs do not match: field %s != %s", - field.getName(), - sField.name()); - // Change for mixed-format table ⬆ - results.add(visitField(sField, field, visitor)); - } - - return results; - } - - public T message(StructType sStruct, MessageType message, List fields) { - return null; - } - - public T struct(StructType sStruct, GroupType struct, List fields) { - return null; - } - - public T list(ArrayType sArray, GroupType array, T element) { - return null; - } - - public T map(MapType sMap, GroupType map, T key, T value) { - return null; - } - - public T primitive(DataType sPrimitive, PrimitiveType primitive) { - return null; - } - - protected String[] currentPath() { - return Lists.newArrayList(fieldNames.descendingIterator()).toArray(new String[0]); - } - - protected String[] path(String name) { - List list = Lists.newArrayList(fieldNames.descendingIterator()); - list.add(name); - return list.toArray(new String[0]); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/iceberg/spark/data/AdaptHiveSparkParquetWriters.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/iceberg/spark/data/AdaptHiveSparkParquetWriters.java deleted file mode 100644 index 066aa48ca0..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/java/org/apache/iceberg/spark/data/AdaptHiveSparkParquetWriters.java +++ /dev/null @@ -1,507 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.iceberg.spark.data; - -import static java.util.concurrent.TimeUnit.MICROSECONDS; - -import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.iceberg.parquet.AdaptHivePrimitiveWriter; -import org.apache.iceberg.parquet.ParquetValueReaders; -import org.apache.iceberg.parquet.ParquetValueWriter; -import org.apache.iceberg.parquet.ParquetValueWriters; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.DecimalUtil; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.DecimalMetadata; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.util.ArrayData; -import org.apache.spark.sql.catalyst.util.MapData; -import org.apache.spark.sql.types.ArrayType; -import org.apache.spark.sql.types.ByteType; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.types.MapType; -import org.apache.spark.sql.types.ShortType; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.unsafe.types.UTF8String; - -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.concurrent.TimeUnit; - -/** Copy from iceberg {@link SparkParquetWriters} to support int96 type. */ -public class AdaptHiveSparkParquetWriters { - - private AdaptHiveSparkParquetWriters() {} - - @SuppressWarnings("unchecked") - public static ParquetValueWriter buildWriter(StructType dfSchema, MessageType type) { - return (ParquetValueWriter) - AdaptHiveParquetWithSparkSchemaVisitor.visit(dfSchema, type, new WriteBuilder(type)); - } - - private static class WriteBuilder - extends AdaptHiveParquetWithSparkSchemaVisitor> { - private final MessageType type; - - WriteBuilder(MessageType type) { - this.type = type; - } - - @Override - public ParquetValueWriter message( - StructType struct, MessageType message, List> fieldWriters) { - return struct(struct, message.asGroupType(), fieldWriters); - } - - @Override - public ParquetValueWriter struct( - StructType structType, GroupType struct, List> fieldWriters) { - List fields = struct.getFields(); - StructField[] sparkFields = structType.fields(); - List> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); - List sparkTypes = Lists.newArrayList(); - for (int i = 0; i < fields.size(); i += 1) { - writers.add(newOption(struct.getType(i), fieldWriters.get(i))); - sparkTypes.add(sparkFields[i].dataType()); - } - - return new InternalRowWriter(writers, sparkTypes); - } - - @Override - public ParquetValueWriter list( - ArrayType arrayType, GroupType array, ParquetValueWriter elementWriter) { - GroupType repeated = array.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath); - int repeatedR = type.getMaxRepetitionLevel(repeatedPath); - - return new ArrayDataWriter<>( - repeatedD, - repeatedR, - newOption(repeated.getType(0), elementWriter), - arrayType.elementType()); - } - - @Override - public ParquetValueWriter map( - MapType mapType, - GroupType map, - ParquetValueWriter keyWriter, - ParquetValueWriter valueWriter) { - GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath); - int repeatedR = type.getMaxRepetitionLevel(repeatedPath); - - return new MapDataWriter<>( - repeatedD, - repeatedR, - newOption(repeatedKeyValue.getType(0), keyWriter), - newOption(repeatedKeyValue.getType(1), valueWriter), - mapType.keyType(), - mapType.valueType()); - } - - private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter writer) { - int maxD = type.getMaxDefinitionLevel(path(fieldType.getName())); - return ParquetValueWriters.option(fieldType, maxD, writer); - } - - @Override - public ParquetValueWriter primitive(DataType dataType, PrimitiveType primitive) { - ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return utf8Strings(desc); - case DATE: - case INT_8: - case INT_16: - case INT_32: - return ints(dataType, desc); - case INT_64: - case TIME_MICROS: - case TIMESTAMP_MICROS: - return ParquetValueWriters.longs(desc); - case DECIMAL: - DecimalMetadata decimal = primitive.getDecimalMetadata(); - switch (primitive.getPrimitiveTypeName()) { - case INT32: - return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale()); - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return byteArrays(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } - } - - switch (primitive.getPrimitiveTypeName()) { - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - return byteArrays(desc); - case BOOLEAN: - return ParquetValueWriters.booleans(desc); - case INT32: - return ints(dataType, desc); - case INT64: - return ParquetValueWriters.longs(desc); - case INT96: - return new TimestampInt96Writer(desc); - case FLOAT: - return ParquetValueWriters.floats(desc); - case DOUBLE: - return ParquetValueWriters.doubles(desc); - default: - throw new UnsupportedOperationException("Unsupported type: " + primitive); - } - } - } - - private static class TimestampInt96Writer extends AdaptHivePrimitiveWriter { - - private static final long JULIAN_DAY_OF_EPOCH = 2440588L; - private static final long MICROS_PER_DAY = 86400000000L; - private static final long MILLIS_IN_DAY = TimeUnit.DAYS.toMillis(1); - private static final long NANOS_PER_MILLISECOND = TimeUnit.MILLISECONDS.toNanos(1); - - public TimestampInt96Writer(ColumnDescriptor descriptor) { - super(descriptor); - } - - /** Writes nano timestamps to parquet int96 */ - void writeBinary(int repetitionLevel, int julianDay, long nanosOfDay) { - ByteBuffer buf = ByteBuffer.allocate(12); - buf.order(ByteOrder.LITTLE_ENDIAN); - buf.putLong(nanosOfDay); - buf.putInt(julianDay); - buf.flip(); - column.writeBinary(repetitionLevel, Binary.fromConstantByteBuffer(buf)); - } - - /** - * Writes Julian day and nanoseconds in a day from the number of microseconds - * - * @param value epoch time in microseconds - */ - void writeLong(int repetitionLevel, long value) { - long julianUs = value + JULIAN_DAY_OF_EPOCH * MICROS_PER_DAY; - int day = (int) (julianUs / MICROS_PER_DAY); - long nanos = MICROSECONDS.toNanos(julianUs % MICROS_PER_DAY); - writeBinary(repetitionLevel, day, nanos); - } - - @Override - public void write(int repetitionLevel, Long value) { - writeLong(repetitionLevel, value); - } - } - - private static ParquetValueWriters.PrimitiveWriter ints(DataType type, ColumnDescriptor desc) { - if (type instanceof ByteType) { - return ParquetValueWriters.tinyints(desc); - } else if (type instanceof ShortType) { - return ParquetValueWriters.shorts(desc); - } - return ParquetValueWriters.ints(desc); - } - - private static ParquetValueWriters.PrimitiveWriter utf8Strings( - ColumnDescriptor desc) { - return new UTF8StringWriter(desc); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsInteger( - ColumnDescriptor desc, int precision, int scale) { - return new IntegerDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsLong( - ColumnDescriptor desc, int precision, int scale) { - return new LongDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsFixed( - ColumnDescriptor desc, int precision, int scale) { - return new FixedDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter byteArrays(ColumnDescriptor desc) { - return new ByteArrayWriter(desc); - } - - private static class UTF8StringWriter extends ParquetValueWriters.PrimitiveWriter { - private UTF8StringWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, UTF8String value) { - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(value.getBytes())); - } - } - - private static class IntegerDecimalWriter extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - - private IntegerDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public void write(int repetitionLevel, Decimal decimal) { - Preconditions.checkArgument( - decimal.scale() == scale, - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - decimal); - Preconditions.checkArgument( - decimal.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - decimal); - - column.writeInteger(repetitionLevel, (int) decimal.toUnscaledLong()); - } - } - - private static class LongDecimalWriter extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - - private LongDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public void write(int repetitionLevel, Decimal decimal) { - Preconditions.checkArgument( - decimal.scale() == scale, - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - decimal); - Preconditions.checkArgument( - decimal.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - decimal); - - column.writeLong(repetitionLevel, decimal.toUnscaledLong()); - } - } - - private static class FixedDecimalWriter extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - private final ThreadLocal bytes; - - private FixedDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - this.bytes = - ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); - } - - @Override - public void write(int repetitionLevel, Decimal decimal) { - byte[] binary = - DecimalUtil.toReusedFixLengthBytes( - precision, scale, decimal.toJavaBigDecimal(), bytes.get()); - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(binary)); - } - } - - private static class ByteArrayWriter extends ParquetValueWriters.PrimitiveWriter { - private ByteArrayWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, byte[] bytes) { - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(bytes)); - } - } - - private static class ArrayDataWriter extends ParquetValueWriters.RepeatedWriter { - private final DataType elementType; - - private ArrayDataWriter( - int definitionLevel, - int repetitionLevel, - ParquetValueWriter writer, - DataType elementType) { - super(definitionLevel, repetitionLevel, writer); - this.elementType = elementType; - } - - @Override - protected Iterator elements(ArrayData list) { - return new ElementIterator<>(list); - } - - private class ElementIterator implements Iterator { - private final int size; - private final ArrayData list; - private int index; - - private ElementIterator(ArrayData list) { - this.list = list; - size = list.numElements(); - index = 0; - } - - @Override - public boolean hasNext() { - return index != size; - } - - @Override - @SuppressWarnings("unchecked") - public E next() { - if (index >= size) { - throw new NoSuchElementException(); - } - - E element; - if (list.isNullAt(index)) { - element = null; - } else { - element = (E) list.get(index, elementType); - } - - index += 1; - - return element; - } - } - } - - private static class MapDataWriter - extends ParquetValueWriters.RepeatedKeyValueWriter { - private final DataType keyType; - private final DataType valueType; - - private MapDataWriter( - int definitionLevel, - int repetitionLevel, - ParquetValueWriter keyWriter, - ParquetValueWriter valueWriter, - DataType keyType, - DataType valueType) { - super(definitionLevel, repetitionLevel, keyWriter, valueWriter); - this.keyType = keyType; - this.valueType = valueType; - } - - @Override - protected Iterator> pairs(MapData map) { - return new EntryIterator<>(map); - } - - private class EntryIterator implements Iterator> { - private final int size; - private final ArrayData keys; - private final ArrayData values; - private final ParquetValueReaders.ReusableEntry entry; - private int index; - - private EntryIterator(MapData map) { - size = map.numElements(); - keys = map.keyArray(); - values = map.valueArray(); - entry = new ParquetValueReaders.ReusableEntry<>(); - index = 0; - } - - @Override - public boolean hasNext() { - return index != size; - } - - @Override - @SuppressWarnings("unchecked") - public Map.Entry next() { - if (index >= size) { - throw new NoSuchElementException(); - } - - if (values.isNullAt(index)) { - entry.set((K) keys.get(index, keyType), null); - } else { - entry.set((K) keys.get(index, keyType), (V) values.get(index, valueType)); - } - - index += 1; - - return entry; - } - } - } - - private static class InternalRowWriter extends ParquetValueWriters.StructWriter { - private final DataType[] types; - - private InternalRowWriter(List> writers, List types) { - super(writers); - this.types = types.toArray(new DataType[types.size()]); - } - - @Override - protected Object get(InternalRow struct, int index) { - return struct.get(index, types[index]); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/MixedFormatSparkExtensions.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/MixedFormatSparkExtensions.scala deleted file mode 100644 index f56e88e693..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/MixedFormatSparkExtensions.scala +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark - -import org.apache.spark.sql.SparkSessionExtensions -import org.apache.spark.sql.catalyst.analysis.{AlignedRowLevelIcebergCommandCheck, AlignRowLevelCommandAssignments, CheckMergeIntoTableConditions, MergeIntoIcebergTableResolutionCheck, ProcedureArgumentCoercion, ResolveMergeIntoTableReferences, ResolveProcedures, RewriteDeleteFromTable, RewriteMergeIntoTable, RewriteUpdateTable} -import org.apache.spark.sql.catalyst.optimizer._ -import org.apache.spark.sql.catalyst.parser.extensions.IcebergSparkSqlExtensionsParser -import org.apache.spark.sql.execution.datasources.v2.{ExtendedDataSourceV2Strategy, ExtendedV2Writes, MixedFormatExtendedDataSourceV2Strategy, OptimizeMetadataOnlyDeleteFromTable, ReplaceRewrittenRowLevelCommand, RowLevelCommandScanRelationPushDown} -import org.apache.spark.sql.execution.dynamicpruning.RowLevelCommandDynamicPruning - -import org.apache.amoro.spark.sql.catalyst.analysis._ -import org.apache.amoro.spark.sql.catalyst.optimize.{OptimizeWriteRule, RewriteAppendMixedFormatTable, RewriteDeleteFromMixedFormatTable, RewriteUpdateMixedFormatTable} -import org.apache.amoro.spark.sql.catalyst.parser.MixedFormatSqlExtensionsParser -import org.apache.amoro.spark.sql.execution - -class MixedFormatSparkExtensions extends (SparkSessionExtensions => Unit) { - - override def apply(extensions: SparkSessionExtensions): Unit = { - extensions.injectParser { - case (_, parser) => new MixedFormatSqlExtensionsParser(parser) - } - // resolve mixed-format command - extensions.injectResolutionRule { spark => ResolveMixedFormatCommand(spark) } - extensions.injectResolutionRule { spark => ResolveMergeIntoMixedFormatTableReferences(spark) } - extensions.injectResolutionRule { _ => MixedFormatAlignRowLevelCommandAssignments } - extensions.injectResolutionRule { spark => RewriteMixedFormatMergeIntoTable(spark) } - - extensions.injectPostHocResolutionRule(spark => RewriteMixedFormatCommand(spark)) - - // mixed-format optimizer rules - extensions.injectPostHocResolutionRule { spark => QueryWithConstraintCheck(spark) } - extensions.injectOptimizerRule { spark => RewriteAppendMixedFormatTable(spark) } - extensions.injectOptimizerRule { spark => RewriteDeleteFromMixedFormatTable(spark) } - extensions.injectOptimizerRule { spark => RewriteUpdateMixedFormatTable(spark) } - - // iceberg extensions - extensions.injectResolutionRule { spark => ResolveMergeIntoTableReferences(spark) } - extensions.injectResolutionRule { _ => CheckMergeIntoTableConditions } - extensions.injectResolutionRule { _ => ProcedureArgumentCoercion } - extensions.injectResolutionRule { _ => AlignRowLevelCommandAssignments } - extensions.injectResolutionRule { _ => RewriteDeleteFromTable } - extensions.injectResolutionRule { _ => RewriteUpdateTable } - extensions.injectResolutionRule { _ => RewriteMergeIntoTable } - extensions.injectCheckRule { _ => MergeIntoIcebergTableResolutionCheck } - extensions.injectCheckRule { _ => AlignedRowLevelIcebergCommandCheck } - - extensions.injectOptimizerRule { _ => ExtendedSimplifyConditionalsInPredicate } - extensions.injectOptimizerRule { _ => ExtendedReplaceNullWithFalseInPredicate } - - extensions.injectPreCBORule { _ => OptimizeMetadataOnlyDeleteFromTable } - extensions.injectPreCBORule { _ => RowLevelCommandScanRelationPushDown } - extensions.injectPreCBORule { _ => ExtendedV2Writes } - extensions.injectPreCBORule { spark => RowLevelCommandDynamicPruning(spark) } - extensions.injectPreCBORule { _ => ReplaceRewrittenRowLevelCommand } - - // planner extensions - extensions.injectPlannerStrategy { spark => MixedFormatExtendedDataSourceV2Strategy(spark) } - // mixed-format optimizer rules - extensions.injectPreCBORule(OptimizeWriteRule) - - // mixed-format strategy rules - extensions.injectPlannerStrategy { spark => execution.ExtendedMixedFormatStrategy(spark) } - - // iceberg sql parser extensions - extensions.injectParser { case (_, parser) => new IcebergSparkSqlExtensionsParser(parser) } - - // iceberg procedure analyzer extensions - extensions.injectResolutionRule { spark => ResolveProcedures(spark) } - } - -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/SupportSparkAdapter.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/SupportSparkAdapter.scala deleted file mode 100644 index 78ce5a8901..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/SupportSparkAdapter.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark - -/** - * Use this trait to get SparkAdapter when scala object need to - * adapt between different spark version - */ -trait SupportSparkAdapter { - lazy val sparkAdapter: SparkAdapter = SparkAdapterLoader.getOrLoad() -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/MixedFormatExtensionUtils.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/MixedFormatExtensionUtils.scala deleted file mode 100644 index 7d30fb3d59..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/MixedFormatExtensionUtils.scala +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql - -import scala.collection.JavaConverters.seqAsJavaList - -import org.apache.iceberg.spark.Spark3Util -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.Resolver -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias} -import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, Table, TableCatalog} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} - -import org.apache.amoro.spark.{MixedFormatSparkCatalog, MixedFormatSparkSessionCatalog} -import org.apache.amoro.spark.table.{MixedSparkTable, SupportsRowLevelOperator, UnkeyedSparkTable} - -object MixedFormatExtensionUtils { - - implicit class MixedFormatTableHelper(table: Table) { - def asMixedSparkTable: MixedSparkTable = { - table match { - case table: MixedSparkTable => table - case _ => throw new IllegalArgumentException(s"$table is not an mixed-format table") - } - } - - def asUpsertWrite: SupportsRowLevelOperator = { - table match { - case table: SupportsRowLevelOperator => table - case _ => throw new IllegalArgumentException(s"$table is not an upsert-able table") - } - } - } - - implicit class MixedFormatRelationHelper(plan: LogicalPlan) { - def asTableRelation: DataSourceV2Relation = { - MixedFormatExtensionUtils.asTableRelation(plan) - } - } - - def isMixedFormatRelation(plan: LogicalPlan): Boolean = { - def isMixedFormatTable(relation: DataSourceV2Relation): Boolean = relation.table match { - case _: MixedSparkTable => true - case _ => false - } - - plan.collectLeaves().exists { - case p: DataSourceV2Relation => isMixedFormatTable(p) - case s: SubqueryAlias => s.child.children.exists { case p: DataSourceV2Relation => - isMixedFormatTable(p) - } - case _ => false - } - } - - def isKeyedRelation(plan: LogicalPlan): Boolean = { - def isKeyedTable(relation: DataSourceV2Relation): Boolean = relation.table match { - case a: MixedSparkTable => - a.table().isKeyedTable - case _ => false - } - - plan.collectLeaves().exists { - case p: DataSourceV2Relation => isKeyedTable(p) - case s: SubqueryAlias => s.child.children.exists { case p: DataSourceV2Relation => - isKeyedTable(p) - } - case _ => false - } - } - - def isUpsert(relation: DataSourceV2Relation): Boolean = { - val upsertWrite = relation.table.asUpsertWrite - upsertWrite.appendAsUpsert() - } - - def isUnkeyedTableRelation(plan: LogicalPlan): Boolean = { - def isIcebergTable(relation: DataSourceV2Relation): Boolean = relation.table match { - case _: UnkeyedSparkTable => true - case _ => false - } - - plan.collectLeaves().exists { - case p: DataSourceV2Relation => isIcebergTable(p) - case s: SubqueryAlias => s.child.children.exists { - case p: DataSourceV2Relation => isIcebergTable(p) - } - } - } - - def isMixedFormatCatalog(catalog: TableCatalog): Boolean = { - catalog match { - case _: MixedFormatSparkCatalog => true - case _: MixedFormatSparkSessionCatalog[_] => true - case _ => false - } - } - - def isMixedFormatCatalog(catalog: CatalogPlugin): Boolean = { - catalog match { - case _: MixedFormatSparkCatalog => true - case _: MixedFormatSparkSessionCatalog[_] => true - case _ => false - } - } - - def isMixedFormatTable(table: Table): Boolean = table match { - case _: MixedSparkTable => true - case _: UnkeyedSparkTable => true - case _ => false - } - - def asTableRelation(plan: LogicalPlan): DataSourceV2Relation = { - plan match { - case s: SubqueryAlias => asTableRelation(s.child) - case p: Project => asTableRelation(p.child.children.head) - case r: DataSourceV2Relation => r - case _ => throw new IllegalArgumentException("Expected a DataSourceV2Relation") - } - } - - def isKeyedTable(relation: DataSourceV2Relation): Boolean = { - relation.table match { - case mixedTable: MixedSparkTable => - mixedTable.table().isKeyedTable - case _ => false - } - } - - def buildCatalogAndIdentifier( - sparkSession: SparkSession, - originIdentifier: TableIdentifier): (TableCatalog, Identifier) = { - var identifier: Seq[String] = Seq.empty[String] - identifier :+= originIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase) - identifier :+= originIdentifier.table - val catalogAndIdentifier = - Spark3Util.catalogAndIdentifier(sparkSession, seqAsJavaList(identifier)) - catalogAndIdentifier.catalog() match { - case a: TableCatalog => (a, catalogAndIdentifier.identifier()) - case _ => - throw new UnsupportedOperationException("Only support TableCatalog or its implementation") - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/MixedFormatAlignRowLevelCommandAssignments.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/MixedFormatAlignRowLevelCommandAssignments.scala deleted file mode 100644 index 7a1926ee57..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/MixedFormatAlignRowLevelCommandAssignments.scala +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.analysis - -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.MixedFormatAssignmentAlignmentSupport -import org.apache.spark.sql.catalyst.expressions.AssignmentUtils -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule - -import org.apache.amoro.spark.sql.catalyst.plans.{MergeIntoMixedFormatTable, UnresolvedMergeIntoMixedFormatTable} - -/** - * A rule that aligns assignments in UPDATE and MERGE operations. - * - * Note that this rule must be run before rewriting row-level commands. - */ -object MixedFormatAlignRowLevelCommandAssignments - extends Rule[LogicalPlan] with MixedFormatAssignmentAlignmentSupport { - - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case m: MergeIntoMixedFormatTable if m.resolved && !m.aligned => - val alignedMatchedActions = m.matchedActions.map { - case u @ UpdateAction(_, assignments) => - u.copy(assignments = alignAssignments(m.targetTable, assignments)) - case d: DeleteAction => - d - case _ => - throw new AnalysisException( - "Matched actions can only contain UPDATE or DELETE", - Array.empty[String]) - } - - val alignedNotMatchedActions = m.notMatchedActions.map { - case i @ InsertAction(_, assignments) => - // check no nested columns are present - val refs = assignments.map(_.key).map(AssignmentUtils.toAssignmentRef) - refs.foreach { ref => - if (ref.size > 1) { - throw new AnalysisException( - "Nested fields are not supported inside INSERT clauses of MERGE operations: " + - s"${ref.mkString("`", "`.`", "`")}", - Array.empty[String]) - } - } - - val colNames = refs.map(_.head) - - // check there are no duplicates - val duplicateColNames = colNames.groupBy(identity).collect { - case (name, matchingNames) if matchingNames.size > 1 => name - } - - if (duplicateColNames.nonEmpty) { - throw new AnalysisException( - s"Duplicate column names inside INSERT clause: ${duplicateColNames.mkString(", ")}", - Array.empty[String]) - } - - // reorder assignments by the target table column order - val assignmentMap = colNames.zip(assignments).toMap - i.copy(assignments = alignInsertActionAssignments(m.targetTable, assignmentMap)) - - case _ => - throw new AnalysisException( - "Not matched actions can only contain INSERT", - Array.empty[String]) - } - - m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions) - } - - private def alignInsertActionAssignments( - targetTable: LogicalPlan, - assignmentMap: Map[String, Assignment]): Seq[Assignment] = { - - val resolver = conf.resolver - - targetTable.output.map { targetAttr => - val assignment = assignmentMap - .find { case (name, _) => resolver(name, targetAttr.name) } - .map { case (_, assignment) => assignment } - - if (assignment.isEmpty) { - throw new AnalysisException( - s"Cannot find column '${targetAttr.name}' of the target table among " + - s"the INSERT columns: ${assignmentMap.keys.mkString(", ")}. " + - "INSERT clauses must provide values for all columns of the target table.", - Array.empty[String]) - } - - val key = assignment.get.key - val value = castIfNeeded(targetAttr, assignment.get.value, resolver) - AssignmentUtils.handleCharVarcharLimits(Assignment(key, value)) - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/QueryWithConstraintCheck.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/QueryWithConstraintCheck.scala deleted file mode 100644 index 653dc9f3ea..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/QueryWithConstraintCheck.scala +++ /dev/null @@ -1,130 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.analysis - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, EqualNullSafe, Expression, GreaterThan, Literal} -import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, Count} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogPlugin, TableCatalog} -import org.apache.spark.sql.execution.datasources.DataSourceAnalysis.resolver -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation - -import org.apache.amoro.spark.{MixedFormatSparkCatalog, MixedFormatSparkSessionCatalog} -import org.apache.amoro.spark.mixed.SparkSQLProperties -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils.isKeyedRelation -import org.apache.amoro.spark.sql.catalyst.plans.QueryWithConstraintCheckPlan -import org.apache.amoro.spark.table.MixedSparkTable - -case class QueryWithConstraintCheck(spark: SparkSession) extends Rule[LogicalPlan] { - - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp { - case a @ AppendData(r: DataSourceV2Relation, query, _, _, _) - if checkDuplicatesEnabled() && isKeyedRelation(r) => - val validateQuery = buildValidatePrimaryKeyDuplication(r, query) - val checkDataQuery = QueryWithConstraintCheckPlan(query, validateQuery) - a.copy(query = checkDataQuery) - - case a @ OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _, _, _) - if checkDuplicatesEnabled() && isKeyedRelation(r) => - val validateQuery = buildValidatePrimaryKeyDuplication(r, query) - val checkDataQuery = QueryWithConstraintCheckPlan(query, validateQuery) - a.copy(query = checkDataQuery) - - case a @ OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, _, _, _) - if checkDuplicatesEnabled() && isKeyedRelation(r) => - val validateQuery = buildValidatePrimaryKeyDuplication(r, query) - var finalExpr: Expression = deleteExpr - deleteExpr match { - case expr: EqualNullSafe => - finalExpr = expr.copy(query.output.last, expr.right) - case _ => - } - val checkDataQuery = QueryWithConstraintCheckPlan(query, validateQuery) - a.copy(query = checkDataQuery) - - case c @ CreateTableAsSelect(catalog, _, _, query, props, _, _) - if checkDuplicatesEnabled() && isCreateKeyedTable(catalog, props) => - val primaries = props("primary.keys").split(",") - val validateQuery = buildValidatePrimaryKeyDuplicationByPrimaries(primaries, query) - val checkDataQuery = QueryWithConstraintCheckPlan(query, validateQuery) - c.copy(query = checkDataQuery) - } - - def checkDuplicatesEnabled(): Boolean = { - java.lang.Boolean.valueOf(spark.sessionState.conf.getConfString( - SparkSQLProperties.CHECK_SOURCE_DUPLICATES_ENABLE, - SparkSQLProperties.CHECK_SOURCE_DUPLICATES_ENABLE_DEFAULT)) - } - - def isCreateKeyedTable(catalog: TableCatalog, props: Map[String, String]): Boolean = { - catalog match { - case _: MixedFormatSparkCatalog => - props.contains("primary.keys") - case _: MixedFormatSparkSessionCatalog[_] => - props("provider").equalsIgnoreCase("arctic") && props.contains("primary.keys") - case _ => - false - } - } - - def buildValidatePrimaryKeyDuplication( - r: DataSourceV2Relation, - query: LogicalPlan): LogicalPlan = { - r.table match { - case mixedTable: MixedSparkTable => - if (mixedTable.table().isKeyedTable) { - val primaries = mixedTable.table().asKeyedTable().primaryKeySpec().fieldNames() - val attributes = query.output.filter(p => primaries.contains(p.name)) - val aggSumCol = Alias( - AggregateExpression(Count(Literal(1)), Complete, isDistinct = false), - SUM_ROW_ID_ALIAS_NAME)() - val aggPlan = Aggregate(attributes, Seq(aggSumCol), query) - val sumAttr = findOutputAttr(aggPlan.output, SUM_ROW_ID_ALIAS_NAME) - val havingExpr = GreaterThan(sumAttr, Literal(1L)) - Filter(havingExpr, aggPlan) - } else { - throw new UnsupportedOperationException(s"UnKeyed table can not validate") - } - } - } - - def buildValidatePrimaryKeyDuplicationByPrimaries( - primaries: Array[String], - query: LogicalPlan): LogicalPlan = { - val attributes = query.output.filter(p => primaries.contains(p.name)) - val aggSumCol = Alias( - AggregateExpression(Count(Literal(1)), Complete, isDistinct = false), - SUM_ROW_ID_ALIAS_NAME)() - val aggPlan = Aggregate(attributes, Seq(aggSumCol), query) - val sumAttr = findOutputAttr(aggPlan.output, SUM_ROW_ID_ALIAS_NAME) - val havingExpr = GreaterThan(sumAttr, Literal(1L)) - Filter(havingExpr, aggPlan) - } - - protected def findOutputAttr(attrs: Seq[Attribute], attrName: String): Attribute = { - attrs.find(attr => resolver(attr.name, attrName)).getOrElse { - throw new UnsupportedOperationException(s"Cannot find $attrName in $attrs") - } - } - - final private val SUM_ROW_ID_ALIAS_NAME = "_sum_" - -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/ResolveMergeIntoMixedFormatTableReferences.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/ResolveMergeIntoMixedFormatTableReferences.scala deleted file mode 100644 index c700e869c7..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/ResolveMergeIntoMixedFormatTableReferences.scala +++ /dev/null @@ -1,281 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.analysis - -import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.analysis.{caseInsensitiveResolution, withPosition, AnalysisErrorAt, EliminateSubqueryAliases, GetColumnByOrdinal, Resolver, UnresolvedAttribute, UnresolvedExtractValue} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentDate, CurrentTimestamp, Expression, ExtractValue, LambdaFunction} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin -import org.apache.spark.sql.catalyst.util.toPrettySQL -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation - -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils.isMixedFormatRelation -import org.apache.amoro.spark.sql.catalyst.plans.{MergeIntoMixedFormatTable, UnresolvedMergeIntoMixedFormatTable} -import org.apache.amoro.spark.table.MixedSparkTable - -case class ResolveMergeIntoMixedFormatTableReferences(spark: SparkSession) - extends Rule[LogicalPlan] { - - def checkConditionIsPrimaryKey(aliasedTable: LogicalPlan, cond: Expression): Unit = { - EliminateSubqueryAliases(aliasedTable) match { - case r @ DataSourceV2Relation(tbl, _, _, _, _) if isMixedFormatRelation(r) => - tbl match { - case mixedSparkTable: MixedSparkTable => - if (mixedSparkTable.table().isKeyedTable) { - val primaryKeys = mixedSparkTable.table().asKeyedTable().primaryKeySpec().fieldNames() - val attributes = aliasedTable.output.filter(p => primaryKeys.contains(p.name)) - val condRefs = cond.references.filter(f => attributes.contains(f)) - if (condRefs.isEmpty) { - throw new UnsupportedOperationException( - s"Condition ${cond.references}. is not allowed because is not a primary key") - } - } - } - case p => - throw new UnsupportedOperationException(s"$p is not an mixed-format table") - } - } - - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp { - case m @ UnresolvedMergeIntoMixedFormatTable( - aliasedTable, - source, - cond, - matchedActions, - notMatchedActions) => - checkConditionIsPrimaryKey(aliasedTable, cond) - - val resolvedMatchedActions = matchedActions.map { - case DeleteAction(cond) => - val resolvedCond = cond.map(resolveCond("DELETE", _, m)) - DeleteAction(resolvedCond) - - case UpdateAction(cond, assignments) => - val resolvedUpdateCondition = cond.map(resolveCond("UPDATE", _, m)) - // for UPDATE *, the value must be from the source table - val resolvedAssignments = - resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true) - UpdateAction(resolvedUpdateCondition, resolvedAssignments) - - case UpdateStarAction(updateCondition) => - val resolvedUpdateCondition = updateCondition.map(resolveCond("UPDATE", _, m)) - val assignments = aliasedTable.output.map { attr => - Assignment(attr, UnresolvedAttribute(Seq(attr.name))) - } - // for UPDATE *, the value must be from the source table - val resolvedAssignments = - resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true) - UpdateAction(resolvedUpdateCondition, resolvedAssignments) - - case _ => - throw new UnsupportedOperationException( - "Matched actions can only contain UPDATE or DELETE") - } - - val resolvedNotMatchedActions = notMatchedActions.map { - case InsertAction(cond, assignments) => - val resolvedCond = cond.map(resolveCond("INSERT", _, Project(Nil, m.sourceTable))) - val resolvedAssignments = - resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true) - InsertAction(resolvedCond, resolvedAssignments) - case InsertStarAction(cond) => - // the insert action is used when not matched, so its condition and value can only - // access columns from the source table - val resolvedCond = cond.map(resolveCond("INSERT", _, Project(Nil, m.sourceTable))) - val assignments = aliasedTable.output.map { attr => - Assignment(attr, UnresolvedAttribute(Seq(attr.name))) - } - val resolvedAssignments = - resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true) - InsertAction(resolvedCond, resolvedAssignments) - - case _ => - throw new UnsupportedOperationException("Not matched actions can only contain INSERT") - } - - val resolvedMergeCondition = resolveCond("SEARCH", cond, m) - - MergeIntoMixedFormatTable( - aliasedTable, - source, - mergeCondition = resolvedMergeCondition, - matchedActions = resolvedMatchedActions, - notMatchedActions = resolvedNotMatchedActions) - } - - private def resolveLiteralFunction( - nameParts: Seq[String], - attribute: UnresolvedAttribute, - plan: LogicalPlan): Option[Expression] = { - if (nameParts.length != 1) return None - val isNamedExpression = plan match { - case Aggregate(_, aggregateExpressions, _) => aggregateExpressions.contains(attribute) - case Project(projectList, _) => projectList.contains(attribute) - case Window(windowExpressions, _, _, _) => windowExpressions.contains(attribute) - case _ => false - } - val wrapper: Expression => Expression = - if (isNamedExpression) f => Alias(f, toPrettySQL(f))() else identity - // support CURRENT_DATE and CURRENT_TIMESTAMP - val literalFunctions = Seq(CurrentDate(), CurrentTimestamp()) - val name = nameParts.head - val func = literalFunctions.find(e => caseInsensitiveResolution(e.prettyName, name)) - func.map(wrapper) - } - - def resolveExpressionBottomUp( - expr: Expression, - plan: LogicalPlan, - throws: Boolean = false): Expression = { - if (expr.resolved) return expr - try { - expr transformUp { - case GetColumnByOrdinal(ordinal, _) => plan.output(ordinal) - case u @ UnresolvedAttribute(nameParts) => - val result = - withPosition(u) { - plan.resolve(nameParts, resolver) - .orElse(resolveLiteralFunction(nameParts, u, plan)) - .getOrElse(u) - } - logDebug(s"Resolving $u to $result") - result - case UnresolvedExtractValue(child, fieldName) if child.resolved => - ExtractValue(child, fieldName, resolver) - } - } catch { - case a: AnalysisException if !throws => expr - } - } - - private def resolveCond(condName: String, cond: Expression, plan: LogicalPlan): Expression = { - val resolvedCond = resolveExpressionBottomUp(cond, plan) - - val unresolvedAttrs = resolvedCond.references.filter(!_.resolved) - if (unresolvedAttrs.nonEmpty) { - throw new UnsupportedOperationException( - s"Cannot resolve ${unresolvedAttrs.map(_.sql).mkString("[", ",", "]")} in $condName condition " + - s"of MERGE operation given input columns: ${plan.inputSet.toSeq.map(_.sql).mkString("[", ",", "]")}") - } - - resolvedCond - } - - def resolver: Resolver = conf.resolver - - def resolveExpressionByPlanChildren( - e: Expression, - q: LogicalPlan): Expression = { - resolveExpression( - e, - resolveColumnByName = nameParts => { - q.resolveChildren(nameParts, resolver) - }, - getAttrCandidates = () => { - assert(q.children.length == 1) - q.children.head.output - }, - throws = true) - } - - private def resolveExpression( - expr: Expression, - resolveColumnByName: Seq[String] => Option[Expression], - getAttrCandidates: () => Seq[Attribute], - throws: Boolean): Expression = { - def innerResolve(e: Expression, isTopLevel: Boolean): Expression = { - if (e.resolved) return e - e match { - case f: LambdaFunction if !f.bound => f - - case GetColumnByOrdinal(ordinal, _) => - val attrCandidates = getAttrCandidates() - assert(ordinal >= 0 && ordinal < attrCandidates.length) - attrCandidates(ordinal) - - case u @ UnresolvedAttribute(nameParts) => - val result = withPosition(u) { - resolveColumnByName(nameParts).map { - case Alias(child, _) if !isTopLevel => child - case other => other - }.getOrElse(u) - } - logDebug(s"Resolving $u to $result") - result - - case u @ UnresolvedExtractValue(child, fieldName) => - val newChild = innerResolve(child, isTopLevel = false) - if (newChild.resolved) { - withOrigin(u.origin) { - ExtractValue(newChild, fieldName, resolver) - } - } else { - u.copy(child = newChild) - } - - case _ => e.mapChildren(innerResolve(_, isTopLevel = false)) - } - } - - try { - innerResolve(expr, isTopLevel = true) - } catch { - case _: AnalysisException if !throws => expr - } - } - - // copied from ResolveReferences in Spark - private def resolveAssignments( - assignments: Seq[Assignment], - mergeInto: UnresolvedMergeIntoMixedFormatTable, - resolveValuesWithSourceOnly: Boolean): Seq[Assignment] = { - assignments.map { assign => - val resolvedKey = assign.key match { - case c if !c.resolved => - resolveMergeExprOrFail(c, Project(Nil, mergeInto.targetTable)) - case o => o - } - val resolvedValue = assign.value match { - // The update values may contain target and/or source references. - case c if !c.resolved => - if (resolveValuesWithSourceOnly) { - resolveMergeExprOrFail(c, Project(Nil, mergeInto.sourceTable)) - } else { - resolveMergeExprOrFail(c, mergeInto) - } - case o => o - } - Assignment(resolvedKey, resolvedValue) - } - } - - // copied from ResolveReferences in Spark - private def resolveMergeExprOrFail(e: Expression, p: LogicalPlan): Expression = { - val resolved = resolveExpressionByPlanChildren(e, p) - resolved.references.filter(!_.resolved).foreach { a => - // Note: This will throw error only on unresolved attribute issues, - // not other resolution errors like mismatched data types. - val cols = p.inputSet.toSeq.map(_.sql).mkString(", ") - a.failAnalysis(s"cannot resolve ${a.sql} in MERGE command given columns [$cols]") - } - resolved - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/ResolveMixedFormatCommand.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/ResolveMixedFormatCommand.scala deleted file mode 100644 index 5d07631bab..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/ResolveMixedFormatCommand.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.analysis - -import scala.collection.JavaConverters.seqAsJavaList - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.rules.Rule - -import org.apache.amoro.spark.command.MigrateToMixedFormatCommand -import org.apache.amoro.spark.sql.catalyst.plans - -case class ResolveMixedFormatCommand(spark: SparkSession) extends Rule[LogicalPlan] { - - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case plans.MigrateToMixedFormatStatement(source, target) => - val command = MigrateToMixedFormatCommand.newBuilder(spark) - .withSource(seqAsJavaList(source)) - .withTarget(seqAsJavaList(target)) - .build() - plans.MigrateToMixedFormatLogicalPlan(command) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/RewriteMixedFormatCommand.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/RewriteMixedFormatCommand.scala deleted file mode 100644 index e91c31e34d..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/RewriteMixedFormatCommand.scala +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.analysis - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.ResolvedTable -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.TableCatalog -import org.apache.spark.sql.execution.command.CreateTableLikeCommand - -import org.apache.amoro.spark.{MixedFormatSparkCatalog, MixedFormatSparkSessionCatalog} -import org.apache.amoro.spark.mixed.MixedSessionCatalogBase -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils.buildCatalogAndIdentifier -import org.apache.amoro.spark.sql.catalyst.plans.{AlterMixedFormatTableDropPartition, TruncateMixedFormatTable} -import org.apache.amoro.spark.table.MixedSparkTable -import org.apache.amoro.spark.writer.WriteMode - -/** - * Rule for rewrite some spark commands to mixed-format's implementation. - */ -case class RewriteMixedFormatCommand(sparkSession: SparkSession) extends Rule[LogicalPlan] { - - private def isCreateMixedFormatTableLikeCommand( - targetTable: TableIdentifier, - provider: Option[String]): Boolean = { - val (targetCatalog, _) = buildCatalogAndIdentifier(sparkSession, targetTable) - isCreateMixedFormatTable(targetCatalog, provider) - } - - private def isCreateMixedFormatTable(catalog: TableCatalog, provider: Option[String]): Boolean = { - catalog match { - case _: MixedFormatSparkCatalog => true - case _: MixedFormatSparkSessionCatalog[_] => - provider.isDefined && MixedSessionCatalogBase.SUPPORTED_PROVIDERS.contains( - provider.get.toLowerCase) - case _ => false - } - } - - override def apply(plan: LogicalPlan): LogicalPlan = { - import org.apache.amoro.spark.sql.MixedFormatExtensionUtils._ - plan match { - // Rewrite the AlterTableDropPartition to AlterMixedFormatTableDropPartition - case DropPartitions(r: ResolvedTable, parts, ifExists, purge) - if isMixedFormatTable(r.table) => - AlterMixedFormatTableDropPartition(r, parts, ifExists, purge) - case t @ TruncateTable(r: ResolvedTable) - if isMixedFormatTable(r.table) => - TruncateMixedFormatTable(t.child) - - case c @ CreateTableAsSelect(catalog, _, _, _, props, options, _) - if isCreateMixedFormatTable(catalog, props.get(TableCatalog.PROP_PROVIDER)) => - var propertiesMap: Map[String, String] = props - var optionsMap: Map[String, String] = options - if (options.contains("primary.keys")) { - propertiesMap += ("primary.keys" -> options("primary.keys")) - } - optionsMap += (WriteMode.WRITE_MODE_KEY -> WriteMode.OVERWRITE_DYNAMIC.mode) - c.copy(properties = propertiesMap, writeOptions = optionsMap) - case c @ CreateTableLikeCommand( - targetTable, - sourceTable, - storage, - provider, - properties, - ifNotExists) - if isCreateMixedFormatTableLikeCommand(targetTable, provider) => { - val (sourceCatalog, sourceIdentifier) = buildCatalogAndIdentifier(sparkSession, sourceTable) - val (targetCatalog, targetIdentifier) = buildCatalogAndIdentifier(sparkSession, targetTable) - val table = sourceCatalog.loadTable(sourceIdentifier) - var targetProperties = properties - table match { - case mixedSparkTable: MixedSparkTable if mixedSparkTable.table().isKeyedTable => - targetProperties += ("primary.keys" -> - String.join( - ",", - mixedSparkTable.table().asKeyedTable().primaryKeySpec().fieldNames())) - case _ => - } - targetProperties += ("provider" -> "arctic") - CreateV2Table( - targetCatalog, - targetIdentifier, - table.schema(), - table.partitioning(), - targetProperties, - ifNotExists) - } - case _ => plan - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/RewriteMixedFormatMergeIntoTable.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/RewriteMixedFormatMergeIntoTable.scala deleted file mode 100644 index 94d6722366..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/analysis/RewriteMixedFormatMergeIntoTable.scala +++ /dev/null @@ -1,328 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.analysis - -import scala.collection.{mutable, Seq} - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.amoro.catalyst.{ExpressionHelper, MixedFormatSpark32Helper} -import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Expression, ExprId, IsNotNull, Literal} -import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral -import org.apache.spark.sql.catalyst.plans.{Inner, RightOuter} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.Table -import org.apache.spark.sql.connector.expressions.NamedReference -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} -import org.apache.spark.sql.types.{IntegerType, StructType} - -import org.apache.amoro.spark.mixed.SparkSQLProperties -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils.isMixedFormatRelation -import org.apache.amoro.spark.sql.catalyst.plans -import org.apache.amoro.spark.sql.catalyst.plans.{MergeIntoMixedFormatTable, MergeRows, MixedFormatRowLevelWrite} -import org.apache.amoro.spark.sql.utils.{FieldReference, ProjectingInternalRow, WriteQueryProjections} -import org.apache.amoro.spark.sql.utils.RowDeltaUtils.{DELETE_OPERATION, INSERT_OPERATION, OPERATION_COLUMN, UPDATE_OPERATION} -import org.apache.amoro.spark.table.MixedSparkTable -import org.apache.amoro.spark.writer.WriteMode - -case class RewriteMixedFormatMergeIntoTable(spark: SparkSession) extends Rule[LogicalPlan] { - - final private val ROW_FROM_SOURCE = "__row_from_source" - final private val ROW_FROM_TARGET = "__row_from_target" - - final private val ROW_FROM_SOURCE_REF = FieldReference(ROW_FROM_SOURCE) - final private val ROW_FROM_TARGET_REF = FieldReference(ROW_FROM_TARGET) - - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case MergeIntoMixedFormatTable( - aliasedTable, - source, - cond, - matchedActions, - notMatchedActions, - None) => - EliminateSubqueryAliases(aliasedTable) match { - case r @ DataSourceV2Relation(tbl, _, _, _, _) if isMixedFormatRelation(r) => - val rewritePlan = - buildRowLevelWritePlan(r, tbl, source, cond, matchedActions, notMatchedActions) - - rewritePlan - - case p => - throw new UnsupportedOperationException(s"$p is not an mixed-format table") - } - } - - def buildRelationAndAttrs( - relation: DataSourceV2Relation, - cond: Expression, - operationTable: Table): (Seq[Attribute], LogicalPlan) = { - relation.table match { - case mixedSparkTable: MixedSparkTable => - if (mixedSparkTable.table().isKeyedTable) { - val keyAttrs = { - val primarys = mixedSparkTable.table().asKeyedTable().primaryKeySpec().fieldNames() - cond.references.filter(p => primarys.contains(p.name)).toSeq - } - val attrs = dedupAttrs(relation.output) - (keyAttrs, relation.copy(table = operationTable, output = attrs)) - } else { - val (keyAttrs, valuesRelation) = { - if (mixedSparkTable.requireAdditionIdentifierColumns()) { - val scanBuilder = mixedSparkTable.newUpsertScanBuilder(relation.options) - scanBuilder.withIdentifierColumns() - val scan = scanBuilder.build() - val outputAttr = toOutputAttrs(scan.readSchema(), relation.output) - val valuesRelation = DataSourceV2ScanRelation(relation, scan, outputAttr) - val references = cond.references.toSeq - (references, valuesRelation) - } else { - throw new UnsupportedOperationException( - s"Can not build relation and keyAttrs for table $mixedSparkTable") - } - } - (keyAttrs, valuesRelation) - } - } - } - - protected def toOutputAttrs( - schema: StructType, - attrs: Seq[AttributeReference]): Seq[AttributeReference] = { - val nameToAttr = attrs.map(_.name).zip(attrs).toMap - schema.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()).map { - a => - nameToAttr.get(a.name) match { - case Some(ref) => - // keep the attribute id if it was present in the relation - a.withExprId(ref.exprId) - case _ => - // if the field is new, create a new attribute - AttributeReference(a.name, a.dataType, a.nullable, a.metadata)() - } - } - } - - def buildWriteQueryProjections( - plan: MergeRows, - source: LogicalPlan, - targetRowAttrs: Seq[AttributeReference], - rowIdAttrs: Seq[Attribute], - isKeyedTable: Boolean): WriteQueryProjections = { - val (frontRowProjection, backRowProjection) = if (isKeyedTable) { - val frontRowProjection = - Some(ProjectingInternalRow.newProjectInternalRow(plan, targetRowAttrs, isFront = true, 0)) - val backRowProjection = - ProjectingInternalRow.newProjectInternalRow( - source, - targetRowAttrs, - isFront = false, - 1 + rowIdAttrs.size) - (frontRowProjection, backRowProjection) - } else { - val frontRowProjection = - Some(ProjectingInternalRow.newProjectInternalRow( - plan, - targetRowAttrs ++ rowIdAttrs, - isFront = true, - 0)) - val backRowProjection = - ProjectingInternalRow.newProjectInternalRow( - source, - targetRowAttrs, - isFront = false, - 1 + rowIdAttrs.size) - (frontRowProjection, backRowProjection) - } - WriteQueryProjections(frontRowProjection, backRowProjection) - } - - def buildRowIdAttrs(relation: LogicalPlan): Seq[Attribute] = { - val attributes = relation.output.filter(r => r.name.equals("_file") || r.name.equals("_pos")) - attributes - } - - // build a rewrite plan for sources that support row deltas - private def buildRowLevelWritePlan( - relation: DataSourceV2Relation, - operationTable: Table, - source: LogicalPlan, - cond: Expression, - matchedActions: Seq[MergeAction], - notMatchedActions: Seq[MergeAction]): MixedFormatRowLevelWrite = { - // construct a scan relation and include all required metadata columns - val rowAttrs = relation.output - val (keyAttrs, readRelation) = buildRelationAndAttrs(relation, cond, operationTable) - val rowIdAttrs = buildRowIdAttrs(readRelation) - val readAttrs = readRelation.output - - // project an extra column to check if a target row exists after the join - val targetTableProjExprs = readAttrs :+ Alias(TrueLiteral, ROW_FROM_TARGET)() - val targetTableProj = Project(targetTableProjExprs, readRelation) - - // project an extra column to check if a source row exists after the join - val sourceTableProjExprs = source.output :+ Alias(TrueLiteral, ROW_FROM_SOURCE)() - val sourceTableProj = Project(sourceTableProjExprs, source) - - // use inner join if there is no NOT MATCHED action, unmatched source rows can be discarded - // use right outer join in all other cases, unmatched source rows may be needed - // also disable broadcasts for the target table to perform the cardinality check - val joinType = if (notMatchedActions.isEmpty) Inner else RightOuter - val joinHint = JoinHint(leftHint = Some(HintInfo(Some(NO_BROADCAST_HASH))), rightHint = None) - val joinPlan = Join(targetTableProj, sourceTableProj, joinType, Some(cond), joinHint) - - val matchedConditions = matchedActions.map(actionCondition) - val matchedOutputs = - matchedActions.map(rowLevelWriteOutput(_, readRelation.output, source.output)) - - val notMatchedConditions = notMatchedActions.map(actionCondition) - val notMatchedOutputs = - notMatchedActions.map(rowLevelWriteOutput(_, readRelation.output, source.output)) - - val operationTypeAttr = AttributeReference(OPERATION_COLUMN, IntegerType, nullable = false)() - val rowFromSourceAttr = resolveAttrRef(ROW_FROM_SOURCE_REF, joinPlan) - val rowFromTargetAttr = resolveAttrRef(ROW_FROM_TARGET_REF, joinPlan) - - // merged rows must contain values for the operation type and all read attrs - val mergeRowsOutput = - buildMergeRowsOutput(matchedOutputs, notMatchedOutputs, operationTypeAttr +: readAttrs) - - val unMatchedRowNeedCheck = java.lang.Boolean.valueOf(spark.sessionState.conf.getConfString( - SparkSQLProperties.CHECK_SOURCE_DUPLICATES_ENABLE, - SparkSQLProperties.CHECK_SOURCE_DUPLICATES_ENABLE_DEFAULT)) && notMatchedOutputs.nonEmpty && - MixedFormatExtensionUtils.isKeyedTable(relation) - - val mergeRows = plans.MergeRows( - isSourceRowPresent = IsNotNull(rowFromSourceAttr), - isTargetRowPresent = - if (notMatchedActions.isEmpty) TrueLiteral else IsNotNull(rowFromTargetAttr), - matchedConditions = matchedConditions, - matchedOutputs = matchedOutputs, - notMatchedConditions = notMatchedConditions, - notMatchedOutputs = notMatchedOutputs, - rowIdAttrs = keyAttrs, - matchedRowCheck = isMatchedRowCheckNeeded(matchedActions), - unMatchedRowCheck = unMatchedRowNeedCheck, - emitNotMatchedTargetRows = false, - output = mergeRowsOutput, - joinPlan) - - // build a plan to write the row delta to the table - val writeRelation = relation.copy(table = operationTable) - var options: Map[String, String] = Map.empty - options += (WriteMode.WRITE_MODE_KEY -> WriteMode.UPSERT.toString) - val projections = buildWriteQueryProjections( - mergeRows, - source, - rowAttrs, - rowIdAttrs, - MixedFormatExtensionUtils.isKeyedTable(relation)) - val writeBuilder = - MixedFormatSpark32Helper.newWriteBuilder(relation.table, mergeRows.schema, options) - val write = writeBuilder.build() - MixedFormatRowLevelWrite(writeRelation, mergeRows, options, projections, Some(write)) - } - - private def actionCondition(action: MergeAction): Expression = { - action.condition.getOrElse(TrueLiteral) - } - - def dedupAttrs(attrs: Seq[AttributeReference]): Seq[AttributeReference] = { - val exprIds = mutable.Set.empty[ExprId] - attrs.flatMap { attr => - if (exprIds.contains(attr.exprId)) { - None - } else { - exprIds += attr.exprId - Some(attr) - } - } - } - - private def rowLevelWriteOutput( - action: MergeAction, - targetOutput: Seq[Expression], - sourceOutput: Seq[Attribute]): Seq[Expression] = { - - action match { - case u: UpdateAction => - val finalSourceOutput = rebuildAttribute(sourceOutput, u.assignments) - Seq(Literal(UPDATE_OPERATION)) ++ targetOutput ++ finalSourceOutput - - case _: DeleteAction => - Seq(Literal(DELETE_OPERATION)) ++ targetOutput ++ sourceOutput - - case i: InsertAction => - val finalSourceOutput = rebuildAttribute(sourceOutput, i.assignments) - Seq(Literal(INSERT_OPERATION)) ++ targetOutput ++ finalSourceOutput - - case other => - throw new UnsupportedOperationException(s"Unexpected action: $other") - } - } - - private def rebuildAttribute( - sourceOutput: Seq[Attribute], - assignments: Seq[Assignment]): Seq[Expression] = { - val expressions = sourceOutput.map(v => { - val assignment = assignments.find(f => { - f.key match { - case a: Attribute => - a.name.equals(v.name) - } - }) - if (assignment.isEmpty) { - v - } else { - assignment.get.value - } - }) - expressions - } - - private def buildMergeRowsOutput( - matchedOutputs: Seq[Seq[Expression]], - notMatchedOutputs: Seq[Seq[Expression]], - attrs: Seq[Attribute]): Seq[Attribute] = { - - // collect all outputs from matched and not matched actions (ignoring DELETEs) - val outputs = matchedOutputs.filter(_.nonEmpty) ++ notMatchedOutputs.filter(_.nonEmpty) - - // build a correct nullability map for output attributes - // an attribute is nullable if at least one matched or not matched action may produce null - val nullabilityMap = attrs.indices.map { index => - index -> outputs.exists(output => output(index).nullable) - }.toMap - - attrs.zipWithIndex.map { case (attr, index) => - attr.withNullability(nullabilityMap(index)) - } - } - - private def isMatchedRowCheckNeeded(actions: Seq[MergeAction]): Boolean = actions match { - case Seq(DeleteAction(None)) => false - case _ => true - } - - private def resolveAttrRef(ref: NamedReference, plan: LogicalPlan): AttributeReference = { - ExpressionHelper.resolveRef[AttributeReference](ref, plan) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/OptimizeWriteRule.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/OptimizeWriteRule.scala deleted file mode 100644 index 71da90baf4..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/OptimizeWriteRule.scala +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.optimize - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.Table -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation - -import org.apache.amoro.spark.SupportSparkAdapter -import org.apache.amoro.spark.mixed.SparkSQLProperties -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils.{isMixedFormatRelation, isUnkeyedTableRelation} -import org.apache.amoro.spark.sql.catalyst.plans.MixedFormatRowLevelWrite -import org.apache.amoro.spark.table.{MixedSparkTable, UnkeyedSparkTable} -import org.apache.amoro.spark.util.DistributionAndOrderingUtil - -case class OptimizeWriteRule(spark: SparkSession) extends Rule[LogicalPlan] - with SupportSparkAdapter { - override def apply(plan: LogicalPlan): LogicalPlan = { - if (!optimizeWriteEnabled()) { - plan - } else { - optimizeWritePlan(plan) - } - } - - // do optimize write for insert overwrite. insert into. - // update will not enable optimize write for reason that we should - // write update_before and update_after in same time. - def optimizeWritePlan(plan: LogicalPlan): LogicalPlan = plan transformDown { - case o @ OverwritePartitionsDynamic(r: DataSourceV2Relation, query, writeOptions, _, _) - if isMixedFormatRelation(r) => - val newQuery = distributionQuery(query, r.table, rowLevelOperation = false) - val options = writeOptions + ("writer.distributed-and-ordered" -> "true") - o.copy(query = newQuery, writeOptions = options) - - case o @ OverwriteByExpression(r: DataSourceV2Relation, _, query, writeOptions, _, _) - if isMixedFormatRelation(r) => - val newQuery = distributionQuery(query, r.table, rowLevelOperation = false) - val options = writeOptions + ("writer.distributed-and-ordered" -> "true") - o.copy(query = newQuery, writeOptions = options) - - case a @ AppendData(r: DataSourceV2Relation, query, writeOptions, _, _) - if isMixedFormatRelation(r) => - val newQuery = distributionQuery(query, r.table, rowLevelOperation = false) - val options = writeOptions + ("writer.distributed-and-ordered" -> "true") - a.copy(query = newQuery, writeOptions = options) - - case a @ AppendData(r: DataSourceV2Relation, query, _, _, _) - if isUnkeyedTableRelation(r) => - val newQuery = distributionQuery(query, r.table, rowLevelOperation = false) - a.copy(query = newQuery) - - case o @ OverwriteByExpression(r: DataSourceV2Relation, _, query, _, _, _) - if isUnkeyedTableRelation(r) => - val newQuery = distributionQuery(query, r.table, rowLevelOperation = false) - o.copy(query = newQuery) - - case o @ OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _, _, _) - if isUnkeyedTableRelation(r) => - val newQuery = distributionQuery(query, r.table, rowLevelOperation = false) - o.copy(query = newQuery) - } - - def optimizeWriteEnabled(): Boolean = { - val optimizeEnabled = spark.sessionState.conf.getConfString( - SparkSQLProperties.OPTIMIZE_WRITE_ENABLED, - SparkSQLProperties.OPTIMIZE_WRITE_ENABLED_DEFAULT) - java.lang.Boolean.parseBoolean(optimizeEnabled) - } - - private def distributionQuery( - query: LogicalPlan, - table: Table, - rowLevelOperation: Boolean, - writeBase: Boolean = true): LogicalPlan = { - import org.apache.spark.sql.connector.expressions.{Expression => Expr} - - def toCatalyst(expr: Expr): Expression = sparkAdapter.expressions().toCatalyst(expr, query) - - val mixedFormatTable = table match { - case t: MixedSparkTable => t.table() - case t: UnkeyedSparkTable => t.table() - } - - val distribution = - DistributionAndOrderingUtil.buildTableRequiredDistribution(mixedFormatTable, writeBase) - .toSeq.map(e => toCatalyst(e)) - .asInstanceOf[Seq[Expression]] - - val queryWithDistribution = if (distribution.nonEmpty) { - val partitionNum = conf.numShufflePartitions - val pp = RepartitionByExpression(distribution, query, partitionNum) - pp - } else { - query - } - - val orderingExpressions = DistributionAndOrderingUtil.buildTableRequiredSortOrder( - mixedFormatTable, - rowLevelOperation, - writeBase) - val ordering = orderingExpressions.toSeq - .map(e => toCatalyst(e)) - .asInstanceOf[Seq[SortOrder]] - - val queryWithDistributionAndOrdering = if (ordering.nonEmpty) { - Sort(ordering, global = false, child = queryWithDistribution) - } else { - queryWithDistribution - } - queryWithDistributionAndOrdering - } - -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/RewriteAppendMixedFormatTable.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/RewriteAppendMixedFormatTable.scala deleted file mode 100644 index 7b6d57ba45..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/RewriteAppendMixedFormatTable.scala +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.optimize - -import java.util - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.amoro.catalyst.MixedFormatSpark32Helper -import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, Cast, EqualTo, Expression, GreaterThan, Literal} -import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, Count} -import org.apache.spark.sql.catalyst.plans.RightOuter -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.types.LongType - -import org.apache.amoro.spark.sql.catalyst.plans._ -import org.apache.amoro.spark.sql.utils.{ProjectingInternalRow, WriteQueryProjections} -import org.apache.amoro.spark.sql.utils.RowDeltaUtils.{OPERATION_COLUMN, UPDATE_OPERATION} -import org.apache.amoro.spark.table.MixedSparkTable -import org.apache.amoro.spark.writer.WriteMode - -case class RewriteAppendMixedFormatTable(spark: SparkSession) extends Rule[LogicalPlan] { - - import org.apache.amoro.spark.sql.MixedFormatExtensionUtils._ - - override def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case AppendData(r: DataSourceV2Relation, query, writeOptions, _, _) - if isMixedFormatRelation(r) && isUpsert(r) => - val upsertQuery = rewriteAppendAsUpsertQuery(r, query) - val insertQuery = Project( - Seq(Alias(Literal(UPDATE_OPERATION), OPERATION_COLUMN)()) ++ upsertQuery.output, - upsertQuery) - val insertAttribute = - insertQuery.output.filter(_.name.contains("_mixed_before_")) - val projections = buildInsertProjections(insertQuery, insertAttribute, isUpsert = true) - val upsertOptions = writeOptions + (WriteMode.WRITE_MODE_KEY -> WriteMode.UPSERT.mode) - val writeBuilder = - MixedFormatSpark32Helper.newWriteBuilder(r.table, query.schema, upsertOptions) - val write = writeBuilder.build() - MixedFormatRowLevelWrite(r, insertQuery, upsertOptions, projections, Some(write)) - } - - def buildInsertProjections( - plan: LogicalPlan, - targetRowAttrs: Seq[Attribute], - isUpsert: Boolean): WriteQueryProjections = { - val (frontRowProjection, backRowProjection) = if (isUpsert) { - val frontRowProjection = - Some(ProjectingInternalRow.newProjectInternalRow(plan, targetRowAttrs, isFront = true, 0)) - val backRowProjection = - ProjectingInternalRow.newProjectInternalRow(plan, targetRowAttrs, isFront = false, 0) - (frontRowProjection, backRowProjection) - } else { - val backRowProjection = - ProjectingInternalRow.newProjectInternalRow(plan, targetRowAttrs, isFront = true, 0) - (null, backRowProjection) - } - WriteQueryProjections(frontRowProjection, backRowProjection) - } - - def buildValidatePrimaryKeyDuplication( - r: DataSourceV2Relation, - query: LogicalPlan): LogicalPlan = { - r.table match { - case mixedSparkTable: MixedSparkTable => - if (mixedSparkTable.table().isKeyedTable) { - val primaries = mixedSparkTable.table().asKeyedTable().primaryKeySpec().fieldNames() - val than = GreaterThan( - AggregateExpression(Count(Literal(1)), Complete, isDistinct = false), - Cast(Literal(1), LongType)) - val alias = Alias(than, "count")() - val attributes = query.output.filter(p => primaries.contains(p.name)) - Aggregate(attributes, Seq(alias), query) - } else { - throw new UnsupportedOperationException(s"UnKeyed table can not validate") - } - } - } - - def isUpsert(relation: DataSourceV2Relation): Boolean = { - val upsertWrite = relation.table.asUpsertWrite - upsertWrite.appendAsUpsert() - } - - def buildJoinCondition( - primaries: util.List[String], - tableScan: LogicalPlan, - insertPlan: LogicalPlan): Expression = { - var i = 0 - var joinCondition: Expression = null - val expressions = new util.ArrayList[Expression] - while (i < primaries.size) { - val primary = primaries.get(i) - val primaryAttr = insertPlan.output.find(_.name == primary).get - val joinAttribute = - tableScan.output.find(_.name.replace("_mixed_before_", "") == primary).get - val experssion = EqualTo(primaryAttr, joinAttribute) - expressions.add(experssion) - i += 1 - } - expressions.forEach(experssion => { - if (joinCondition == null) { - joinCondition = experssion - } else { - joinCondition = And(joinCondition, experssion) - } - }) - joinCondition - } - - def rewriteAppendAsUpsertQuery( - r: DataSourceV2Relation, - query: LogicalPlan): LogicalPlan = { - r.table match { - case mixedSparkTable: MixedSparkTable => - if (mixedSparkTable.table().isKeyedTable) { - val primaries = mixedSparkTable.table().asKeyedTable().primaryKeySpec().fieldNames() - val tablePlan = buildKeyedTableBeforeProject(r) - // val insertPlan = buildKeyedTableInsertProjection(query) - val joinCondition = buildJoinCondition(primaries, tablePlan, query) - Join(tablePlan, query, RightOuter, Some(joinCondition), JoinHint.NONE) - } else { - query - } - } - } - - private def buildKeyedTableInsertProjection(relation: LogicalPlan): LogicalPlan = { - val output = relation.output - val outputWithValues = output.map(a => { - Alias(a, "_mixed_after_" + a.name)() - }) - Project(outputWithValues, relation) - } - - private def buildKeyedTableBeforeProject(relation: DataSourceV2Relation): LogicalPlan = { - val output = relation.output - val outputWithValues = output.map(a => { - Alias(a, "_mixed_before_" + a.name)() - }) - Project(outputWithValues, relation) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/RewriteDeleteFromMixedFormatTable.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/RewriteDeleteFromMixedFormatTable.scala deleted file mode 100644 index c594e1c725..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/RewriteDeleteFromMixedFormatTable.scala +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.optimize - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.amoro.catalyst.MixedFormatSpark32Helper -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, Literal} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} -import org.apache.spark.sql.types.StructType - -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils.{asTableRelation, isMixedFormatRelation, MixedFormatTableHelper} -import org.apache.amoro.spark.sql.catalyst.plans.MixedFormatRowLevelWrite -import org.apache.amoro.spark.sql.utils.{MixedFormatRewriteHelper, ProjectingInternalRow, WriteQueryProjections} -import org.apache.amoro.spark.sql.utils.RowDeltaUtils.{DELETE_OPERATION, OPERATION_COLUMN} -import org.apache.amoro.spark.table.{MixedSparkTable, SupportsExtendIdentColumns, SupportsRowLevelOperator} -import org.apache.amoro.spark.writer.WriteMode - -case class RewriteDeleteFromMixedFormatTable(spark: SparkSession) extends Rule[LogicalPlan] - with MixedFormatRewriteHelper { - - def buildDeleteProjections( - plan: LogicalPlan, - targetRowAttrs: Seq[AttributeReference], - isKeyedTable: Boolean): WriteQueryProjections = { - val (frontRowProjection, backRowProjection) = if (isKeyedTable) { - val frontRowProjection = - Some(ProjectingInternalRow.newProjectInternalRow(plan, targetRowAttrs, isFront = true, 0)) - (frontRowProjection, null) - } else { - val attributes = plan.output.filter(r => r.name.equals("_file") || r.name.equals("_pos")) - val frontRowProjection = - Some(ProjectingInternalRow.newProjectInternalRow( - plan, - targetRowAttrs ++ attributes, - isFront = true, - 0)) - (frontRowProjection, null) - } - WriteQueryProjections(frontRowProjection, backRowProjection) - } - - override def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case DeleteFromTable(table, condition) if isMixedFormatRelation(table) => - val r = asTableRelation(table) - val upsertWrite = r.table.asUpsertWrite - val scanBuilder = upsertWrite.newUpsertScanBuilder(r.options) - if (condition.isEmpty) { - val cond = Literal.TrueLiteral - pushFilter(scanBuilder, cond, r.output) - } else { - pushFilter(scanBuilder, condition.get, r.output) - } - val query = buildUpsertQuery(r, upsertWrite, scanBuilder, condition) - var options: Map[String, String] = Map.empty - options += (WriteMode.WRITE_MODE_KEY -> WriteMode.UPSERT.toString) - val writeBuilder = MixedFormatSpark32Helper.newWriteBuilder(r.table, query.schema, options) - val write = writeBuilder.build() - - val projections = - buildDeleteProjections(query, r.output, MixedFormatExtensionUtils.isKeyedTable(r)) - MixedFormatRowLevelWrite(r, query, options, projections, Some(write)) - } - - def buildUpsertQuery( - r: DataSourceV2Relation, - upsert: SupportsRowLevelOperator, - scanBuilder: SupportsExtendIdentColumns, - condition: Option[Expression]): LogicalPlan = { - r.table match { - case table: MixedSparkTable => - if (table.table().isUnkeyedTable) { - if (upsert.requireAdditionIdentifierColumns()) { - scanBuilder.withIdentifierColumns() - } - } - case _ => - } - val scan = scanBuilder.build() - val outputAttr = toOutputAttrs(scan.readSchema(), r.output) - val valuesRelation = DataSourceV2ScanRelation(r, scan, outputAttr) - - val matchValueQuery = if (condition.isDefined) { - Filter(condition.get, valuesRelation) - } else { - valuesRelation - } - val withOperation = - Seq(Alias(Literal(DELETE_OPERATION), OPERATION_COLUMN)()) ++ matchValueQuery.output - val deleteQuery = Project(withOperation, matchValueQuery) - deleteQuery - } - - def toOutputAttrs(schema: StructType, attrs: Seq[AttributeReference]): Seq[AttributeReference] = { - val nameToAttr = attrs.map(_.name).zip(attrs).toMap - schema.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()).map { - a => - nameToAttr.get(a.name) match { - case Some(ref) => - // keep the attribute id if it was present in the relation - a.withExprId(ref.exprId) - case _ => - // if the field is new, create a new attribute - AttributeReference(a.name, a.dataType, a.nullable, a.metadata)() - } - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/RewriteUpdateMixedFormatTable.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/RewriteUpdateMixedFormatTable.scala deleted file mode 100644 index ca0992ff29..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/optimize/RewriteUpdateMixedFormatTable.scala +++ /dev/null @@ -1,212 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.optimize - -import java.util - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.amoro.catalyst.MixedFormatSpark32Helper -import org.apache.spark.sql.catalyst.expressions.{Alias, And, AttributeReference, Cast, EqualTo, Expression, Literal} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} -import org.apache.spark.sql.types.StructType - -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils.{asTableRelation, isMixedFormatRelation, MixedFormatTableHelper} -import org.apache.amoro.spark.sql.catalyst.plans.MixedFormatRowLevelWrite -import org.apache.amoro.spark.sql.utils.{MixedFormatRewriteHelper, ProjectingInternalRow, WriteQueryProjections} -import org.apache.amoro.spark.sql.utils.RowDeltaUtils.{DELETE_OPERATION, INSERT_OPERATION, OPERATION_COLUMN, UPDATE_OPERATION} -import org.apache.amoro.spark.table.{MixedSparkTable, SupportsExtendIdentColumns, SupportsRowLevelOperator} -import org.apache.amoro.spark.writer.WriteMode - -/** - * rewrite update table plan as append upsert data. - */ -case class RewriteUpdateMixedFormatTable(spark: SparkSession) extends Rule[LogicalPlan] - with MixedFormatRewriteHelper { - - def buildUpdateProjections( - plan: LogicalPlan, - targetRowAttrs: Seq[AttributeReference], - isKeyedTable: Boolean): WriteQueryProjections = { - val (frontRowProjection, backRowProjection) = if (isKeyedTable) { - val frontRowProjection = - Some(ProjectingInternalRow.newProjectInternalRow(plan, targetRowAttrs, isFront = true, 0)) - val backRowProjection = - ProjectingInternalRow.newProjectInternalRow(plan, targetRowAttrs, isFront = false, 0) - (frontRowProjection, backRowProjection) - } else { - val attributes = plan.output.filter(r => r.name.equals("_file") || r.name.equals("_pos")) - val frontRowProjection = - Some(ProjectingInternalRow.newProjectInternalRow( - plan, - targetRowAttrs ++ attributes, - isFront = true, - 0)) - val backRowProjection = - ProjectingInternalRow.newProjectInternalRow(plan, targetRowAttrs, isFront = true, 0) - (frontRowProjection, backRowProjection) - } - WriteQueryProjections(frontRowProjection, backRowProjection) - } - - override def apply(plan: LogicalPlan): LogicalPlan = plan match { - case u: UpdateTable if isMixedFormatRelation(u.table) => - val mixedFormatRelation = asTableRelation(u.table) - val upsertWrite = mixedFormatRelation.table.asUpsertWrite - val scanBuilder = upsertWrite.newUpsertScanBuilder(mixedFormatRelation.options) - if (u.condition.isEmpty) { - val cond = Literal.TrueLiteral - pushFilter(scanBuilder, cond, mixedFormatRelation.output) - } else { - pushFilter(scanBuilder, u.condition.get, mixedFormatRelation.output) - } - val upsertQuery = - buildUpsertQuery(mixedFormatRelation, upsertWrite, scanBuilder, u.assignments, u.condition) - val query = upsertQuery - var options: Map[String, String] = Map.empty - options += (WriteMode.WRITE_MODE_KEY -> WriteMode.UPSERT.toString) - val writeBuilder = - MixedFormatSpark32Helper.newWriteBuilder(mixedFormatRelation.table, query.schema, options) - val write = writeBuilder.build() - val projections = buildUpdateProjections( - query, - mixedFormatRelation.output, - MixedFormatExtensionUtils.isKeyedTable(mixedFormatRelation)) - MixedFormatRowLevelWrite(mixedFormatRelation, query, options, projections, Some(write)) - - case _ => plan - } - - def buildUpsertQuery( - r: DataSourceV2Relation, - upsert: SupportsRowLevelOperator, - scanBuilder: SupportsExtendIdentColumns, - assignments: Seq[Assignment], - condition: Option[Expression]): LogicalPlan = { - r.table match { - case table: MixedSparkTable => - if (table.table().isUnkeyedTable) { - if (upsert.requireAdditionIdentifierColumns()) { - scanBuilder.withIdentifierColumns() - } - } - case _ => - } - val scan = scanBuilder.build() - val outputAttr = toOutputAttrs(scan.readSchema(), r.output) - val valuesRelation = DataSourceV2ScanRelation(r, scan, outputAttr) - val matchedRowsQuery = if (condition.isDefined) { - Filter(condition.get, valuesRelation) - } else { - valuesRelation - } - r.table match { - case a: MixedSparkTable => - if (a.table().isKeyedTable) { - val updatedRowsQuery = - buildKeyedTableUpdateInsertProjection(valuesRelation, matchedRowsQuery, assignments) - val primaries = a.table().asKeyedTable().primaryKeySpec().fieldNames() - validatePrimaryKey(primaries, assignments) - updatedRowsQuery - } else { - val updatedRowsQuery = - buildUnKeyedTableUpdateInsertProjection(valuesRelation, matchedRowsQuery, assignments) - val deleteQuery = Project( - Seq(Alias(Literal(DELETE_OPERATION), OPERATION_COLUMN)()) - ++ matchedRowsQuery.output.iterator, - matchedRowsQuery) - val insertQuery = Project( - Seq(Alias(Literal(INSERT_OPERATION), OPERATION_COLUMN)()) - ++ updatedRowsQuery.output.iterator, - updatedRowsQuery) - Union(deleteQuery, insertQuery) - } - } - } - - def validatePrimaryKey(primaries: util.List[String], assignments: Seq[Assignment]): Unit = { - assignments.map(_.key).foreach(f => { - val name = f.asInstanceOf[AttributeReference].name - if (primaries.contains(name)) { - throw new UnsupportedOperationException(s"primary key: ${name} can not be updated") - } - }) - } - - protected def toOutputAttrs( - schema: StructType, - attrs: Seq[AttributeReference]): Seq[AttributeReference] = { - val nameToAttr = attrs.map(_.name).zip(attrs).toMap - schema.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()).map { - a => - nameToAttr.get(a.name) match { - case Some(ref) => - // keep the attribute id if it was present in the relation - a.withExprId(ref.exprId) - case _ => - // if the field is new, create a new attribute - AttributeReference(a.name, a.dataType, a.nullable, a.metadata)() - } - } - } - - private def buildKeyedTableUpdateInsertProjection( - relation: LogicalPlan, - scanPlan: LogicalPlan, - assignments: Seq[Assignment]): LogicalPlan = { - val output = relation.output - val assignmentMap = assignments.map(a => - if (a.value.dataType.catalogString.equals(a.key.dataType.catalogString)) { - a.key.asInstanceOf[AttributeReference].name -> a.value - } else { - a.key.asInstanceOf[AttributeReference].name -> Cast(a.value, a.key.dataType) - }).toMap - val outputWithValues = relation.output ++ output.map(a => { - if (assignmentMap.contains(a.name)) { - Alias(assignmentMap(a.name), "_mixed_after_" + a.name)() - } else { - Alias(a, "_mixed_after_" + a.name)() - } - }) - Project(Seq(Alias(Literal(UPDATE_OPERATION), OPERATION_COLUMN)()) ++ outputWithValues, scanPlan) - } - - private def buildUnKeyedTableUpdateInsertProjection( - relation: LogicalPlan, - scanPlan: LogicalPlan, - assignments: Seq[Assignment]): LogicalPlan = { - val output = relation.output - val assignmentMap = assignments.map(a => - if (a.value.dataType.catalogString.equals(a.key.dataType.catalogString)) { - a.key.asInstanceOf[AttributeReference].name -> a.value - } else { - a.key.asInstanceOf[AttributeReference].name -> Cast(a.value, a.key.dataType) - }).toMap - val outputWithValues = output.map(a => { - if (assignmentMap.contains(a.name)) { - Alias(assignmentMap(a.name), a.name)() - } else { - a - } - }) - Project(outputWithValues, scanPlan) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/parser/MixedFormatParserUtils.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/parser/MixedFormatParserUtils.scala deleted file mode 100644 index ab014be031..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/parser/MixedFormatParserUtils.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.parser - -import org.antlr.v4.runtime.{ParserRuleContext, Token} -import org.antlr.v4.runtime.misc.Interval -import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin} - -/* Partially copied from Apache Spark's Parser to avoid dependency on Spark Internals */ -object MixedFormatParserUtils { - def withOrigin[T](ctx: ParserRuleContext)(f: => T): T = { - val current = CurrentOrigin.get - CurrentOrigin.set(position(ctx.getStart)) - try { - f - } finally { - CurrentOrigin.set(current) - } - } - - def position(token: Token): Origin = { - val opt = Option(token) - Origin(opt.map(_.getLine), opt.map(_.getCharPositionInLine)) - } - - /** Get the command which created the token. */ - def command(ctx: ParserRuleContext): String = { - val stream = ctx.getStart.getInputStream - stream.getText(Interval.of(0, stream.size() - 1)) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/parser/MixedFormatSqlExtensionsParser.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/parser/MixedFormatSqlExtensionsParser.scala deleted file mode 100644 index 8dbcdca7b6..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/parser/MixedFormatSqlExtensionsParser.scala +++ /dev/null @@ -1,303 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.parser - -import java.util.Locale - -import scala.collection.JavaConverters.seqAsJavaListConverter -import scala.util.Try - -import org.antlr.v4.runtime._ -import org.antlr.v4.runtime.atn.PredictionMode -import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} -import org.apache.iceberg.spark.Spark3Util -import org.apache.iceberg.spark.source.SparkTable -import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.amoro.parser.MixedFormatSqlExtendAstBuilder -import org.apache.spark.sql.catalyst.{FunctionIdentifier, SQLConfHelper, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, UnresolvedRelation} -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.trees.Origin -import org.apache.spark.sql.connector.catalog.{Table, TableCatalog} -import org.apache.spark.sql.types.{DataType, StructType} - -import org.apache.amoro.spark.sql.catalyst.plans.UnresolvedMergeIntoMixedFormatTable -import org.apache.amoro.spark.sql.parser._ -import org.apache.amoro.spark.table.MixedSparkTable -import org.apache.amoro.spark.util.MixedFormatSparkUtils - -class MixedFormatSqlExtensionsParser(delegate: ParserInterface) extends ParserInterface - with SQLConfHelper { - - private lazy val createTableAstBuilder = new MixedFormatSqlExtendAstBuilder() - - /** - * Parse a string to a DataType. - */ - override def parseDataType(sqlText: String): DataType = { - delegate.parseDataType(sqlText) - } - - /** - * Parse a string to a raw DataType without CHAR/VARCHAR replacement. - */ - def parseRawDataType(sqlText: String): DataType = throw new UnsupportedOperationException() - - /** - * Parse a string to an Expression. - */ - override def parseExpression(sqlText: String): Expression = { - delegate.parseExpression(sqlText) - } - - /** - * Parse a string to a TableIdentifier. - */ - override def parseTableIdentifier(sqlText: String): TableIdentifier = { - delegate.parseTableIdentifier(sqlText) - } - - /** - * Parse a string to a FunctionIdentifier. - */ - override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = { - delegate.parseFunctionIdentifier(sqlText) - } - - /** - * Parse a string to a multi-part identifier. - */ - override def parseMultipartIdentifier(sqlText: String): Seq[String] = { - delegate.parseMultipartIdentifier(sqlText) - } - - /** - * Creates StructType for a given SQL string, which is a comma separated list of field - * definitions which will preserve the correct Hive metadata. - */ - override def parseTableSchema(sqlText: String): StructType = { - delegate.parseTableSchema(sqlText) - } - - private val mixedFormatExtendSqlFilters: Seq[String => Boolean] = Seq( - s => s.contains("create table") && s.contains("primary key"), - s => s.contains("create temporary table") && s.contains("primary key")) - - private def isMixedFormatExtendSql(sqlText: String): Boolean = { - val normalized = sqlText.toLowerCase(Locale.ROOT).trim().replaceAll("\\s+", " ") - mixedFormatExtendSqlFilters.exists(f => f(normalized)) - } - - private def buildLexer(sql: String): Option[Lexer] = { - lazy val charStream = new UpperCaseCharStream(CharStreams.fromString(sql)) - if (isMixedFormatExtendSql(sql)) { - Some(new MixedFormatSqlExtendLexer(charStream)) - } else { - Option.empty - } - } - - private def buildAntlrParser(stream: TokenStream, lexer: Lexer): Parser = { - lexer match { - case _: MixedFormatSqlExtendLexer => - val parser = new MixedFormatSqlExtendParser(stream) - parser.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled - parser.SQL_standard_keyword_behavior = conf.ansiEnabled - parser - case _ => - throw new IllegalStateException("no suitable parser found") - } - } - - private def toLogicalResult(parser: Parser): LogicalPlan = parser match { - case p: MixedFormatSqlExtendParser => - createTableAstBuilder.visitExtendStatement(p.extendStatement()) - } - - /** - * Parse a string to a LogicalPlan. - */ - override def parsePlan(sqlText: String): LogicalPlan = { - val lexerOpt = buildLexer(sqlText) - if (lexerOpt.isDefined) { - val lexer = lexerOpt.get - lexer.removeErrorListeners() - - val tokenStream = new CommonTokenStream(lexer) - val parser = buildAntlrParser(tokenStream, lexer) - parser.removeErrorListeners() - - try { - try { - // first, try parsing with potentially faster SLL mode - parser.getInterpreter.setPredictionMode(PredictionMode.SLL) - toLogicalResult(parser) - } catch { - case _: ParseCancellationException => - // if we fail, parse with LL mode - tokenStream.seek(0) // rewind input stream - parser.reset() - // Try Again. - parser.getInterpreter.setPredictionMode(PredictionMode.LL) - toLogicalResult(parser) - } - } catch { - case e: ParseException if e.command.isDefined => - throw e - case e: ParseException => throw e.withCommand(sqlText) - case e: AnalysisException => - val position = Origin(e.line, e.startPosition) - throw new ParseException(Option(sqlText), e.message, position, position) - } - } else { - val parsedPlan = delegate.parsePlan(sqlText) - parsedPlan match { - case p => - replaceMergeIntoCommands(p) - } - } - } - - private def replaceMergeIntoCommands(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsDown { - - case MergeIntoTable( - UnresolvedMixedFormatTable(aliasedTable), - source, - cond, - matchedActions, - notMatchedActions) => - UnresolvedMergeIntoMixedFormatTable( - aliasedTable, - source, - cond, - matchedActions, - notMatchedActions) - - case DeleteFromTable(UnresolvedIcebergTable(aliasedTable), condition) => - DeleteFromIcebergTable(aliasedTable, condition) - - case UpdateTable(UnresolvedIcebergTable(aliasedTable), assignments, condition) => - UpdateIcebergTable(aliasedTable, assignments, condition) - - case MergeIntoTable( - UnresolvedIcebergTable(aliasedTable), - source, - cond, - matchedActions, - notMatchedActions) => - // cannot construct MergeIntoIcebergTable right away as MERGE operations require special resolution - // that's why the condition and actions must be hidden from the regular resolution rules in Spark - // see ResolveMergeIntoTableReferences for details - val context = MergeIntoContext(cond, matchedActions, notMatchedActions) - UnresolvedMergeIntoIcebergTable(aliasedTable, source, context) - } - - object UnresolvedIcebergTable { - - def unapply(plan: LogicalPlan): Option[LogicalPlan] = { - EliminateSubqueryAliases(plan) match { - case UnresolvedRelation(multipartIdentifier, _, _) if isIcebergTable(multipartIdentifier) => - Some(plan) - case _ => - None - } - } - - private def isIcebergTable(multipartIdent: Seq[String]): Boolean = { - val catalogAndIdentifier = - Spark3Util.catalogAndIdentifier(SparkSession.active, multipartIdent.asJava) - catalogAndIdentifier.catalog match { - case tableCatalog: TableCatalog => - Try(tableCatalog.loadTable(catalogAndIdentifier.identifier)) - .map(isIcebergTable) - .getOrElse(false) - - case _ => - false - } - } - - private def isIcebergTable(table: Table): Boolean = table match { - case _: SparkTable => true - case _ => false - } - } - - object UnresolvedMixedFormatTable { - - def unapply(plan: LogicalPlan): Option[LogicalPlan] = { - EliminateSubqueryAliases(plan) match { - case UnresolvedRelation(multipartIdentifier, _, _) - if isKeyedTable(multipartIdentifier) => - Some(plan) - case _ => - None - } - } - - private def isKeyedTable(multipartIdent: Seq[String]): Boolean = { - val catalogAndIdentifier = - MixedFormatSparkUtils.tableCatalogAndIdentifier(SparkSession.active, multipartIdent.asJava) - catalogAndIdentifier.catalog match { - case tableCatalog: TableCatalog => - Try(tableCatalog.loadTable(catalogAndIdentifier.identifier)) - .map(isKeyedTable) - .getOrElse(false) - - case _ => - false - } - } - - private def isKeyedTable(table: Table): Boolean = table match { - case _: MixedSparkTable => - true - case _ => false - } - } -} - -/* Copied from Apache Spark's to avoid dependency on Spark Internals */ -class UpperCaseCharStream(wrapped: CodePointCharStream) extends CharStream { - override def consume(): Unit = wrapped.consume - - override def getSourceName(): String = wrapped.getSourceName - - override def index(): Int = wrapped.index - - override def mark(): Int = wrapped.mark - - override def release(marker: Int): Unit = wrapped.release(marker) - - override def seek(where: Int): Unit = wrapped.seek(where) - - override def size(): Int = wrapped.size - - override def getText(interval: Interval): String = wrapped.getText(interval) - - // scalastyle:off - override def LA(i: Int): Int = { - val la = wrapped.LA(i) - if (la == 0 || la == IntStream.EOF) la - else Character.toUpperCase(la) - } - // scalastyle:on -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/AlterMixedFormatTableDropPartition.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/AlterMixedFormatTableDropPartition.scala deleted file mode 100644 index 21b60173d9..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/AlterMixedFormatTableDropPartition.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.plans - -import org.apache.spark.sql.catalyst.analysis.PartitionSpec -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2PartitionCommand} - -case class AlterMixedFormatTableDropPartition( - table: LogicalPlan, - parts: Seq[PartitionSpec], - ifExists: Boolean, - purge: Boolean) extends V2PartitionCommand { - override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = { - copy(table = newChild) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/MergeIntoMixedFormatTable.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/MergeIntoMixedFormatTable.scala deleted file mode 100644 index b8ff205e9e..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/MergeIntoMixedFormatTable.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.plans - -import org.apache.spark.sql.amoro.catalyst.AssignmentHelper -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.logical._ - -case class MergeIntoMixedFormatTable( - targetTable: LogicalPlan, - sourceTable: LogicalPlan, - mergeCondition: Expression, - matchedActions: Seq[MergeAction], - notMatchedActions: Seq[MergeAction], - rewritePlan: Option[LogicalPlan] = None) extends BinaryCommand { - - lazy val aligned: Boolean = { - val matchedActionsAligned = matchedActions.forall { - case UpdateAction(_, assignments) => - AssignmentHelper.aligned(targetTable, assignments) - case _: DeleteAction => - true - case _ => - false - } - - val notMatchedActionsAligned = notMatchedActions.forall { - case InsertAction(_, assignments) => - AssignmentHelper.aligned(targetTable, assignments) - case _ => - false - } - - matchedActionsAligned && notMatchedActionsAligned - } - - def condition: Option[Expression] = Some(mergeCondition) - - override def left: LogicalPlan = targetTable - - override def right: LogicalPlan = sourceTable - - override protected def withNewChildrenInternal( - newLeft: LogicalPlan, - newRight: LogicalPlan): LogicalPlan = { - copy(targetTable = newLeft, sourceTable = newRight) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/MergeRows.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/MergeRows.scala deleted file mode 100644 index bfc8d31fcc..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/MergeRows.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.plans - -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode} -import org.apache.spark.sql.catalyst.util.truncatedString - -case class MergeRows( - isSourceRowPresent: Expression, - isTargetRowPresent: Expression, - matchedConditions: Seq[Expression], - matchedOutputs: Seq[Seq[Expression]], - notMatchedConditions: Seq[Expression], - notMatchedOutputs: Seq[Seq[Expression]], - rowIdAttrs: Seq[Attribute], - matchedRowCheck: Boolean, - unMatchedRowCheck: Boolean, - emitNotMatchedTargetRows: Boolean, - output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode { - - require(!emitNotMatchedTargetRows) - - override lazy val producedAttributes: AttributeSet = { - AttributeSet(output.filterNot(attr => inputSet.contains(attr))) - } - - override lazy val references: AttributeSet = child.outputSet - - override def simpleString(maxFields: Int): String = { - s"MergeRows${truncatedString(output, "[", ", ", "]", maxFields)}" - } - - override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = { - copy(child = newChild) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/MixedFormatRowLevelWrite.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/MixedFormatRowLevelWrite.scala deleted file mode 100644 index 4d6c3c44f0..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/MixedFormatRowLevelWrite.scala +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.plans - -import org.apache.spark.sql.catalyst.analysis.NamedRelation -import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, V2WriteCommandLike} -import org.apache.spark.sql.connector.write.Write - -import org.apache.amoro.spark.sql.utils.WriteQueryProjections - -case class MixedFormatRowLevelWrite( - table: NamedRelation, - query: LogicalPlan, - options: Map[String, String], - projections: WriteQueryProjections, - write: Option[Write] = None) extends V2WriteCommandLike with Command { - - def isByName: Boolean = false - - override def outputResolved: Boolean = true - - override protected def withNewChildInternal(newChild: LogicalPlan): MixedFormatRowLevelWrite = { - copy(query = newChild) - } - -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/QueryWithConstraintCheckPlan.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/QueryWithConstraintCheckPlan.scala deleted file mode 100644 index 005af5a980..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/QueryWithConstraintCheckPlan.scala +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.plans - -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet} -import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, LogicalPlan} -import org.apache.spark.sql.catalyst.util.truncatedString - -case class QueryWithConstraintCheckPlan( - scanPlan: LogicalPlan, - fileFilterPlan: LogicalPlan) extends BinaryNode { - - @transient - override lazy val references: AttributeSet = AttributeSet(fileFilterPlan.output) - - override def left: LogicalPlan = scanPlan - override def right: LogicalPlan = fileFilterPlan - override def output: Seq[Attribute] = scanPlan.output - - override def simpleString(maxFields: Int): String = { - s"QueryWithConstraintCheck${truncatedString(output, "[", ", ", "]", maxFields)}" - } - - override protected def withNewChildrenInternal( - newLeft: LogicalPlan, - newRight: LogicalPlan): LogicalPlan = { - copy(scanPlan = newLeft, fileFilterPlan = newRight) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/TruncateMixedFormatTable.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/TruncateMixedFormatTable.scala deleted file mode 100644 index 0a42ebf5f1..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/TruncateMixedFormatTable.scala +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.plans - -import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} - -case class TruncateMixedFormatTable(child: LogicalPlan) extends Command { - override def children: Seq[LogicalPlan] = child :: Nil - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]) - : LogicalPlan = child -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/UnresolvedMergeIntoMixedFormatTable.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/UnresolvedMergeIntoMixedFormatTable.scala deleted file mode 100644 index 9e2a3b5408..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/UnresolvedMergeIntoMixedFormatTable.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.plans - -import org.apache.spark.sql.catalyst.expressions.{AssignmentUtils, Expression} -import org.apache.spark.sql.catalyst.plans.logical.{BinaryCommand, DeleteAction, InsertAction, LogicalPlan, MergeAction, UpdateAction} - -/** - * A node that hides the MERGE condition and actions from regular Spark resolution. - */ -case class UnresolvedMergeIntoMixedFormatTable( - targetTable: LogicalPlan, - sourceTable: LogicalPlan, - mergeCondition: Expression, - matchedActions: Seq[MergeAction], - notMatchedActions: Seq[MergeAction]) extends BinaryCommand { - - lazy val aligned: Boolean = { - val matchedActionsAligned = matchedActions.forall { - case UpdateAction(_, assignments) => - AssignmentUtils.aligned(targetTable, assignments) - case _: DeleteAction => - true - case _ => - false - } - - val notMatchedActionsAligned = notMatchedActions.forall { - case InsertAction(_, assignments) => - AssignmentUtils.aligned(targetTable, assignments) - case _ => - false - } - - matchedActionsAligned && notMatchedActionsAligned - } - - def duplicateResolved: Boolean = targetTable.outputSet.intersect(sourceTable.outputSet).isEmpty - - override def left: LogicalPlan = targetTable - override def right: LogicalPlan = sourceTable - - override protected def withNewChildrenInternal( - newLeft: LogicalPlan, - newRight: LogicalPlan): LogicalPlan = { - copy(targetTable = newLeft, sourceTable = newRight) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/logicals.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/logicals.scala deleted file mode 100644 index 1e79170103..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/logicals.scala +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.plans - -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} -import org.apache.spark.sql.catalyst.util.truncatedString - -import org.apache.amoro.spark.command.{MigrateToMixedFormatCommand, MixedFormatSparkCommand} - -abstract class MixedFormatCommandLogicalPlan(command: MixedFormatSparkCommand) extends Command { - override def output: Seq[Attribute] = { - command.outputType().map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) - } - - override def simpleString(maxFields: Int): String = { - s"${command.name()}LogicPlan${truncatedString(output, "[", ",", "]", maxFields)} ${command.execInfo}" - } -} - -case class MigrateToMixedFormatLogicalPlan(command: MigrateToMixedFormatCommand) - extends MixedFormatCommandLogicalPlan(command) { - override def children: Seq[LogicalPlan] = Nil - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]) - : LogicalPlan = null -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/statements.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/statements.scala deleted file mode 100644 index 7e8a7b5fb5..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/catalyst/plans/statements.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.catalyst.plans - -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ParsedStatement} - -case class MigrateToMixedFormatStatement(source: Seq[String], target: Seq[String]) - extends ParsedStatement { - override def children: Seq[LogicalPlan] = Nil - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]) - : LogicalPlan = null -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/AlterMixedFormatTableDropPartitionExec.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/AlterMixedFormatTableDropPartitionExec.scala deleted file mode 100644 index c3e50d60e8..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/AlterMixedFormatTableDropPartitionExec.scala +++ /dev/null @@ -1,130 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.execution - -import java.util - -import org.apache.iceberg.spark.SparkFilters -import org.apache.spark.sql.amoro.catalyst.ExpressionHelper -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{PartitionSpec, ResolvedPartitionSpec} -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, EqualNullSafe, Expression, Literal} -import org.apache.spark.sql.connector.catalog.Table -import org.apache.spark.sql.execution.datasources.v2.LeafV2CommandExec -import org.apache.spark.sql.types._ - -import org.apache.amoro.op.OverwriteBaseFiles -import org.apache.amoro.spark.table.{MixedSparkTable, UnkeyedSparkTable} - -case class AlterMixedFormatTableDropPartitionExec( - table: Table, - parts: Seq[PartitionSpec]) extends LeafV2CommandExec { - override protected def run(): Seq[InternalRow] = { - // build partitions - val rows: Seq[InternalRow] = parts.map { - case ResolvedPartitionSpec(_, ident, _) => - ident - } - val names: Seq[Seq[String]] = parts.map { - case ResolvedPartitionSpec(names, _, _) => - names - } - val expressions = new util.ArrayList[Expression] - var index = 0; - while (index < names.size) { - var i = 0 - val name = names.apply(index) - val row = rows.apply(index) - while (i < name.size) { - val dataType = table.schema().apply(name.apply(i)).dataType - val data = convertInternalRowDataByType(dataType, row, i) - val experssion = EqualNullSafe( - AttributeReference( - name.apply(i), - dataType)(), - Literal(data)) - expressions.add(experssion) - i += 1 - } - index += 1 - } - - var deleteExpr: Expression = null - expressions.forEach(exp => { - if (deleteExpr == null) { - deleteExpr = exp - } else { - deleteExpr = And(deleteExpr, exp) - } - }); - // build filters - val filters = splitConjunctivePredicates(deleteExpr).map { - filter => - ExpressionHelper.translateFilter(deleteExpr).getOrElse( - throw new UnsupportedOperationException("Cannot translate expression to source filter")) - }.toArray - val expression = SparkFilters.convert(filters) - table match { - case mixedSparkTable: MixedSparkTable => - if (mixedSparkTable.table().isKeyedTable) { - val txId = mixedSparkTable.table().asKeyedTable().beginTransaction(null) - val overwriteBaseFiles: OverwriteBaseFiles = - mixedSparkTable.table().asKeyedTable().newOverwriteBaseFiles() - overwriteBaseFiles.overwriteByRowFilter(expression) - overwriteBaseFiles.updateOptimizedSequenceDynamically(txId) - overwriteBaseFiles.commit() - } else { - val overwriteFiles = mixedSparkTable.table().asUnkeyedTable().newOverwrite() - overwriteFiles.overwriteByRowFilter(expression) - overwriteFiles.commit() - } - case unkeyedSparkTable: UnkeyedSparkTable => - val overwriteFiles = unkeyedSparkTable.table().newOverwrite() - overwriteFiles.overwriteByRowFilter(expression) - overwriteFiles.commit() - } - Nil - } - - def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { - condition match { - case And(cond1, cond2) => - splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) - case other => other :: Nil - } - } - - def convertInternalRowDataByType(dataType: DataType, data: InternalRow, index: Int): Any = { - dataType match { - case BinaryType => data.getBinary(index) - case IntegerType => data.getInt(index) - case BooleanType => data.getBoolean(index) - case LongType => data.getLong(index) - case DoubleType => data.getDouble(index) - case FloatType => data.getFloat(index) - case ShortType => data.getShort(index) - case ByteType => data.getByte(index) - case StringType => data.getString(index) - case _ => - throw new UnsupportedOperationException("Cannot convert data by this type") - } - } - - override def output: Seq[Attribute] = Nil -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/DescribeKeyedTableExec.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/DescribeKeyedTableExec.scala deleted file mode 100644 index d3a32aac6d..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/DescribeKeyedTableExec.scala +++ /dev/null @@ -1,164 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.execution - -import scala.collection.JavaConverters.mapAsScalaMapConverter -import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.connector.catalog._ -import org.apache.spark.sql.execution.datasources.v2.LeafV2CommandExec -import org.apache.spark.sql.types.{MetadataBuilder, StringType, StructField, StructType} - -import org.apache.amoro.spark.table.MixedSparkTable -import org.apache.amoro.table.KeyedTable - -case class DescribeKeyedTableExec( - table: Table, - catalog: TableCatalog, - ident: Identifier, - isExtended: Boolean) extends LeafV2CommandExec { - val outputAttrs: Seq[AttributeReference] = Seq( - AttributeReference( - "col_name", - StringType, - nullable = false, - new MetadataBuilder().putString("comment", "name of the column").build())(), - AttributeReference( - "data_type", - StringType, - nullable = false, - new MetadataBuilder().putString("comment", "data type of the column").build())(), - AttributeReference( - "comment", - StringType, - nullable = true, - new MetadataBuilder().putString("comment", "comment of the column").build())()) - - private[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = - StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) - - private val toRow = { - RowEncoder(fromAttributes(outputAttrs)).resolveAndBind().createSerializer() - } - - private def addPrimaryColumns( - rows: ArrayBuffer[InternalRow], - keyedTable: MixedSparkTable): Unit = { - keyedTable.table() match { - case table: KeyedTable => - rows += emptyRow() - rows += toCatalystRow("# Primary keys", "", "") - if (!table.primaryKeySpec.primaryKeyExisted()) { - rows += toCatalystRow("Not keyed table", "", "") - } else { - table.primaryKeySpec().primaryKeyStruct().fields().toSeq.foreach(k => { - rows += toCatalystRow(k.name(), k.`type`().toString, "") - }) - } - case _ => - } - } - - override protected def run(): Seq[InternalRow] = { - val rows = new ArrayBuffer[InternalRow]() - addSchema(rows, table) - addPartitioning(rows, table) - - if (isExtended) { - addMetadataColumns(rows, table) - addTableDetails(rows, table) - } - - table match { - case keyedTable: MixedSparkTable => - addPrimaryColumns(rows, keyedTable) - case _ => - Nil - } - rows.toSeq - } - - val TABLE_RESERVED_PROPERTIES = - Seq( - TableCatalog.PROP_COMMENT, - TableCatalog.PROP_LOCATION, - TableCatalog.PROP_PROVIDER, - TableCatalog.PROP_OWNER) - - private def addTableDetails(rows: ArrayBuffer[InternalRow], table: Table): Unit = { - rows += emptyRow() - rows += toCatalystRow("# Detailed Table Information", "", "") - rows += toCatalystRow("Name", table.name(), "") - - TABLE_RESERVED_PROPERTIES.foreach(propKey => { - if (table.properties.containsKey(propKey)) { - rows += toCatalystRow(propKey.capitalize, table.properties.get(propKey), "") - } - }) - val properties = - table.properties.asScala.toList - .filter(kv => !TABLE_RESERVED_PROPERTIES.contains(kv._1)) - .sortBy(_._1).map { - case (key, value) => key + "=" + value - }.mkString("[", ",", "]") - rows += toCatalystRow("Table Properties", properties, "") - } - - private def addSchema(rows: ArrayBuffer[InternalRow], table: Table): Unit = { - rows ++= table.schema.map { column => - toCatalystRow( - column.name, - column.dataType.simpleString, - column.getComment().getOrElse("")) - } - } - - private def addMetadataColumns(rows: ArrayBuffer[InternalRow], table: Table): Unit = table match { - case hasMeta: SupportsMetadataColumns if hasMeta.metadataColumns.nonEmpty => - rows += emptyRow() - rows += toCatalystRow("# Metadata Columns", "", "") - rows ++= hasMeta.metadataColumns.map { column => - toCatalystRow( - column.name, - column.dataType.simpleString, - Option(column.comment()).getOrElse("")) - } - case _ => - } - - private def addPartitioning(rows: ArrayBuffer[InternalRow], table: Table): Unit = { - rows += emptyRow() - rows += toCatalystRow("# Partitioning", "", "") - if (table.partitioning.isEmpty) { - rows += toCatalystRow("Not partitioned", "", "") - } else { - rows ++= table.partitioning.zipWithIndex.map { - case (transform, index) => toCatalystRow(s"Part $index", transform.describe(), "") - } - } - } - - private def emptyRow(): InternalRow = toCatalystRow("", "", "") - - override def output: Seq[Attribute] = outputAttrs -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/ExtendedMixedFormatStrategy.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/ExtendedMixedFormatStrategy.scala deleted file mode 100644 index f9e806d18f..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/ExtendedMixedFormatStrategy.scala +++ /dev/null @@ -1,104 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.execution - -import scala.collection.JavaConverters.mapAsJavaMapConverter - -import org.apache.spark.sql.{SparkSession, Strategy} -import org.apache.spark.sql.catalyst.analysis.{NamedRelation, ResolvedTable} -import org.apache.spark.sql.catalyst.expressions.PredicateHelper -import org.apache.spark.sql.catalyst.plans.logical.{DescribeRelation, LogicalPlan} -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.datasources.v2._ -import org.apache.spark.sql.util.CaseInsensitiveStringMap - -import org.apache.amoro.spark.sql.MixedFormatExtensionUtils.{isMixedFormatTable, MixedFormatTableHelper} -import org.apache.amoro.spark.sql.catalyst.plans._ - -case class ExtendedMixedFormatStrategy(spark: SparkSession) extends Strategy with PredicateHelper { - - override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case DescribeRelation(r: ResolvedTable, partitionSpec, isExtended, _) - if isMixedFormatTable(r.table) => - if (partitionSpec.nonEmpty) { - throw new RuntimeException("DESCRIBE does not support partition for v2 tables.") - } - DescribeKeyedTableExec(r.table, r.catalog, r.identifier, isExtended) :: Nil - - case MigrateToMixedFormatLogicalPlan(command) => - MigrateToMixedFormatExec(command) :: Nil - - case MixedFormatRowLevelWrite( - table: DataSourceV2Relation, - query, - options, - projs, - Some(write)) => - MixedFormatRowLevelWriteExec( - table.table.asMixedSparkTable, - planLater(query), - new CaseInsensitiveStringMap(options.asJava), - projs, - refreshCache(table), - write) :: Nil - - case MergeRows( - isSourceRowPresent, - isTargetRowPresent, - matchedConditions, - matchedOutputs, - notMatchedConditions, - notMatchedOutputs, - rowIdAttrs, - matchedRowCheck, - unMatchedRowCheck, - emitNotMatchedTargetRows, - output, - child) => - MergeRowsExec( - isSourceRowPresent, - isTargetRowPresent, - matchedConditions, - matchedOutputs, - notMatchedConditions, - notMatchedOutputs, - rowIdAttrs, - matchedRowCheck, - unMatchedRowCheck, - emitNotMatchedTargetRows, - output, - planLater(child)) :: Nil - - case d @ AlterMixedFormatTableDropPartition(r: ResolvedTable, _, _, _) => - AlterMixedFormatTableDropPartitionExec(r.table, d.parts) :: Nil - - case QueryWithConstraintCheckPlan(scanPlan, fileFilterPlan) => - QueryWithConstraintCheckExec(planLater(scanPlan), planLater(fileFilterPlan)) :: Nil - - case TruncateMixedFormatTable(r: ResolvedTable) => - TruncateMixedFormatTableExec(r.table) :: Nil - - case _ => Nil - } - - private def refreshCache(r: NamedRelation)(): Unit = { - spark.sharedState.cacheManager.recacheByPlan(spark, r) - } - -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/MergeRowsExec.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/MergeRowsExec.scala deleted file mode 100644 index dda2f16cb7..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/MergeRowsExec.scala +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.execution - -import org.apache.spark.SparkException -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeSet, BasePredicate, Expression, SortOrder, UnsafeProjection} -import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate -import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} - -case class MergeRowsExec( - isSourceRowPresent: Expression, - isTargetRowPresent: Expression, - matchedConditions: Seq[Expression], - matchedOutputs: Seq[Seq[Expression]], - notMatchedConditions: Seq[Expression], - notMatchedOutputs: Seq[Seq[Expression]], - rowIdAttrs: Seq[Attribute], - matchedRowCheck: Boolean, - unMatchedRowCheck: Boolean, - emitNotMatchedTargetRows: Boolean, - output: Seq[Attribute], - child: SparkPlan) extends UnaryExecNode { - - override def requiredChildOrdering: Seq[Seq[SortOrder]] = { - if (matchedRowCheck || unMatchedRowCheck) { - // request a local sort by the row ID attrs to co-locate matches for the same target row - Seq(rowIdAttrs.map(attr => SortOrder(attr, Ascending))) - } else { - Seq(Nil) - } - } - - @transient override lazy val producedAttributes: AttributeSet = { - AttributeSet(output.filterNot(attr => inputSet.contains(attr))) - } - - @transient override lazy val references: AttributeSet = child.outputSet - - override def simpleString(maxFields: Int): String = { - s"MergeRowsExec${truncatedString(output, "[", ", ", "]", maxFields)}" - } - - override protected def doExecute(): RDD[InternalRow] = { - child.execute().mapPartitions(processPartition) - } - - override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { - copy(child = newChild) - } - - private def createProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = { - UnsafeProjection.create(exprs, attrs) - } - - private def createPredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = { - GeneratePredicate.generate(expr, attrs) - } - - private def applyProjection( - actions: Seq[(BasePredicate, Option[UnsafeProjection])], - inputRow: InternalRow): InternalRow = { - - // find the first action where the predicate evaluates to true - // if there are overlapping conditions in actions, use the first matching action - // in the example below, when id = 5, both actions match but the first one is applied - // WHEN MATCHED AND id > 1 AND id < 10 UPDATE * - // WHEN MATCHED AND id = 5 OR id = 21 DELETE - - val pair = actions.find { - case (predicate, _) => predicate.eval(inputRow) - } - - // apply the projection to produce an output row, or return null to suppress this row - pair match { - case Some((_, Some(projection))) => - val row = projection.apply(inputRow) - row - case _ => - null - } - } - - private def processPartition(rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = { - val inputAttrs = child.output - - val isSourceRowPresentPred = createPredicate(isSourceRowPresent, inputAttrs) - val isTargetRowPresentPred = createPredicate(isTargetRowPresent, inputAttrs) - - val matchedPreds = matchedConditions.map(createPredicate(_, inputAttrs)) - val matchedProjs = matchedOutputs.map { - case output if output.nonEmpty => - Some(createProjection(output, inputAttrs)) - case _ => None - } - val matchedPairs = matchedPreds zip matchedProjs - - val notMatchedPreds = notMatchedConditions.map(createPredicate(_, inputAttrs)) - val notMatchedProjs = notMatchedOutputs.map { - case output if output.nonEmpty => Some(createProjection(output, inputAttrs)) - case _ => None - } - val nonMatchedPairs = notMatchedPreds zip notMatchedProjs - - val projectTargetCols = createProjection(Nil, inputAttrs) - val rowIdProj = createProjection(rowIdAttrs, inputAttrs) - - def processRow(inputRow: InternalRow): InternalRow = { - if (emitNotMatchedTargetRows && !isSourceRowPresentPred.eval(inputRow)) { - projectTargetCols.apply(inputRow) - } else if (!isTargetRowPresentPred.eval(inputRow)) { - applyProjection(nonMatchedPairs, inputRow) - } else { - applyProjection(matchedPairs, inputRow) - } - } - - var lastMatchedRowId: InternalRow = null - - def processRowWithMatchedOrUnMatchedRowCheck(inputRow: InternalRow): InternalRow = { - val isSourceRowPresent = isSourceRowPresentPred.eval(inputRow) - val isTargetRowPresent = isTargetRowPresentPred.eval(inputRow) - - if (isSourceRowPresent && isTargetRowPresent) { - val currentRowId = rowIdProj.apply(inputRow) - if (currentRowId == lastMatchedRowId) { - throw new SparkException( - "The ON search condition of the MERGE statement matched a single row from " + - "the target table with multiple rows of the source table. ") - } - lastMatchedRowId = currentRowId.copy() - } else if (isSourceRowPresent && !isTargetRowPresent && unMatchedRowCheck) { - val currentRowId = rowIdProj.apply(inputRow) - if (currentRowId == lastMatchedRowId) { - throw new SparkException( - "There are multiple duplicate primary key data in the inserted data, " + - "which cannot guarantee the uniqueness of the primary key. ") - } - lastMatchedRowId = currentRowId.copy() - } else { - lastMatchedRowId = null - } - - if (emitNotMatchedTargetRows && !isSourceRowPresent) { - projectTargetCols.apply(inputRow) - } else if (!isTargetRowPresent) { - applyProjection(nonMatchedPairs, inputRow) - } else { - applyProjection(matchedPairs, inputRow) - } - } - - val processFunc: InternalRow => InternalRow = if (matchedRowCheck || unMatchedRowCheck) { - processRowWithMatchedOrUnMatchedRowCheck - } else { - processRow - } - - rowIterator - .map(processFunc) - .filter(row => row != null) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/MixedFormatCommandExec.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/MixedFormatCommandExec.scala deleted file mode 100644 index 5f262a34bb..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/MixedFormatCommandExec.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.execution - -import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.datasources.v2.V2CommandExec - -import org.apache.amoro.spark.command.{MigrateToMixedFormatCommand, MixedFormatSparkCommand} - -abstract class MixedFormatCommandExec(command: MixedFormatSparkCommand) extends V2CommandExec { - - val rowEncoder: ExpressionEncoder[Row] = RowEncoder(command.outputType()).resolveAndBind() - - override def run(): Seq[InternalRow] = { - val rows = command.execute().toSeq -// val encoder = RowEncoder(command.outputType()).resolveAndBind().createSerializer() - val m = rows.map(r => rowEncoder.createSerializer()(r)) - m - } - - override def output: Seq[Attribute] = { - val out = command.outputType().map(f => - AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) - out - } - - override def simpleString(maxFields: Int): String = { - s"${command.name()}CommandExec${truncatedString(output, "[", ",", "]", maxFields)} ${command.execInfo}" - } -} - -case class MigrateToMixedFormatExec(command: MigrateToMixedFormatCommand) - extends MixedFormatCommandExec(command) { - override def children: Seq[SparkPlan] = Nil - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - null -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/MixedFormatRowLevelWriteExec.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/MixedFormatRowLevelWriteExec.scala deleted file mode 100644 index 39a9c6c0aa..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/MixedFormatRowLevelWriteExec.scala +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.execution - -import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.write.Write -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.datasources.v2.{ExtendedV2ExistingTableWriteExec, WritingSparkTask} -import org.apache.spark.sql.util.CaseInsensitiveStringMap - -import org.apache.amoro.spark.sql.utils.WriteQueryProjections -import org.apache.amoro.spark.table.MixedSparkTable -import org.apache.amoro.spark.writer.RowLevelWriter - -/** - * Physical plan node to write a delta of rows to an existing table. - */ -case class MixedFormatRowLevelWriteExec( - table: MixedSparkTable, - query: SparkPlan, - writeOptions: CaseInsensitiveStringMap, - projections: WriteQueryProjections, - refreshCache: () => Unit, - write: Write) extends ExtendedV2ExistingTableWriteExec[RowLevelWriter[InternalRow]] { - - override protected def run(): Seq[InternalRow] = { - val writtenRows = writeWithV2(write.toBatch) - refreshCache() - writtenRows - } - - override def output: Seq[Attribute] = Nil - - override def child: SparkPlan = query - - override lazy val writingTask: WritingSparkTask[RowLevelWriter[InternalRow]] = { - DeltaWithMetadataWritingSparkTask(projections) - } - - override protected def withNewChildInternal(newChild: SparkPlan): MixedFormatRowLevelWriteExec = { - copy(query = newChild) - } -} - -case class DeltaWithMetadataWritingSparkTask( - projs: WriteQueryProjections) extends WritingSparkTask[RowLevelWriter[InternalRow]] { - - private lazy val frontRowProjection = projs.frontRowProjection.orNull - private lazy val backRowProjection = projs.backRowProjection - - override protected def writeFunc(writer: RowLevelWriter[InternalRow], row: InternalRow): Unit = { - val operation = row.getString(0) - - operation match { - case "D" => - frontRowProjection.project(row) - writer.delete(frontRowProjection) - - case "U" => - frontRowProjection.project(row) - backRowProjection.project(row) - writer.update(frontRowProjection, backRowProjection) - - case "I" => - backRowProjection.project(row) - writer.insert(backRowProjection) - - case other => - throw new SparkException(s"Unexpected operation ID: $other") - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/QueryWithConstraintCheckExec.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/QueryWithConstraintCheckExec.scala deleted file mode 100644 index 3555addefd..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/QueryWithConstraintCheckExec.scala +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.execution - -import org.apache.spark.SparkException -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} -import org.apache.spark.sql.catalyst.plans.physical -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} -import org.apache.spark.sql.vectorized.ColumnarBatch - -case class QueryWithConstraintCheckExec( - scanExec: SparkPlan, - fileFilterExec: SparkPlan) - extends BinaryExecNode { - - override protected def doPrepare(): Unit = { - val rows = fileFilterExec.executeCollect() - if (rows.length > 0) { - throw new SparkException( - "There are multiple duplicate primary key data in the inserted data, " + - "which cannot guarantee the uniqueness of the primary key. ") - } - } - - override def left: SparkPlan = scanExec - - override def right: SparkPlan = fileFilterExec - - override def output: Seq[Attribute] = scanExec.output - - override def outputPartitioning: physical.Partitioning = scanExec.outputPartitioning - - override def outputOrdering: Seq[SortOrder] = scanExec.outputOrdering - - override def supportsColumnar: Boolean = scanExec.supportsColumnar - - override protected def withNewChildrenInternal( - newLeft: SparkPlan, - newRight: SparkPlan): SparkPlan = { - copy(scanExec = newLeft, fileFilterExec = newRight) - } - - override protected def doExecute(): RDD[InternalRow] = { - val result = scanExec.execute() - if (result.partitions.length == 0) { - sparkContext.parallelize(Array.empty[InternalRow], 1) - } else { - result - } - } - - override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { - val result = scanExec.executeColumnar() - if (result.partitions.length == 0) { - sparkContext.parallelize(Array.empty[ColumnarBatch], 1) - } else { - result - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/TruncateMixedFormatTableExec.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/TruncateMixedFormatTableExec.scala deleted file mode 100644 index 255ae8761b..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/execution/TruncateMixedFormatTableExec.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.execution - -import org.apache.iceberg.expressions.Expressions -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.Table -import org.apache.spark.sql.execution.datasources.v2.LeafV2CommandExec - -import org.apache.amoro.op.OverwriteBaseFiles -import org.apache.amoro.spark.table.{MixedSparkTable, UnkeyedSparkTable} - -case class TruncateMixedFormatTableExec(table: Table) extends LeafV2CommandExec { - override protected def run(): Seq[InternalRow] = { - table match { - case mixedSparkTable: MixedSparkTable => - if (mixedSparkTable.table().isKeyedTable) { - val txId = mixedSparkTable.table().asKeyedTable().beginTransaction(null); - val overwriteBaseFiles: OverwriteBaseFiles = - mixedSparkTable.table().asKeyedTable().newOverwriteBaseFiles() - overwriteBaseFiles.overwriteByRowFilter(Expressions.alwaysTrue()) - overwriteBaseFiles.updateOptimizedSequenceDynamically(txId) - overwriteBaseFiles.commit() - } else { - val overwriteFiles = mixedSparkTable.table().asUnkeyedTable().newOverwrite() - overwriteFiles.overwriteByRowFilter(Expressions.alwaysTrue()) - overwriteFiles.commit() - } - case unkeyedSparkTable: UnkeyedSparkTable => - val overwriteFiles = unkeyedSparkTable.table().newOverwrite() - overwriteFiles.overwriteByRowFilter(Expressions.alwaysTrue()) - overwriteFiles.commit() - } - Nil - } - - override def output: Seq[Attribute] = Nil -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/MixedFormatRewriteHelper.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/MixedFormatRewriteHelper.scala deleted file mode 100644 index 07571d86e9..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/MixedFormatRewriteHelper.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.utils - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper} -import org.apache.spark.sql.connector.read.ScanBuilder -import org.apache.spark.sql.execution.datasources.v2.PushDownUtils - -trait MixedFormatRewriteHelper extends PredicateHelper with Logging { - def pushFilter( - scanBuilder: ScanBuilder, - cond: Expression, - tableAttrs: Seq[AttributeReference]): Unit = { - val predicates = extractFilters(cond, tableAttrs) - if (predicates.nonEmpty) { - val normalizedPredicates = normalizeExprs(predicates, tableAttrs) - PushDownUtils.pushFilters(scanBuilder, normalizedPredicates) - } - } - - private def extractFilters( - cond: Expression, - tableAttrs: Seq[AttributeReference]): Seq[Expression] = { - val tableAttrSet = AttributeSet(tableAttrs) - splitConjunctivePredicates(cond).filter(_.references.subsetOf(tableAttrSet)) - } - - def normalizeExprs( - exprs: Seq[Expression], - attributes: Seq[AttributeReference]): Seq[Expression] = { - exprs.map { e => - e transform { - case a: AttributeReference => - a.withName(attributes.find(_.semanticEquals(a)).getOrElse(a).name) - } - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/ProjectingInternalRow.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/ProjectingInternalRow.scala deleted file mode 100644 index e2ac90ee0a..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/ProjectingInternalRow.scala +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.utils - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} -import org.apache.spark.sql.types.{DataType, Decimal, StructField, StructType} -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} - -/** - * An InternalRow that projects particular columns from another InternalRow without copying - * the underlying data. - */ -case class ProjectingInternalRow(schema: StructType, colOrdinals: Seq[Int]) extends InternalRow { - assert(schema.size == colOrdinals.size) - - private var row: InternalRow = _ - - override def numFields: Int = colOrdinals.size - - def project(row: InternalRow): Unit = { - this.row = row - } - - def getRow: InternalRow = { - row - } - - override def setNullAt(i: Int): Unit = { - throw new UnsupportedOperationException("Cannot modify InternalRowProjection") - } - - override def update(i: Int, value: Any): Unit = { - throw new UnsupportedOperationException("Cannot modify InternalRowProjection") - } - - override def copy(): InternalRow = { - val newRow = if (row != null) row.copy() else null - val newProjection = ProjectingInternalRow(schema, colOrdinals) - newProjection.project(newRow) - newProjection - } - - override def isNullAt(ordinal: Int): Boolean = { - row.isNullAt(colOrdinals(ordinal)) - } - - override def getBoolean(ordinal: Int): Boolean = { - row.getBoolean(colOrdinals(ordinal)) - } - - override def getByte(ordinal: Int): Byte = { - row.getByte(colOrdinals(ordinal)) - } - - override def getShort(ordinal: Int): Short = { - row.getShort(colOrdinals(ordinal)) - } - - override def getInt(ordinal: Int): Int = { - row.getInt(colOrdinals(ordinal)) - } - - override def getLong(ordinal: Int): Long = { - row.getLong(colOrdinals(ordinal)) - } - - override def getFloat(ordinal: Int): Float = { - row.getFloat(colOrdinals(ordinal)) - } - - override def getDouble(ordinal: Int): Double = { - row.getDouble(colOrdinals(ordinal)) - } - - override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = { - row.getDecimal(colOrdinals(ordinal), precision, scale) - } - - override def getUTF8String(ordinal: Int): UTF8String = { - row.getUTF8String(colOrdinals(ordinal)) - } - - override def getBinary(ordinal: Int): Array[Byte] = { - row.getBinary(colOrdinals(ordinal)) - } - - override def getInterval(ordinal: Int): CalendarInterval = { - row.getInterval(colOrdinals(ordinal)) - } - - override def getStruct(ordinal: Int, numFields: Int): InternalRow = { - row.getStruct(colOrdinals(ordinal), numFields) - } - - override def getArray(ordinal: Int): ArrayData = { - row.getArray(colOrdinals(ordinal)) - } - - override def getMap(ordinal: Int): MapData = { - row.getMap(colOrdinals(ordinal)) - } - - override def get(ordinal: Int, dataType: DataType): AnyRef = { - row.get(colOrdinals(ordinal), dataType) - } -} - -object ProjectingInternalRow { - def newProjectInternalRow( - plan: LogicalPlan, - attrs: Seq[Attribute], - isFront: Boolean, - offset: Int): ProjectingInternalRow = { - - val colOrdinals = - attrs.map(attr => plan.output.indexWhere(_.name == attr.name)).filter(p => p.!=(-1)) - val planAttrs = colOrdinals.map(plan.output(_)) - val schema = - StructType(planAttrs.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) - if (!isFront) { - val backColOrdinals = colOrdinals.map(c => c + colOrdinals.size + offset) - ProjectingInternalRow(schema, backColOrdinals) - } else { - ProjectingInternalRow(schema, colOrdinals) - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/RowDeltaUtils.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/RowDeltaUtils.scala deleted file mode 100644 index c32b31c46a..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/RowDeltaUtils.scala +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.utils - -object RowDeltaUtils { - final val OPERATION_COLUMN: String = "_upsert_op" - final val DELETE_OPERATION: String = "D" - final val UPDATE_OPERATION: String = "U" - final val INSERT_OPERATION: String = "I" -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/WriteQueryProjections.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/WriteQueryProjections.scala deleted file mode 100644 index 950c8c1908..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/WriteQueryProjections.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.utils - -case class WriteQueryProjections( - frontRowProjection: Option[ProjectingInternalRow], - backRowProjection: ProjectingInternalRow) diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/expressions.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/expressions.scala deleted file mode 100644 index fb1fab6b50..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/amoro/spark/sql/utils/expressions.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.sql.utils - -import org.apache.spark.sql.catalyst -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.connector.expressions.{Literal, NamedReference, Transform} -import org.apache.spark.sql.types.{DataType, IntegerType, StringType} - -/** - * This file copy form apache spark. - * Helper methods for working with the logical expressions API. - * - * Factory methods can be used when referencing the logical expression nodes is ambiguous because - * logical and internal expressions are used. - */ -object LogicalExpressions { - def parseReference(name: String): NamedReference = - FieldReference(Seq(name)) -} - -case class FieldReference(parts: Seq[String]) extends NamedReference { - override def fieldNames: Array[String] = parts.toArray - - override def describe: String = parts.map(quoteIfNeeded).mkString(".") - - override def toString: String = describe - - def quoteIfNeeded(part: String): String = { - if (part.contains(".") || part.contains("`")) { - s"`${part.replace("`", "``")}`" - } else { - part - } - } -} - -object FieldReference { - def apply(column: String): NamedReference = { - LogicalExpressions.parseReference(column) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/catalyst/AssignmentHelper.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/catalyst/AssignmentHelper.scala deleted file mode 100644 index 35ae3a2a4c..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/catalyst/AssignmentHelper.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.amoro.catalyst - -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.SQLConfHelper -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, ExtractValue, GetStructField} -import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan} -import org.apache.spark.sql.types.DataType - -object AssignmentHelper extends SQLConfHelper { - - /** - * Checks whether assignments are aligned and match table columns. - * - * @param table a target table - * @param assignments assignments to check - * @return true if the assignments are aligned - */ - def aligned(table: LogicalPlan, assignments: Seq[Assignment]): Boolean = { - val sameSize = table.output.size == assignments.size - sameSize && table.output.zip(assignments).forall { case (attr, assignment) => - val key = assignment.key - val value = assignment.value - val refsEqual = toAssignmentRef(attr).zip(toAssignmentRef(key)) - .forall { case (attrRef, keyRef) => conf.resolver(attrRef, keyRef) } - - refsEqual && - DataType.equalsIgnoreCompatibleNullability(value.dataType, attr.dataType) && - (attr.nullable || !value.nullable) - } - } - - def toAssignmentRef(expr: Expression): Seq[String] = expr match { - case attr: AttributeReference => - Seq(attr.name) - case Alias(child, _) => - toAssignmentRef(child) - case GetStructField(child, _, Some(name)) => - toAssignmentRef(child) :+ name - case other: ExtractValue => - throw new AnalysisException(s"Updating nested fields is only supported for structs: $other") - case other => - throw new AnalysisException(s"Cannot convert to a reference, unsupported expression: $other") - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/catalyst/ExpressionHelper.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/catalyst/ExpressionHelper.scala deleted file mode 100644 index 972eae7814..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/catalyst/ExpressionHelper.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.amoro.catalyst - -import org.apache.spark.sql.catalyst.SQLConfHelper -import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.connector.expressions.NamedReference -import org.apache.spark.sql.execution.datasources.DataSourceStrategy -import org.apache.spark.sql.sources.Filter - -/** - * A class that is inspired by V2ExpressionUtils in Spark but supports Iceberg transforms. - */ -object ExpressionHelper extends SQLConfHelper { - - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper - - def resolveRef[T <: NamedExpression](ref: NamedReference, plan: LogicalPlan): T = { - plan.resolve(ref.fieldNames.toSeq, conf.resolver) match { - case Some(namedExpr) => - namedExpr.asInstanceOf[T] - case None => - val name = ref.fieldNames.toSeq.quoted - val outputString = plan.output.map(_.name).mkString(",") - throw new UnsupportedOperationException(s"Unable to resolve $name given $outputString") - } - } - - def translateFilter(deleteExpr: Expression): Option[Filter] = { - DataSourceStrategy.translateFilter(deleteExpr, supportNestedPredicatePushdown = true) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/catalyst/MixedFormatSpark32Helper.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/catalyst/MixedFormatSpark32Helper.scala deleted file mode 100644 index 2fc392d368..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/catalyst/MixedFormatSpark32Helper.scala +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.amoro.catalyst - -import java.util.UUID - -import org.apache.iceberg.Schema -import org.apache.iceberg.spark.SparkSchemaUtil -import org.apache.spark.sql.{catalyst, connector, AnalysisException} -import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} -import org.apache.spark.sql.catalyst.expressions.{Expression, IcebergBucketTransform, IcebergDayTransform, IcebergHourTransform, IcebergMonthTransform, IcebergTruncateTransform, IcebergYearTransform, NamedExpression, NullIntolerant} -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.connector.catalog.Table -import org.apache.spark.sql.connector.expressions._ -import org.apache.spark.sql.connector.write.{ExtendedLogicalWriteInfoImpl, WriteBuilder} -import org.apache.spark.sql.types.{DataType, LongType, StructType} - -import org.apache.amoro.data.PrimaryKeyData -import org.apache.amoro.spark.SparkInternalRowWrapper -import org.apache.amoro.spark.sql.connector.expressions.FileIndexBucket - -object MixedFormatSpark32Helper extends SQLConfHelper { - - import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ - - def toCatalyst(expr: connector.expressions.Expression, query: LogicalPlan): Expression = { - def resolve(parts: Seq[String]): NamedExpression = { - val resolver = conf.resolver - query.resolve(parts, resolver) match { - case Some(attr) => - attr - case None => - val ref = parts - throw new AnalysisException(s"Cannot resolve '$ref' using ${query.output}") - } - } - - expr match { - case it: IdentityTransform => - resolve(it.ref.fieldNames) - case fbt: FileIndexBucket => - MixedFormatFileIndexBucketTransform(fbt.mask(), fbt.primaryKeyData(), fbt.schema()) - case MixedFormatBucketTransform(n, r) => - IcebergBucketTransform(n, ExpressionHelper.resolveRef[NamedExpression](r, query)) - case yt: YearsTransform => - IcebergYearTransform(resolve(yt.ref.fieldNames)) - case mt: MonthsTransform => - IcebergMonthTransform(resolve(mt.ref.fieldNames)) - case dt: DaysTransform => - IcebergDayTransform(resolve(dt.ref.fieldNames)) - case ht: HoursTransform => - IcebergHourTransform(resolve(ht.ref.fieldNames)) - case ref: FieldReference => - resolve(ref.fieldNames) - case TruncateTransform(n, ref) => - IcebergTruncateTransform(resolve(ref.fieldNames), width = n) - case sort: SortOrder => - val catalystChild = toCatalyst(sort.expression(), query) - catalyst.expressions.SortOrder( - catalystChild, - toCatalystSortDirection(sort.direction()), - toCatalystNullOrdering(sort.nullOrdering()), - Seq.empty) - case _ => - throw new RuntimeException(s"$expr is not currently supported") - } - } - - private def toCatalystSortDirection(direction: SortDirection) - : catalyst.expressions.SortDirection = { - direction match { - case SortDirection.ASCENDING => catalyst.expressions.Ascending - case SortDirection.DESCENDING => catalyst.expressions.Descending - } - } - - private def toCatalystNullOrdering(nullOrdering: NullOrdering) - : catalyst.expressions.NullOrdering = { - nullOrdering match { - case NullOrdering.NULLS_FIRST => catalyst.expressions.NullsFirst - case NullOrdering.NULLS_LAST => catalyst.expressions.NullsLast - } - } - - private object MixedFormatBucketTransform { - def unapply(transform: Transform): Option[(Int, FieldReference)] = transform match { - case bt: BucketTransform => bt.columns match { - case Seq(nf: NamedReference) => - Some(bt.numBuckets.value(), FieldReference(nf.fieldNames())) - case _ => - None - } - case _ => None - } - } - - case class MixedFormatFileIndexBucketTransform( - numBuckets: Int, - keyData: PrimaryKeyData, - schema: Schema) - extends Expression with CodegenFallback with NullIntolerant { - - @transient - lazy val internalRowToStruct: SparkInternalRowWrapper = - new SparkInternalRowWrapper(SparkSchemaUtil.convert(schema)) - - override def dataType: DataType = LongType - - override def eval(input: InternalRow): Any = { - if (null == input) { - return null - } - internalRowToStruct.wrap(input) - keyData.primaryKey(internalRowToStruct) - val node = keyData.treeNode(numBuckets) - node.getIndex - } - - override def nullable: Boolean = true - - override def children: Seq[Expression] = Nil - - override def toString(): String = { - s"MixedFormatFileIndexBucket($numBuckets)" - } - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) - : Expression = null - } - - def newWriteBuilder( - table: Table, - rowSchema: StructType, - writeOptions: Map[String, String], - rowIdSchema: StructType = null, - metadataSchema: StructType = null): WriteBuilder = { - val info = ExtendedLogicalWriteInfoImpl( - queryId = UUID.randomUUID().toString, - rowSchema, - writeOptions.asOptions, - rowIdSchema, - metadataSchema) - table.asWritable.newWriteBuilder(info) - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/parser/MixedFormatSqlExtendAstBuilder.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/parser/MixedFormatSqlExtendAstBuilder.scala deleted file mode 100644 index 4501cf0b36..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/parser/MixedFormatSqlExtendAstBuilder.scala +++ /dev/null @@ -1,3133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.amoro.parser - -import java.util -import java.util.Locale -import java.util.concurrent.TimeUnit - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, Set} - -import org.antlr.v4.runtime.{ParserRuleContext, Token} -import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} -import org.apache.commons.codec.DecoderException -import org.apache.commons.codec.binary.Hex -import org.apache.spark.internal.Logging -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{FunctionIdentifier, SQLConfHelper, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{MultiAlias, UnresolvedAlias, UnresolvedAttribute, UnresolvedExtractValue, UnresolvedFunction, UnresolvedGenerator, UnresolvedHaving, UnresolvedInlineTable, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedTable, UnresolvedTableOrView, UnresolvedTableValuedFunction, UnresolvedView} -import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat} -import org.apache.spark.sql.catalyst.expressions.{Add, Alias, And, Ascending, AttributeReference, BaseGroupingSets, BitwiseAnd, BitwiseNot, BitwiseOr, BitwiseXor, CaseWhen, Cast, Concat, CreateNamedStruct, CreateStruct, Cube, CurrentDate, CurrentRow, CurrentTimestamp, CurrentUser, Descending, Divide, EmptyRow, EqualNullSafe, EqualTo, Exists, Expression, GreaterThan, GreaterThanOrEqual, GroupingSets, In, InSubquery, IntegralDivide, IsNotNull, IsNotUnknown, IsNull, IsUnknown, LambdaFunction, LateralSubquery, LessThan, LessThanOrEqual, Like, LikeAll, LikeAny, ListQuery, Literal, Multiply, NamedExpression, Not, NotLikeAll, NotLikeAny, NullsFirst, NullsLast, Or, Overlay, Predicate, RangeFrame, Remainder, RLike, Rollup, RowFrame, ScalarSubquery, SortOrder, SpecifiedWindowFrame, StringLocate, StringTrim, StringTrimLeft, StringTrimRight, Substring, Subtract, TryCast, UnaryMinus, UnaryPositive, UnboundedFollowing, UnboundedPreceding, UnresolvedNamedLambdaVariable, UnresolvedWindowExpression, UnspecifiedFrame, WindowExpression, WindowSpec, WindowSpecDefinition, WindowSpecReference} -import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} -import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils} -import org.apache.spark.sql.catalyst.util.DateTimeUtils._ -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog} -import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition -import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -import org.apache.spark.util.Utils.isTesting -import org.apache.spark.util.random.RandomSampler - -import org.apache.amoro.spark.sql.parser.{MixedFormatSqlExtendBaseVisitor, MixedFormatSqlExtendParser} -import org.apache.amoro.spark.sql.parser.MixedFormatSqlExtendParser._ - -class MixedFormatSqlExtendAstBuilder() - extends MixedFormatSqlExtendBaseVisitor[AnyRef] with SQLConfHelper with Logging { - import org.apache.spark.sql.catalyst.parser.ParserUtils._ - - def setPrimaryKeyNotNull(columns: Seq[StructField], primary: Seq[String]): Seq[StructField] = { - columns.map(c => - if (primary.contains(c.name)) { - StructField( - name = c.name, - dataType = c.dataType, - nullable = false, - metadata = c.metadata) - } else { - c - }) - } - - type colListAndPk = (Seq[StructField], Seq[String]) - - private def visitColListAndPk(ctx: MixedFormatSqlExtendParser.ColListAndPkContext) - : colListAndPk = { - ctx match { - case colWithPk: ColListWithPkContext => - (visitColTypeList(colWithPk.colTypeList()), visitPrimarySpec(colWithPk.primarySpec())) - case colOnlyPk: ColListOnlyPkContext => - (Nil, visitPrimarySpec(colOnlyPk.primarySpec())) - case _ => - throw new ParseException("Invalid InsertIntoContext", ctx) - } - } - - override def visitExtendStatement(ctx: MixedFormatSqlExtendParser.ExtendStatementContext) - : LogicalPlan = withOrigin(ctx) { - visit(ctx.statement()).asInstanceOf[LogicalPlan] - } - - override def visitCreateTableWithPk(ctx: MixedFormatSqlExtendParser.CreateTableWithPkContext) - : LogicalPlan = withOrigin(ctx) { - val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) - - val colListAndPk = visitColListAndPk(ctx.colListAndPk()) - val columns = Option(colListAndPk._1).getOrElse(Nil) - val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText) - val ( - partTransforms, - partCols, - bucketSpec, - properties, - options, - location, - comment, - serdeInfo) = { - visitCreateTableClauses(ctx.createTableClauses()) - } - - var primaryForCtas: Seq[String] = Seq.empty - if (columns.isEmpty) { - primaryForCtas = colListAndPk._2 - } - - if (provider.isDefined && serdeInfo.isDefined) { - operationNotAllowed(s"CREATE TABLE ... USING ... ${serdeInfo.get.describe}", ctx) - } - - if (temp) { - val asSelect = if (ctx.query == null) "" else " AS ..." - operationNotAllowed( - s"CREATE TEMPORARY TABLE ...$asSelect, use CREATE TEMPORARY VIEW instead", - ctx) - } - - val partitioning = partitionExpressions(partTransforms, partCols, ctx) - - Option(ctx.query).map(plan) match { - case Some(_) if columns.nonEmpty => - operationNotAllowed( - "Schema may not be specified in a Create Table As Select (CTAS) statement", - ctx) - - case Some(_) if partCols.nonEmpty => - // non-reference partition columns are not allowed because schema can't be specified - operationNotAllowed( - "Partition column types may not be specified in Create Table As Select (CTAS)", - ctx) - - case Some(query) => - val propertiesMap = buildProperties(primaryForCtas, properties) - var writeOptions: Map[String, String] = Map.empty - CreateTableAsSelectStatement( - table, - query, - partitioning, - bucketSpec, - propertiesMap, - provider, - options, - location, - comment, - writeOptions, - serdeInfo, - external = external, - ifNotExists = ifNotExists) - - case _ => - // Note: table schema includes both the table columns list and the partition columns - // with data type. - if (primaryForCtas.nonEmpty) { - operationNotAllowed( - "Primary keys do not allow this input format in a Create Table", - ctx) - } - val primary = colListAndPk._2 - // Setting the primary key not nullable - val newColumns = setPrimaryKeyNotNull(columns, primary) - val schema = StructType(newColumns ++ partCols) - val propertiesMap = buildProperties(primary, properties) - CreateTableStatement( - table, - schema, - partitioning, - bucketSpec, - propertiesMap, - provider, - options, - location, - comment, - serdeInfo, - external = external, - ifNotExists = ifNotExists) - } - } - - /** - * Build a properties with primary key. - */ - private def buildProperties(primary: Seq[String], properties: Map[String, String]) = { - var propertiesMap: Map[String, String] = properties - propertiesMap += ("provider" -> "arctic") - val primaryMap = seqAsJavaList(primary) - if (primaryMap != null) { - propertiesMap += ("primary.keys" -> String.join(",", primaryMap)) - } - propertiesMap - } - - type TableClauses = ( - Seq[Transform], - Seq[StructField], - Option[BucketSpec], - Map[String, String], - Map[String, String], - Option[String], - Option[String], - Option[SerdeInfo]) - - protected def visitPrimarySpecList(ctx: util.List[MixedFormatSqlExtendParser.PrimarySpecContext]) - : Option[Seq[String]] = { - ctx.asScala.headOption.map(visitPrimarySpec) - } - - /** - * Create a comment string. - */ - override def visitPrimarySpec(ctx: MixedFormatSqlExtendParser.PrimarySpecContext): Seq[String] = - withOrigin(ctx) { - visitIdentifierList(ctx.identifierList()) - } - - override def visitCreateTableClauses(ctx: MixedFormatSqlExtendParser.CreateTableClausesContext) - : TableClauses = { - checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) - checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) - checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) - checkDuplicateClauses(ctx.createFileFormat, "STORED AS/BY", ctx) - checkDuplicateClauses(ctx.rowFormat, "ROW FORMAT", ctx) - checkDuplicateClauses(ctx.commentSpec(), "COMMENT", ctx) - checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) - checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) - - if (ctx.skewSpec.size > 0) { - operationNotAllowed("CREATE TABLE ... SKEWED BY", ctx) - } - val (partTransforms, partCols) = - Option(ctx.partitioning).map(visitPartitionFieldList).getOrElse((Nil, Nil)) - val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) - val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) - val cleanedProperties = cleanTableProperties(ctx, properties) - val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) - val location = visitLocationSpecList(ctx.locationSpec()) - val (cleanedOptions, newLocation) = cleanTableOptions(ctx, options, location) - val comment = visitCommentSpecList(ctx.commentSpec()) - val serdeInfo = - getSerdeInfo(ctx.rowFormat.asScala.toSeq, ctx.createFileFormat.asScala.toSeq, ctx) - ( - partTransforms, - partCols, - bucketSpec, - cleanedProperties, - cleanedOptions, - newLocation, - comment, - serdeInfo) - } - - protected def typedVisit[T](ctx: ParseTree): T = { - ctx.accept(this).asInstanceOf[T] - } - - /** - * Override the default behavior for all visit methods. This will only return a non-null result - * when the context has only one child. This is done because there is no generic method to - * combine the results of the context children. In all other cases null is returned. - */ - override def visitChildren(node: RuleNode): AnyRef = { - if (node.getChildCount == 1) { - node.getChild(0).accept(this) - } else { - null - } - } - - /* ******************************************************************************************** - * Plan parsing - * ******************************************************************************************** */ - protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree) - - /** - * Create a top-level plan with Common Table Expressions. - */ - override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) { - val query = plan(ctx.queryTerm).optionalMap(ctx.queryOrganization)(withQueryResultClauses) - - // Apply CTEs - query.optionalMap(ctx.ctes)(withCTE) - } - - private def withCTE(ctx: CtesContext, plan: LogicalPlan): LogicalPlan = { - val ctes = ctx.namedQuery.asScala.map { nCtx => - val namedQuery = visitNamedQuery(nCtx) - (namedQuery.alias, namedQuery) - } - // Check for duplicate names. - val duplicates = ctes.groupBy(_._1).filter(_._2.size > 1).keys - if (duplicates.nonEmpty) { - throw QueryParsingErrors.duplicateCteDefinitionNamesError( - duplicates.mkString("'", "', '", "'"), - ctx) - } - UnresolvedWith(plan, ctes.toSeq) - } - - /** - * Create a logical query plan for a hive-style FROM statement body. - */ - private def withFromStatementBody( - ctx: FromStatementBodyContext, - plan: LogicalPlan): LogicalPlan = withOrigin(ctx) { - // two cases for transforms and selects - if (ctx.transformClause != null) { - withTransformQuerySpecification( - ctx, - ctx.transformClause, - ctx.lateralView, - ctx.whereClause, - ctx.aggregationClause, - ctx.havingClause, - ctx.windowClause, - plan) - } else { - withSelectQuerySpecification( - ctx, - ctx.selectClause, - ctx.lateralView, - ctx.whereClause, - ctx.aggregationClause, - ctx.havingClause, - ctx.windowClause, - plan) - } - } - - override def visitFromStatement(ctx: FromStatementContext): LogicalPlan = withOrigin(ctx) { - val from = visitFromClause(ctx.fromClause) - val selects = ctx.fromStatementBody.asScala.map { body => - withFromStatementBody(body, from). - // Add organization statements. - optionalMap(body.queryOrganization)(withQueryResultClauses) - } - // If there are multiple SELECT just UNION them together into one query. - if (selects.length == 1) { - selects.head - } else { - Union(selects.toSeq) - } - } - - /** - * Create a named logical plan. - * - * This is only used for Common Table Expressions. - */ - override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) { - val subQuery: LogicalPlan = - plan(ctx.query).optionalMap(ctx.columnAliases)((columnAliases, plan) => - UnresolvedSubqueryColumnAliases(visitIdentifierList(columnAliases), plan)) - SubqueryAlias(ctx.name.getText, subQuery) - } - - /** - * Parameters used for writing query to a table: - * (UnresolvedRelation, tableColumnList, partitionKeys, ifPartitionNotExists). - */ - type InsertTableParams = (UnresolvedRelation, Seq[String], Map[String, Option[String]], Boolean) - - /** - * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider). - */ - type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String]) - - private def getTableAliasWithoutColumnAlias( - ctx: TableAliasContext, - op: String): Option[String] = { - if (ctx == null) { - None - } else { - val ident = ctx.strictIdentifier() - if (ctx.identifierList() != null) { - throw QueryParsingErrors.columnAliasInOperationNotAllowedError(op, ctx) - } - if (ident != null) Some(ident.getText) else None - } - } - - /** - * Convert a constant of any type into a string. This is typically used in DDL commands, and its - * main purpose is to prevent slight differences due to back to back conversions i.e.: - * String -> Literal -> String. - */ - protected def visitStringConstant( - ctx: ConstantContext, - legacyNullAsString: Boolean): String = withOrigin(ctx) { - expression(ctx) match { - case Literal(null, _) if !legacyNullAsString => null - case l @ Literal(null, _) => l.toString - case l: Literal => - // TODO For v2 commands, we will cast the string back to its actual value, - // which is a waste and can be improved in the future. - Cast(l, StringType, Some(conf.sessionLocalTimeZone)).eval().toString - case other => - throw new IllegalArgumentException(s"Only literals are allowed in the " + - s"partition spec, but got ${other.sql}") - } - } - - /** - * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan. These - * clauses determine the shape (ordering/partitioning/rows) of the query result. - */ - private def withQueryResultClauses( - ctx: QueryOrganizationContext, - query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - import ctx._ - - // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause. - val withOrder = - if (!order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) { - // ORDER BY ... - Sort(order.asScala.map(visitSortItem).toSeq, global = true, query) - } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) { - // SORT BY ... - Sort(sort.asScala.map(visitSortItem).toSeq, global = false, query) - } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) { - // DISTRIBUTE BY ... - withRepartitionByExpression(ctx, expressionList(distributeBy), query) - } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) { - // SORT BY ... DISTRIBUTE BY ... - Sort( - sort.asScala.map(visitSortItem).toSeq, - global = false, - withRepartitionByExpression(ctx, expressionList(distributeBy), query)) - } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) { - // CLUSTER BY ... - val expressions = expressionList(clusterBy) - Sort( - expressions.map(SortOrder(_, Ascending)), - global = false, - withRepartitionByExpression(ctx, expressions, query)) - } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) { - // [EMPTY] - query - } else { - throw QueryParsingErrors.combinationQueryResultClausesUnsupportedError(ctx) - } - - // WINDOWS - val withWindow = withOrder.optionalMap(windowClause)(withWindowClause) - - // LIMIT - // - LIMIT ALL is the same as omitting the LIMIT clause - withWindow.optional(limit) { - Limit(typedVisit(limit), withWindow) - } - } - - /** - * Create a clause for DISTRIBUTE BY. - */ - protected def withRepartitionByExpression( - ctx: QueryOrganizationContext, - expressions: Seq[Expression], - query: LogicalPlan): LogicalPlan = { - throw QueryParsingErrors.distributeByUnsupportedError(ctx) - } - - override def visitTransformQuerySpecification( - ctx: TransformQuerySpecificationContext): LogicalPlan = withOrigin(ctx) { - val from = OneRowRelation().optional(ctx.fromClause) { - visitFromClause(ctx.fromClause) - } - withTransformQuerySpecification( - ctx, - ctx.transformClause, - ctx.lateralView, - ctx.whereClause, - ctx.aggregationClause, - ctx.havingClause, - ctx.windowClause, - from) - } - - override def visitRegularQuerySpecification( - ctx: RegularQuerySpecificationContext): LogicalPlan = withOrigin(ctx) { - val from = OneRowRelation().optional(ctx.fromClause) { - visitFromClause(ctx.fromClause) - } - withSelectQuerySpecification( - ctx, - ctx.selectClause, - ctx.lateralView, - ctx.whereClause, - ctx.aggregationClause, - ctx.havingClause, - ctx.windowClause, - from) - } - - override def visitNamedExpressionSeq( - ctx: NamedExpressionSeqContext): Seq[Expression] = { - Option(ctx).toSeq - .flatMap(_.namedExpression.asScala) - .map(typedVisit[Expression]) - } - - override def visitExpressionSeq(ctx: ExpressionSeqContext): Seq[Expression] = { - Option(ctx).toSeq - .flatMap(_.expression.asScala) - .map(typedVisit[Expression]) - } - - /** - * Create a logical plan using a having clause. - */ - private def withHavingClause( - ctx: HavingClauseContext, - plan: LogicalPlan): LogicalPlan = { - // Note that we add a cast to non-predicate expressions. If the expression itself is - // already boolean, the optimizer will get rid of the unnecessary cast. - val predicate = expression(ctx.booleanExpression) match { - case p: Predicate => p - case e => Cast(e, BooleanType) - } - UnresolvedHaving(predicate, plan) - } - - /** - * Create a logical plan using a where clause. - */ - private def withWhereClause(ctx: WhereClauseContext, plan: LogicalPlan): LogicalPlan = { - Filter(expression(ctx.booleanExpression), plan) - } - - /** - * Add a hive-style transform (SELECT TRANSFORM/MAP/REDUCE) query specification to a logical plan. - */ - private def withTransformQuerySpecification( - ctx: ParserRuleContext, - transformClause: TransformClauseContext, - lateralView: java.util.List[LateralViewContext], - whereClause: WhereClauseContext, - aggregationClause: AggregationClauseContext, - havingClause: HavingClauseContext, - windowClause: WindowClauseContext, - relation: LogicalPlan): LogicalPlan = withOrigin(ctx) { - if (transformClause.setQuantifier != null) { - throw QueryParsingErrors.transformNotSupportQuantifierError(transformClause.setQuantifier) - } - // Create the attributes. - val (attributes, schemaLess) = if (transformClause.colTypeList != null) { - // Typed return columns. - (createSchema(transformClause.colTypeList).toAttributes, false) - } else if (transformClause.identifierSeq != null) { - // Untyped return columns. - val attrs = visitIdentifierSeq(transformClause.identifierSeq).map { name => - AttributeReference(name, StringType, nullable = true)() - } - (attrs, false) - } else { - ( - Seq(AttributeReference("key", StringType)(), AttributeReference("value", StringType)()), - true) - } - - val plan = visitCommonSelectQueryClausePlan( - relation, - visitExpressionSeq(transformClause.expressionSeq), - lateralView, - whereClause, - aggregationClause, - havingClause, - windowClause, - isDistinct = false) - - ScriptTransformation( - string(transformClause.script), - attributes, - plan, - withScriptIOSchema( - ctx, - transformClause.inRowFormat, - transformClause.recordWriter, - transformClause.outRowFormat, - transformClause.recordReader, - schemaLess)) - } - - /** - * Add a regular (SELECT) query specification to a logical plan. The query specification - * is the core of the logical plan, this is where sourcing (FROM clause), projection (SELECT), - * aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place. - * - * Note that query hints are ignored (both by the parser and the builder). - */ - private def withSelectQuerySpecification( - ctx: ParserRuleContext, - selectClause: SelectClauseContext, - lateralView: java.util.List[LateralViewContext], - whereClause: WhereClauseContext, - aggregationClause: AggregationClauseContext, - havingClause: HavingClauseContext, - windowClause: WindowClauseContext, - relation: LogicalPlan): LogicalPlan = withOrigin(ctx) { - val isDistinct = selectClause.setQuantifier() != null && - selectClause.setQuantifier().DISTINCT() != null - - val plan = visitCommonSelectQueryClausePlan( - relation, - visitNamedExpressionSeq(selectClause.namedExpressionSeq), - lateralView, - whereClause, - aggregationClause, - havingClause, - windowClause, - isDistinct) - - // Hint - selectClause.hints.asScala.foldRight(plan)(withHints) - } - - def visitCommonSelectQueryClausePlan( - relation: LogicalPlan, - expressions: Seq[Expression], - lateralView: java.util.List[LateralViewContext], - whereClause: WhereClauseContext, - aggregationClause: AggregationClauseContext, - havingClause: HavingClauseContext, - windowClause: WindowClauseContext, - isDistinct: Boolean): LogicalPlan = { - // Add lateral views. - val withLateralView = lateralView.asScala.foldLeft(relation)(withGenerate) - - // Add where. - val withFilter = withLateralView.optionalMap(whereClause)(withWhereClause) - - // Add aggregation or a project. - val namedExpressions = expressions.map { - case e: NamedExpression => e - case e: Expression => UnresolvedAlias(e) - } - - def createProject() = if (namedExpressions.nonEmpty) { - Project(namedExpressions, withFilter) - } else { - withFilter - } - - val withProject = if (aggregationClause == null && havingClause != null) { - if (conf.getConf(SQLConf.LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE)) { - // If the legacy conf is set, treat HAVING without GROUP BY as WHERE. - val predicate = expression(havingClause.booleanExpression) match { - case p: Predicate => p - case e => Cast(e, BooleanType) - } - Filter(predicate, createProject()) - } else { - // According to SQL standard, HAVING without GROUP BY means global aggregate. - withHavingClause(havingClause, Aggregate(Nil, namedExpressions, withFilter)) - } - } else if (aggregationClause != null) { - val aggregate = withAggregationClause(aggregationClause, namedExpressions, withFilter) - aggregate.optionalMap(havingClause)(withHavingClause) - } else { - // When hitting this branch, `having` must be null. - createProject() - } - - // Distinct - val withDistinct = if (isDistinct) { - Distinct(withProject) - } else { - withProject - } - - // Window - val withWindow = withDistinct.optionalMap(windowClause)(withWindowClause) - - withWindow - } - - // Script Transform's input/output format. - type ScriptIOFormat = - (Seq[(String, String)], Option[String], Seq[(String, String)], Option[String]) - - protected def getRowFormatDelimited(ctx: RowFormatDelimitedContext): ScriptIOFormat = { - // TODO we should use the visitRowFormatDelimited function here. However HiveScriptIOSchema - // expects a seq of pairs in which the old parsers' token names are used as keys. - // Transforming the result of visitRowFormatDelimited would be quite a bit messier than - // retrieving the key value pairs ourselves. - val entries = entry("TOK_TABLEROWFORMATFIELD", ctx.fieldsTerminatedBy) ++ - entry("TOK_TABLEROWFORMATCOLLITEMS", ctx.collectionItemsTerminatedBy) ++ - entry("TOK_TABLEROWFORMATMAPKEYS", ctx.keysTerminatedBy) ++ - entry("TOK_TABLEROWFORMATNULL", ctx.nullDefinedAs) ++ - Option(ctx.linesSeparatedBy).toSeq.map { token => - val value = string(token) - validate( - value == "\n", - s"LINES TERMINATED BY only supports newline '\\n' right now: $value", - ctx) - "TOK_TABLEROWFORMATLINES" -> value - } - - (entries, None, Seq.empty, None) - } - - /** - * Create a [[ScriptInputOutputSchema]]. - */ - protected def withScriptIOSchema( - ctx: ParserRuleContext, - inRowFormat: RowFormatContext, - recordWriter: Token, - outRowFormat: RowFormatContext, - recordReader: Token, - schemaLess: Boolean): ScriptInputOutputSchema = { - - def format(fmt: RowFormatContext): ScriptIOFormat = fmt match { - case c: RowFormatDelimitedContext => - getRowFormatDelimited(c) - - case c: RowFormatSerdeContext => - throw QueryParsingErrors.transformWithSerdeUnsupportedError(ctx) - - // SPARK-32106: When there is no definition about format, we return empty result - // to use a built-in default Serde in SparkScriptTransformationExec. - case null => - (Nil, None, Seq.empty, None) - } - - val (inFormat, inSerdeClass, inSerdeProps, reader) = format(inRowFormat) - - val (outFormat, outSerdeClass, outSerdeProps, writer) = format(outRowFormat) - - ScriptInputOutputSchema( - inFormat, - outFormat, - inSerdeClass, - outSerdeClass, - inSerdeProps, - outSerdeProps, - reader, - writer, - schemaLess) - } - - /** - * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma - * separated) relations here, these get converted into a single plan by condition-less inner join. - */ - override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) { - val from = ctx.relation.asScala.foldLeft(null: LogicalPlan) { (left, relation) => - val right = plan(relation.relationPrimary) - val join = right.optionalMap(left) { (left, right) => - if (relation.LATERAL != null) { - if (!relation.relationPrimary.isInstanceOf[AliasedQueryContext]) { - throw QueryParsingErrors.invalidLateralJoinRelationError(relation.relationPrimary) - } - LateralJoin(left, LateralSubquery(right), Inner, None) - } else { - Join(left, right, Inner, None, JoinHint.NONE) - } - } - withJoinRelations(join, relation) - } - if (ctx.pivotClause() != null) { - if (!ctx.lateralView.isEmpty) { - throw QueryParsingErrors.lateralWithPivotInFromClauseNotAllowedError(ctx) - } - withPivot(ctx.pivotClause, from) - } else { - ctx.lateralView.asScala.foldLeft(from)(withGenerate) - } - } - - /** - * Connect two queries by a Set operator. - * - * Supported Set operators are: - * - UNION [ DISTINCT | ALL ] - * - EXCEPT [ DISTINCT | ALL ] - * - MINUS [ DISTINCT | ALL ] - * - INTERSECT [DISTINCT | ALL] - */ - override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) { - val left = plan(ctx.left) - val right = plan(ctx.right) - val all = Option(ctx.setQuantifier()).exists(_.ALL != null) - ctx.operator.getType match { - case MixedFormatSqlExtendParser.UNION if all => - Union(left, right) - case MixedFormatSqlExtendParser.UNION => - Distinct(Union(left, right)) - case MixedFormatSqlExtendParser.INTERSECT if all => - Intersect(left, right, isAll = true) - case MixedFormatSqlExtendParser.INTERSECT => - Intersect(left, right, isAll = false) - case MixedFormatSqlExtendParser.EXCEPT if all => - Except(left, right, isAll = true) - case MixedFormatSqlExtendParser.EXCEPT => - Except(left, right, isAll = false) - case MixedFormatSqlExtendParser.SETMINUS if all => - Except(left, right, isAll = true) - case MixedFormatSqlExtendParser.SETMINUS => - Except(left, right, isAll = false) - } - } - - /** - * Add a [[WithWindowDefinition]] operator to a logical plan. - */ - private def withWindowClause( - ctx: WindowClauseContext, - query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - // Collect all window specifications defined in the WINDOW clause. - val baseWindowTuples = ctx.namedWindow.asScala.map { - wCtx => - (wCtx.name.getText, typedVisit[WindowSpec](wCtx.windowSpec)) - } - baseWindowTuples.groupBy(_._1).foreach { kv => - if (kv._2.size > 1) { - throw QueryParsingErrors.repetitiveWindowDefinitionError(kv._1, ctx) - } - } - val baseWindowMap = baseWindowTuples.toMap - - // Handle cases like - // window w1 as (partition by p_mfgr order by p_name - // range between 2 preceding and 2 following), - // w2 as w1 - val windowMapView = baseWindowMap.mapValues { - case WindowSpecReference(name) => - baseWindowMap.get(name) match { - case Some(spec: WindowSpecDefinition) => - spec - case Some(ref) => - throw QueryParsingErrors.invalidWindowReferenceError(name, ctx) - case None => - throw QueryParsingErrors.cannotResolveWindowReferenceError(name, ctx) - } - case spec: WindowSpecDefinition => spec - } - - // Note that mapValues creates a view instead of materialized map. We force materialization by - // mapping over identity. - WithWindowDefinition(windowMapView.map(identity).toMap, query) - } - - /** - * Add an [[Aggregate]] to a logical plan. - */ - private def withAggregationClause( - ctx: AggregationClauseContext, - selectExpressions: Seq[NamedExpression], - query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - if (ctx.groupingExpressionsWithGroupingAnalytics.isEmpty) { - val groupByExpressions = expressionList(ctx.groupingExpressions) - if (ctx.GROUPING != null) { - // GROUP BY ... GROUPING SETS (...) - // `groupByExpressions` can be non-empty for Hive compatibility. It may add extra grouping - // expressions that do not exist in GROUPING SETS (...), and the value is always null. - // For example, `SELECT a, b, c FROM ... GROUP BY a, b, c GROUPING SETS (a, b)`, the output - // of column `c` is always null. - val groupingSets = - ctx.groupingSet.asScala.map(_.expression.asScala.map(e => expression(e)).toSeq) - Aggregate( - Seq(GroupingSets(groupingSets.toSeq, groupByExpressions)), - selectExpressions, - query) - } else { - // GROUP BY .... (WITH CUBE | WITH ROLLUP)? - val mappedGroupByExpressions = if (ctx.CUBE != null) { - Seq(Cube(groupByExpressions.map(Seq(_)))) - } else if (ctx.ROLLUP != null) { - Seq(Rollup(groupByExpressions.map(Seq(_)))) - } else { - groupByExpressions - } - Aggregate(mappedGroupByExpressions, selectExpressions, query) - } - } else { - val groupByExpressions = - ctx.groupingExpressionsWithGroupingAnalytics.asScala - .map(groupByExpr => { - val groupingAnalytics = groupByExpr.groupingAnalytics - if (groupingAnalytics != null) { - visitGroupingAnalytics(groupingAnalytics) - } else { - expression(groupByExpr.expression) - } - }) - Aggregate(groupByExpressions.toSeq, selectExpressions, query) - } - } - - override def visitGroupingAnalytics( - groupingAnalytics: GroupingAnalyticsContext): BaseGroupingSets = { - val groupingSets = groupingAnalytics.groupingSet.asScala - .map(_.expression.asScala.map(e => expression(e)).toSeq) - if (groupingAnalytics.CUBE != null) { - // CUBE(A, B, (A, B), ()) is not supported. - if (groupingSets.exists(_.isEmpty)) { - throw QueryParsingErrors.invalidGroupingSetError("CUBE", groupingAnalytics) - } - Cube(groupingSets.toSeq) - } else if (groupingAnalytics.ROLLUP != null) { - // ROLLUP(A, B, (A, B), ()) is not supported. - if (groupingSets.exists(_.isEmpty)) { - throw QueryParsingErrors.invalidGroupingSetError("ROLLUP", groupingAnalytics) - } - Rollup(groupingSets.toSeq) - } else { - assert(groupingAnalytics.GROUPING != null && groupingAnalytics.SETS != null) - val groupingSets = groupingAnalytics.groupingElement.asScala.flatMap { expr => - val groupingAnalytics = expr.groupingAnalytics() - if (groupingAnalytics != null) { - visitGroupingAnalytics(groupingAnalytics).selectedGroupByExprs - } else { - Seq(expr.groupingSet().expression().asScala.map(e => expression(e)).toSeq) - } - } - GroupingSets(groupingSets.toSeq) - } - } - - /** - * Add [[UnresolvedHint]]s to a logical plan. - */ - private def withHints( - ctx: HintContext, - query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - var plan = query - ctx.hintStatements.asScala.reverse.foreach { stmt => - plan = - UnresolvedHint(stmt.hintName.getText, stmt.parameters.asScala.map(expression).toSeq, plan) - } - plan - } - - /** - * Add a [[Pivot]] to a logical plan. - */ - private def withPivot( - ctx: PivotClauseContext, - query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - val aggregates = Option(ctx.aggregates).toSeq - .flatMap(_.namedExpression.asScala) - .map(typedVisit[Expression]) - val pivotColumn = if (ctx.pivotColumn.identifiers.size == 1) { - UnresolvedAttribute.quoted(ctx.pivotColumn.identifier.getText) - } else { - CreateStruct( - ctx.pivotColumn.identifiers.asScala.map(identifier => - UnresolvedAttribute.quoted(identifier.getText)).toSeq) - } - val pivotValues = ctx.pivotValues.asScala.map(visitPivotValue) - Pivot(None, pivotColumn, pivotValues.toSeq, aggregates, query) - } - - /** - * Create a Pivot column value with or without an alias. - */ - override def visitPivotValue(ctx: PivotValueContext): Expression = withOrigin(ctx) { - val e = expression(ctx.expression) - if (ctx.identifier != null) { - Alias(e, ctx.identifier.getText)() - } else { - e - } - } - - /** - * Add a [[Generate]] (Lateral View) to a logical plan. - */ - private def withGenerate( - query: LogicalPlan, - ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) { - val expressions = expressionList(ctx.expression) - Generate( - UnresolvedGenerator(visitFunctionName(ctx.qualifiedName), expressions), - unrequiredChildIndex = Nil, - outer = ctx.OUTER != null, - // scalastyle:off caselocale - Some(ctx.tblName.getText.toLowerCase), - // scalastyle:on caselocale - ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.quoted).toSeq, - query) - } - - /** - * Create a single relation referenced in a FROM clause. This method is used when a part of the - * join condition is nested, for example: - * {{{ - * select * from t1 join (t2 cross join t3) on col1 = col2 - * }}} - */ - override def visitRelation(ctx: RelationContext): LogicalPlan = withOrigin(ctx) { - withJoinRelations(plan(ctx.relationPrimary), ctx) - } - - /** - * Join one more [[LogicalPlan]]s to the current logical plan. - */ - private def withJoinRelations(base: LogicalPlan, ctx: RelationContext): LogicalPlan = { - ctx.joinRelation.asScala.foldLeft(base) { (left, join) => - withOrigin(join) { - val baseJoinType = join.joinType match { - case null => Inner - case jt if jt.CROSS != null => Cross - case jt if jt.FULL != null => FullOuter - case jt if jt.SEMI != null => LeftSemi - case jt if jt.ANTI != null => LeftAnti - case jt if jt.LEFT != null => LeftOuter - case jt if jt.RIGHT != null => RightOuter - case _ => Inner - } - - if (join.LATERAL != null && !join.right.isInstanceOf[AliasedQueryContext]) { - throw QueryParsingErrors.invalidLateralJoinRelationError(join.right) - } - - // Resolve the join type and join condition - val (joinType, condition) = Option(join.joinCriteria) match { - case Some(c) if c.USING != null => - if (join.LATERAL != null) { - throw QueryParsingErrors.lateralJoinWithUsingJoinUnsupportedError(ctx) - } - (UsingJoin(baseJoinType, visitIdentifierList(c.identifierList)), None) - case Some(c) if c.booleanExpression != null => - (baseJoinType, Option(expression(c.booleanExpression))) - case Some(c) => - throw new IllegalStateException(s"Unimplemented joinCriteria: $c") - case None if join.NATURAL != null => - if (join.LATERAL != null) { - throw QueryParsingErrors.lateralJoinWithNaturalJoinUnsupportedError(ctx) - } - if (baseJoinType == Cross) { - throw QueryParsingErrors.naturalCrossJoinUnsupportedError(ctx) - } - (NaturalJoin(baseJoinType), None) - case None => - (baseJoinType, None) - } - if (join.LATERAL != null) { - if (!Seq(Inner, Cross, LeftOuter).contains(joinType)) { - throw QueryParsingErrors.unsupportedLateralJoinTypeError(ctx, joinType.sql) - } - LateralJoin(left, LateralSubquery(plan(join.right)), joinType, condition) - } else { - Join(left, plan(join.right), joinType, condition, JoinHint.NONE) - } - } - } - } - - /** - * Add a [[Sample]] to a logical plan. - * - * This currently supports the following sampling methods: - * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows. - * - TABLESAMPLE(x PERCENT) [REPEATABLE (y)]: Sample the table down to the given percentage with - * seed 'y'. Note that percentages are defined as a number between 0 and 100. - * - TABLESAMPLE(BUCKET x OUT OF y) [REPEATABLE (z)]: Sample the table down to a 'x' divided by - * 'y' fraction with seed 'z'. - */ - private def withSample(ctx: SampleContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - // Create a sampled plan if we need one. - def sample(fraction: Double): Sample = { - // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling - // function takes X PERCENT as the input and the range of X is [0, 100], we need to - // adjust the fraction. - val eps = RandomSampler.roundingEpsilon - validate( - fraction >= 0.0 - eps && fraction <= 1.0 + eps, - s"Sampling fraction ($fraction) must be on interval [0, 1]", - ctx) - Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query) - } - - if (ctx.sampleMethod() == null) { - throw QueryParsingErrors.emptyInputForTableSampleError(ctx) - } - - ctx.sampleMethod() match { - case ctx: SampleByRowsContext => - Limit(expression(ctx.expression), query) - - case ctx: SampleByPercentileContext => - val fraction = ctx.percentage.getText.toDouble - val sign = if (ctx.negativeSign == null) 1 else -1 - sample(sign * fraction / 100.0d) - - case ctx: SampleByBytesContext => - val bytesStr = ctx.bytes.getText - if (bytesStr.matches("[0-9]+[bBkKmMgG]")) { - throw QueryParsingErrors.tableSampleByBytesUnsupportedError("byteLengthLiteral", ctx) - } else { - throw QueryParsingErrors.invalidByteLengthLiteralError(bytesStr, ctx) - } - - case ctx: SampleByBucketContext if ctx.ON() != null => - if (ctx.identifier != null) { - throw QueryParsingErrors.tableSampleByBytesUnsupportedError( - "BUCKET x OUT OF y ON colname", - ctx) - } else { - throw QueryParsingErrors.tableSampleByBytesUnsupportedError( - "BUCKET x OUT OF y ON function", - ctx) - } - - case ctx: SampleByBucketContext => - sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble) - } - } - - /** - * Create a logical plan for a sub-query. - */ - override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) { - plan(ctx.query) - } - - /** - * Create an un-aliased table reference. This is typically used for top-level table references, - * for example: - * {{{ - * INSERT INTO db.tbl2 - * TABLE db.tbl1 - * }}} - */ - override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) { - UnresolvedRelation(visitMultipartIdentifier(ctx.multipartIdentifier)) - } - - /** - * Create an aliased table reference. This is typically used in FROM clauses. - */ - override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) { - val tableId = visitMultipartIdentifier(ctx.multipartIdentifier) - val table = mayApplyAliasPlan(ctx.tableAlias, UnresolvedRelation(tableId)) - table.optionalMap(ctx.sample)(withSample) - } - - protected def getFunctionIdentifier(ctx: FunctionNameContext): FunctionIdentifier = { - if (ctx.qualifiedName != null) { - visitFunctionName(ctx.qualifiedName) - } else { - FunctionIdentifier(ctx.getText, None) - } - } - - /** - * Create a table-valued function call with arguments, e.g. range(1000) - */ - override def visitTableValuedFunction(ctx: TableValuedFunctionContext): LogicalPlan = - withOrigin(ctx) { - val func = ctx.functionTable - val aliases = if (func.tableAlias.identifierList != null) { - visitIdentifierList(func.tableAlias.identifierList) - } else { - Seq.empty - } - val name = getFunctionIdentifier(func.functionName) - if (name.database.nonEmpty) { - operationNotAllowed(s"table valued function cannot specify database name: $name", ctx) - } - - val tvf = UnresolvedTableValuedFunction( - name, - func.expression.asScala.map(expression).toSeq, - aliases) - tvf.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) - } - - /** - * Create an inline table (a virtual table in Hive parlance). - */ - override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) { - // Get the backing expressions. - val rows = ctx.expression.asScala.map { e => - expression(e) match { - // inline table comes in two styles: - // style 1: values (1), (2), (3) -- multiple columns are supported - // style 2: values 1, 2, 3 -- only a single column is supported here - case struct: CreateNamedStruct => struct.valExprs // style 1 - case child => Seq(child) // style 2 - } - } - - val aliases = if (ctx.tableAlias.identifierList != null) { - visitIdentifierList(ctx.tableAlias.identifierList) - } else { - Seq.tabulate(rows.head.size)(i => s"col${i + 1}") - } - - val table = UnresolvedInlineTable(aliases, rows.toSeq) - table.optionalMap(ctx.tableAlias.strictIdentifier)(aliasPlan) - } - - /** - * Create an alias (SubqueryAlias) for a join relation. This is practically the same as - * visitAliasedQuery and visitNamedExpression, ANTLR4 however requires us to use 3 different - * hooks. We could add alias names for output columns, for example: - * {{{ - * SELECT a, b, c, d FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) - * }}} - */ - override def visitAliasedRelation(ctx: AliasedRelationContext): LogicalPlan = withOrigin(ctx) { - val relation = plan(ctx.relation).optionalMap(ctx.sample)(withSample) - mayApplyAliasPlan(ctx.tableAlias, relation) - } - - /** - * Create an alias (SubqueryAlias) for a sub-query. This is practically the same as - * visitAliasedRelation and visitNamedExpression, ANTLR4 however requires us to use 3 different - * hooks. We could add alias names for output columns, for example: - * {{{ - * SELECT col1, col2 FROM testData AS t(col1, col2) - * }}} - */ - override def visitAliasedQuery(ctx: AliasedQueryContext): LogicalPlan = withOrigin(ctx) { - val relation = plan(ctx.query).optionalMap(ctx.sample)(withSample) - if (ctx.tableAlias.strictIdentifier == null) { - // For un-aliased subqueries, use a default alias name that is not likely to conflict with - // normal subquery names, so that parent operators can only access the columns in subquery by - // unqualified names. Users can still use this special qualifier to access columns if they - // know it, but that's not recommended. - SubqueryAlias("__auto_generated_subquery_name", relation) - } else { - mayApplyAliasPlan(ctx.tableAlias, relation) - } - } - - /** - * Create an alias ([[SubqueryAlias]]) for a [[LogicalPlan]]. - */ - private def aliasPlan(alias: ParserRuleContext, plan: LogicalPlan): LogicalPlan = { - SubqueryAlias(alias.getText, plan) - } - - /** - * If aliases specified in a FROM clause, create a subquery alias ([[SubqueryAlias]]) and - * column aliases for a [[LogicalPlan]]. - */ - private def mayApplyAliasPlan(tableAlias: TableAliasContext, plan: LogicalPlan): LogicalPlan = { - if (tableAlias.strictIdentifier != null) { - val alias = tableAlias.strictIdentifier.getText - if (tableAlias.identifierList != null) { - val columnNames = visitIdentifierList(tableAlias.identifierList) - SubqueryAlias(alias, UnresolvedSubqueryColumnAliases(columnNames, plan)) - } else { - SubqueryAlias(alias, plan) - } - } else { - plan - } - } - - /** - * Create a Sequence of Strings for a parenthesis enclosed alias list. - */ - override def visitIdentifierList(ctx: IdentifierListContext): Seq[String] = withOrigin(ctx) { - visitIdentifierSeq(ctx.identifierSeq) - } - - /** - * Create a Sequence of Strings for an identifier list. - */ - override def visitIdentifierSeq(ctx: IdentifierSeqContext): Seq[String] = withOrigin(ctx) { - ctx.ident.asScala.map(_.getText).toSeq - } - - /** - * Create a multi-part identifier. - */ - override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = - withOrigin(ctx) { - ctx.parts.asScala.map(_.getText).toSeq - } - - /* ******************************************************************************************** - * Expression parsing - * ******************************************************************************************** */ - - /** - * Create an expression from the given context. This method just passes the context on to the - * visitor and only takes care of typing (We assume that the visitor returns an Expression here). - */ - protected def expression(ctx: ParserRuleContext): Expression = typedVisit(ctx) - - /** - * Create sequence of expressions from the given sequence of contexts. - */ - private def expressionList(trees: java.util.List[ExpressionContext]): Seq[Expression] = { - trees.asScala.map(expression).toSeq - } - - /** - * Create a star (i.e. all) expression; this selects all elements (in the specified object). - * Both un-targeted (global) and targeted aliases are supported. - */ - override def visitStar(ctx: StarContext): Expression = withOrigin(ctx) { - UnresolvedStar(Option(ctx.qualifiedName()).map(_.identifier.asScala.map(_.getText).toSeq)) - } - - /** - * Create an aliased expression if an alias is specified. Both single and multi-aliases are - * supported. - */ - override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) { - val e = expression(ctx.expression) - if (ctx.name != null) { - Alias(e, ctx.name.getText)() - } else if (ctx.identifierList != null) { - MultiAlias(e, visitIdentifierList(ctx.identifierList)) - } else { - e - } - } - - /** - * Combine a number of boolean expressions into a balanced expression tree. These expressions are - * either combined by a logical [[And]] or a logical [[Or]]. - * - * A balanced binary tree is created because regular left recursive trees cause considerable - * performance degradations and can cause stack overflows. - */ - override def visitLogicalBinary(ctx: LogicalBinaryContext): Expression = withOrigin(ctx) { - val expressionType = ctx.operator.getType - val expressionCombiner = expressionType match { - case MixedFormatSqlExtendParser.AND => And.apply _ - case MixedFormatSqlExtendParser.OR => Or.apply _ - } - - // Collect all similar left hand contexts. - val contexts = ArrayBuffer(ctx.right) - var current = ctx.left - - def collectContexts: Boolean = current match { - case lbc: LogicalBinaryContext if lbc.operator.getType == expressionType => - contexts += lbc.right - current = lbc.left - true - case _ => - contexts += current - false - } - - while (collectContexts) { - // No body - all updates take place in the collectContexts. - } - - // Reverse the contexts to have them in the same sequence as in the SQL statement & turn them - // into expressions. - val expressions = contexts.reverseMap(expression) - - // Create a balanced tree. - def reduceToExpressionTree(low: Int, high: Int): Expression = high - low match { - case 0 => - expressions(low) - case 1 => - expressionCombiner(expressions(low), expressions(high)) - case x => - val mid = low + x / 2 - expressionCombiner( - reduceToExpressionTree(low, mid), - reduceToExpressionTree(mid + 1, high)) - } - - reduceToExpressionTree(0, expressions.size - 1) - } - - /** - * Invert a boolean expression. - */ - override def visitLogicalNot(ctx: LogicalNotContext): Expression = withOrigin(ctx) { - Not(expression(ctx.booleanExpression())) - } - - /** - * Create a filtering correlated sub-query (EXISTS). - */ - override def visitExists(ctx: ExistsContext): Expression = { - Exists(plan(ctx.query)) - } - - /** - * Create a comparison expression. This compares two expressions. The following comparison - * operators are supported: - * - Equal: '=' or '==' - * - Null-safe Equal: '<=>' - * - Not Equal: '<>' or '!=' - * - Less than: '<' - * - Less then or Equal: '<=' - * - Greater than: '>' - * - Greater then or Equal: '>=' - */ - override def visitComparison(ctx: ComparisonContext): Expression = withOrigin(ctx) { - val left = expression(ctx.left) - val right = expression(ctx.right) - val operator = ctx.comparisonOperator().getChild(0).asInstanceOf[TerminalNode] - operator.getSymbol.getType match { - case MixedFormatSqlExtendParser.EQ => - EqualTo(left, right) - case MixedFormatSqlExtendParser.NSEQ => - EqualNullSafe(left, right) - case MixedFormatSqlExtendParser.NEQ | MixedFormatSqlExtendParser.NEQJ => - Not(EqualTo(left, right)) - case MixedFormatSqlExtendParser.LT => - LessThan(left, right) - case MixedFormatSqlExtendParser.LTE => - LessThanOrEqual(left, right) - case MixedFormatSqlExtendParser.GT => - GreaterThan(left, right) - case MixedFormatSqlExtendParser.GTE => - GreaterThanOrEqual(left, right) - } - } - - /** - * Create a predicated expression. A predicated expression is a normal expression with a - * predicate attached to it, for example: - * {{{ - * a + 1 IS NULL - * }}} - */ - override def visitPredicated(ctx: PredicatedContext): Expression = withOrigin(ctx) { - val e = expression(ctx.valueExpression) - if (ctx.predicate != null) { - withPredicate(e, ctx.predicate) - } else { - e - } - } - - /** - * Add a predicate to the given expression. Supported expressions are: - * - (NOT) BETWEEN - * - (NOT) IN - * - (NOT) (LIKE | ILIKE) (ANY | SOME | ALL) - * - (NOT) RLIKE - * - IS (NOT) NULL. - * - IS (NOT) (TRUE | FALSE | UNKNOWN) - * - IS (NOT) DISTINCT FROM - */ - private def withPredicate(e: Expression, ctx: PredicateContext): Expression = withOrigin(ctx) { - // Invert a predicate if it has a valid NOT clause. - def invertIfNotDefined(e: Expression): Expression = ctx.NOT match { - case null => e - case not => Not(e) - } - - def getValueExpressions(e: Expression): Seq[Expression] = e match { - case c: CreateNamedStruct => c.valExprs - case other => Seq(other) - } - - // Create the predicate. - ctx.kind.getType match { - case MixedFormatSqlExtendParser.BETWEEN => - // BETWEEN is translated to lower <= e && e <= upper - invertIfNotDefined(And( - GreaterThanOrEqual(e, expression(ctx.lower)), - LessThanOrEqual(e, expression(ctx.upper)))) - case MixedFormatSqlExtendParser.IN if ctx.query != null => - invertIfNotDefined(InSubquery(getValueExpressions(e), ListQuery(plan(ctx.query)))) - case MixedFormatSqlExtendParser.IN => - invertIfNotDefined(In(e, ctx.expression.asScala.map(expression).toSeq)) - case MixedFormatSqlExtendParser.LIKE => - Option(ctx.quantifier).map(_.getType) match { - case Some(MixedFormatSqlExtendParser.ANY) | Some(MixedFormatSqlExtendParser.SOME) => - validate(!ctx.expression.isEmpty, "Expected something between '(' and ')'.", ctx) - val expressions = expressionList(ctx.expression) - if (expressions.forall(_.foldable) && expressions.forall(_.dataType == StringType)) { - // If there are many pattern expressions, will throw StackOverflowError. - // So we use LikeAny or NotLikeAny instead. - val patterns = expressions.map(_.eval(EmptyRow).asInstanceOf[UTF8String]) - ctx.NOT match { - case null => LikeAny(e, patterns) - case _ => NotLikeAny(e, patterns) - } - } else { - ctx.expression.asScala.map(expression) - .map(p => invertIfNotDefined(new Like(e, p))).toSeq.reduceLeft(Or) - } - case Some(MixedFormatSqlExtendParser.ALL) => - validate(!ctx.expression.isEmpty, "Expected something between '(' and ')'.", ctx) - val expressions = expressionList(ctx.expression) - if (expressions.forall(_.foldable) && expressions.forall(_.dataType == StringType)) { - // If there are many pattern expressions, will throw StackOverflowError. - // So we use LikeAll or NotLikeAll instead. - val patterns = expressions.map(_.eval(EmptyRow).asInstanceOf[UTF8String]) - ctx.NOT match { - case null => LikeAll(e, patterns) - case _ => NotLikeAll(e, patterns) - } - } else { - ctx.expression.asScala.map(expression) - .map(p => invertIfNotDefined(new Like(e, p))).toSeq.reduceLeft(And) - } - case _ => - val escapeChar = Option(ctx.escapeChar).map(string).map { str => - if (str.length != 1) { - throw QueryParsingErrors.invalidEscapeStringError(ctx) - } - str.charAt(0) - }.getOrElse('\\') - invertIfNotDefined(Like(e, expression(ctx.pattern), escapeChar)) - } - case MixedFormatSqlExtendParser.RLIKE => - invertIfNotDefined(RLike(e, expression(ctx.pattern))) - case MixedFormatSqlExtendParser.NULL if ctx.NOT != null => - IsNotNull(e) - case MixedFormatSqlExtendParser.NULL => - IsNull(e) - case MixedFormatSqlExtendParser.TRUE => ctx.NOT match { - case null => EqualNullSafe(e, Literal(true)) - case _ => Not(EqualNullSafe(e, Literal(true))) - } - case MixedFormatSqlExtendParser.FALSE => ctx.NOT match { - case null => EqualNullSafe(e, Literal(false)) - case _ => Not(EqualNullSafe(e, Literal(false))) - } - case MixedFormatSqlExtendParser.UNKNOWN => ctx.NOT match { - case null => IsUnknown(e) - case _ => IsNotUnknown(e) - } - case MixedFormatSqlExtendParser.DISTINCT if ctx.NOT != null => - EqualNullSafe(e, expression(ctx.right)) - case MixedFormatSqlExtendParser.DISTINCT => - Not(EqualNullSafe(e, expression(ctx.right))) - } - } - - /** - * Create a binary arithmetic expression. The following arithmetic operators are supported: - * - Multiplication: '*' - * - Division: '/' - * - Hive Long Division: 'DIV' - * - Modulo: '%' - * - Addition: '+' - * - Subtraction: '-' - * - Binary AND: '&' - * - Binary XOR - * - Binary OR: '|' - */ - override def visitArithmeticBinary(ctx: ArithmeticBinaryContext): Expression = withOrigin(ctx) { - val left = expression(ctx.left) - val right = expression(ctx.right) - ctx.operator.getType match { - case MixedFormatSqlExtendParser.ASTERISK => - Multiply(left, right) - case MixedFormatSqlExtendParser.SLASH => - Divide(left, right) - case MixedFormatSqlExtendParser.PERCENT => - Remainder(left, right) - case MixedFormatSqlExtendParser.DIV => - IntegralDivide(left, right) - case MixedFormatSqlExtendParser.PLUS => - Add(left, right) - case MixedFormatSqlExtendParser.MINUS => - Subtract(left, right) - case MixedFormatSqlExtendParser.CONCAT_PIPE => - Concat(left :: right :: Nil) - case MixedFormatSqlExtendParser.AMPERSAND => - BitwiseAnd(left, right) - case MixedFormatSqlExtendParser.HAT => - BitwiseXor(left, right) - case MixedFormatSqlExtendParser.PIPE => - BitwiseOr(left, right) - } - } - - /** - * Create a unary arithmetic expression. The following arithmetic operators are supported: - * - Plus: '+' - * - Minus: '-' - * - Bitwise Not: '~' - */ - override def visitArithmeticUnary(ctx: ArithmeticUnaryContext): Expression = withOrigin(ctx) { - val value = expression(ctx.valueExpression) - ctx.operator.getType match { - case MixedFormatSqlExtendParser.PLUS => - UnaryPositive(value) - case MixedFormatSqlExtendParser.MINUS => - UnaryMinus(value) - case MixedFormatSqlExtendParser.TILDE => - BitwiseNot(value) - } - } - - override def visitCurrentLike(ctx: CurrentLikeContext): Expression = withOrigin(ctx) { - if (conf.ansiEnabled) { - ctx.name.getType match { - case MixedFormatSqlExtendParser.CURRENT_DATE => - CurrentDate() - case MixedFormatSqlExtendParser.CURRENT_TIMESTAMP => - CurrentTimestamp() - case MixedFormatSqlExtendParser.CURRENT_USER => - CurrentUser() - } - } else { - // If the parser is not in ansi mode, we should return `UnresolvedAttribute`, in case there - // are columns named `CURRENT_DATE` or `CURRENT_TIMESTAMP`. - UnresolvedAttribute.quoted(ctx.name.getText) - } - } - - /** - * Create a [[Cast]] expression. - */ - override def visitCast(ctx: CastContext): Expression = withOrigin(ctx) { - val rawDataType = typedVisit[DataType](ctx.dataType()) - val dataType = CharVarcharUtils.replaceCharVarcharWithStringForCast(rawDataType) - val cast = ctx.name.getType match { - case MixedFormatSqlExtendParser.CAST => - Cast(expression(ctx.expression), dataType) - - case MixedFormatSqlExtendParser.TRY_CAST => - TryCast(expression(ctx.expression), dataType) - } - cast.setTagValue(Cast.USER_SPECIFIED_CAST, true) - cast - } - - /** - * Create a [[CreateStruct]] expression. - */ - override def visitStruct(ctx: StructContext): Expression = withOrigin(ctx) { - CreateStruct.create(ctx.argument.asScala.map(expression).toSeq) - } - - /** - * Create a [[First]] expression. - */ - override def visitFirst(ctx: FirstContext): Expression = withOrigin(ctx) { - val ignoreNullsExpr = ctx.IGNORE != null - First(expression(ctx.expression), ignoreNullsExpr).toAggregateExpression() - } - - /** - * Create a [[Last]] expression. - */ - override def visitLast(ctx: LastContext): Expression = withOrigin(ctx) { - val ignoreNullsExpr = ctx.IGNORE != null - Last(expression(ctx.expression), ignoreNullsExpr).toAggregateExpression() - } - - /** - * Create a Position expression. - */ - override def visitPosition(ctx: PositionContext): Expression = withOrigin(ctx) { - new StringLocate(expression(ctx.substr), expression(ctx.str)) - } - - /** - * Create a Extract expression. - */ - override def visitExtract(ctx: ExtractContext): Expression = withOrigin(ctx) { - val arguments = Seq(Literal(ctx.field.getText), expression(ctx.source)) - UnresolvedFunction("extract", arguments, isDistinct = false) - } - - /** - * Create a Substring/Substr expression. - */ - override def visitSubstring(ctx: SubstringContext): Expression = withOrigin(ctx) { - if (ctx.len != null) { - Substring(expression(ctx.str), expression(ctx.pos), expression(ctx.len)) - } else { - new Substring(expression(ctx.str), expression(ctx.pos)) - } - } - - /** - * Create a Trim expression. - */ - override def visitTrim(ctx: TrimContext): Expression = withOrigin(ctx) { - val srcStr = expression(ctx.srcStr) - val trimStr = Option(ctx.trimStr).map(expression) - Option(ctx.trimOption).map(_.getType).getOrElse(MixedFormatSqlExtendParser.BOTH) match { - case MixedFormatSqlExtendParser.BOTH => - StringTrim(srcStr, trimStr) - case MixedFormatSqlExtendParser.LEADING => - StringTrimLeft(srcStr, trimStr) - case MixedFormatSqlExtendParser.TRAILING => - StringTrimRight(srcStr, trimStr) - case other => - throw QueryParsingErrors.trimOptionUnsupportedError(other, ctx) - } - } - - /** - * Create a Overlay expression. - */ - override def visitOverlay(ctx: OverlayContext): Expression = withOrigin(ctx) { - val input = expression(ctx.input) - val replace = expression(ctx.replace) - val position = expression(ctx.position) - val lengthOpt = Option(ctx.length).map(expression) - lengthOpt match { - case Some(length) => Overlay(input, replace, position, length) - case None => new Overlay(input, replace, position) - } - } - - /** - * Create a (windowed) Function expression. - */ - override def visitFunctionCall(ctx: FunctionCallContext): Expression = withOrigin(ctx) { - // Create the function call. - val name = ctx.functionName.getText - val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) - // Call `toSeq`, otherwise `ctx.argument.asScala.map(expression)` is `Buffer` in Scala 2.13 - val arguments = ctx.argument.asScala.map(expression).toSeq match { - case Seq(UnresolvedStar(None)) - if name.toLowerCase(Locale.ROOT) == "count" && !isDistinct => - // Transform COUNT(*) into COUNT(1). - Seq(Literal(1)) - case expressions => - expressions - } - val filter = Option(ctx.where).map(expression(_)) - val ignoreNulls = - Option(ctx.nullsOption).map(_.getType == MixedFormatSqlExtendParser.IGNORE).getOrElse(false) - val function = UnresolvedFunction( - getFunctionMultiparts(ctx.functionName), - arguments, - isDistinct, - filter, - ignoreNulls) - - // Check if the function is evaluated in a windowed context. - ctx.windowSpec match { - case spec: WindowRefContext => - UnresolvedWindowExpression(function, visitWindowRef(spec)) - case spec: WindowDefContext => - WindowExpression(function, visitWindowDef(spec)) - case _ => function - } - } - - /** - * Create a function database (optional) and name pair. - */ - protected def visitFunctionName(ctx: QualifiedNameContext): FunctionIdentifier = { - visitFunctionName(ctx, ctx.identifier().asScala.map(_.getText).toSeq) - } - - /** - * Create a function database (optional) and name pair. - */ - private def visitFunctionName(ctx: ParserRuleContext, texts: Seq[String]): FunctionIdentifier = { - texts match { - case Seq(db, fn) => FunctionIdentifier(fn, Option(db)) - case Seq(fn) => FunctionIdentifier(fn, None) - case other => - throw QueryParsingErrors.functionNameUnsupportedError(texts.mkString("."), ctx) - } - } - - protected def getFunctionMultiparts(ctx: FunctionNameContext): Seq[String] = { - if (ctx.qualifiedName != null) { - ctx.qualifiedName().identifier().asScala.map(_.getText).toSeq - } else { - Seq(ctx.getText) - } - } - - /** - * Create an [[LambdaFunction]]. - */ - override def visitLambda(ctx: LambdaContext): Expression = withOrigin(ctx) { - val arguments = ctx.identifier().asScala.map { name => - UnresolvedNamedLambdaVariable(UnresolvedAttribute.quoted(name.getText).nameParts) - } - val function = expression(ctx.expression).transformUp { - case a: UnresolvedAttribute => UnresolvedNamedLambdaVariable(a.nameParts) - } - LambdaFunction(function, arguments.toSeq) - } - - /** - * Create a reference to a window frame, i.e. [[WindowSpecReference]]. - */ - override def visitWindowRef(ctx: WindowRefContext): WindowSpecReference = withOrigin(ctx) { - WindowSpecReference(ctx.name.getText) - } - - /** - * Create a window definition, i.e. [[WindowSpecDefinition]]. - */ - override def visitWindowDef(ctx: WindowDefContext): WindowSpecDefinition = withOrigin(ctx) { - // CLUSTER BY ... | PARTITION BY ... ORDER BY ... - val partition = ctx.partition.asScala.map(expression) - val order = ctx.sortItem.asScala.map(visitSortItem) - - // RANGE/ROWS BETWEEN ... - val frameSpecOption = Option(ctx.windowFrame).map { frame => - val frameType = frame.frameType.getType match { - case MixedFormatSqlExtendParser.RANGE => RangeFrame - case MixedFormatSqlExtendParser.ROWS => RowFrame - } - - SpecifiedWindowFrame( - frameType, - visitFrameBound(frame.start), - Option(frame.end).map(visitFrameBound).getOrElse(CurrentRow)) - } - - WindowSpecDefinition( - partition.toSeq, - order.toSeq, - frameSpecOption.getOrElse(UnspecifiedFrame)) - } - - /** - * Create or resolve a frame boundary expressions. - */ - override def visitFrameBound(ctx: FrameBoundContext): Expression = withOrigin(ctx) { - def value: Expression = { - val e = expression(ctx.expression) - validate(e.resolved && e.foldable, "Frame bound value must be a literal.", ctx) - e - } - - ctx.boundType.getType match { - case MixedFormatSqlExtendParser.PRECEDING if ctx.UNBOUNDED != null => - UnboundedPreceding - case MixedFormatSqlExtendParser.PRECEDING => - UnaryMinus(value) - case MixedFormatSqlExtendParser.CURRENT => - CurrentRow - case MixedFormatSqlExtendParser.FOLLOWING if ctx.UNBOUNDED != null => - UnboundedFollowing - case MixedFormatSqlExtendParser.FOLLOWING => - value - } - } - - /** - * Create a [[CreateStruct]] expression. - */ - override def visitRowConstructor(ctx: RowConstructorContext): Expression = withOrigin(ctx) { - CreateStruct(ctx.namedExpression().asScala.map(expression).toSeq) - } - - /** - * Create a [[ScalarSubquery]] expression. - */ - override def visitSubqueryExpression( - ctx: SubqueryExpressionContext): Expression = withOrigin(ctx) { - ScalarSubquery(plan(ctx.query)) - } - - /** - * Create a value based [[CaseWhen]] expression. This has the following SQL form: - * {{{ - * CASE [expression] - * WHEN [value] THEN [expression] - * ... - * ELSE [expression] - * END - * }}} - */ - override def visitSimpleCase(ctx: SimpleCaseContext): Expression = withOrigin(ctx) { - val e = expression(ctx.value) - val branches = ctx.whenClause.asScala.map { wCtx => - (EqualTo(e, expression(wCtx.condition)), expression(wCtx.result)) - } - CaseWhen(branches.toSeq, Option(ctx.elseExpression).map(expression)) - } - - /** - * Create a condition based [[CaseWhen]] expression. This has the following SQL syntax: - * {{{ - * CASE - * WHEN [predicate] THEN [expression] - * ... - * ELSE [expression] - * END - * }}} - * - * @param ctx the parse tree - */ - override def visitSearchedCase(ctx: SearchedCaseContext): Expression = withOrigin(ctx) { - val branches = ctx.whenClause.asScala.map { wCtx => - (expression(wCtx.condition), expression(wCtx.result)) - } - CaseWhen(branches.toSeq, Option(ctx.elseExpression).map(expression)) - } - - /** - * Currently only regex in expressions of SELECT statements are supported; in other - * places, e.g., where `(a)?+.+` = 2, regex are not meaningful. - */ - private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) { - var parent = ctx.getParent - while (parent != null) { - if (parent.isInstanceOf[NamedExpressionContext]) return true - parent = parent.getParent - } - return false - } - - /** - * Returns whether the pattern is a regex expression (instead of a normal - * string). Normal string is a string with all alphabets/digits and "_". - */ - private def isRegex(pattern: String): Boolean = { - pattern.exists(p => !Character.isLetterOrDigit(p) && p != '_') - } - - /** - * Create a dereference expression. The return type depends on the type of the parent. - * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or - * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression, - * it can be [[UnresolvedExtractValue]]. - */ - override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) { - val attr = ctx.fieldName.getText - expression(ctx.base) match { - case unresolved_attr @ UnresolvedAttribute(nameParts) => - ctx.fieldName.getStart.getText match { - case escapedIdentifier(columnNameRegex) - if conf.supportQuotedRegexColumnName && - isRegex(columnNameRegex) && canApplyRegex(ctx) => - UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name), conf.caseSensitiveAnalysis) - case _ => - UnresolvedAttribute(nameParts :+ attr) - } - case e => - UnresolvedExtractValue(e, Literal(attr)) - } - } - - /** - * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex - * quoted in `` - */ - override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { - ctx.getStart.getText match { - case escapedIdentifier(columnNameRegex) - if conf.supportQuotedRegexColumnName && - isRegex(columnNameRegex) && canApplyRegex(ctx) => - UnresolvedRegex(columnNameRegex, None, conf.caseSensitiveAnalysis) - case _ => - UnresolvedAttribute.quoted(ctx.getText) - } - - } - - /** - * Create an [[UnresolvedExtractValue]] expression, this is used for subscript access to an array. - */ - override def visitSubscript(ctx: SubscriptContext): Expression = withOrigin(ctx) { - UnresolvedExtractValue(expression(ctx.value), expression(ctx.index)) - } - - /** - * Create an expression for an expression between parentheses. This is need because the ANTLR - * visitor cannot automatically convert the nested context into an expression. - */ - override def visitParenthesizedExpression( - ctx: ParenthesizedExpressionContext): Expression = withOrigin(ctx) { - expression(ctx.expression) - } - - /** - * Create a [[SortOrder]] expression. - */ - override def visitSortItem(ctx: SortItemContext): SortOrder = withOrigin(ctx) { - val direction = if (ctx.DESC != null) { - Descending - } else { - Ascending - } - val nullOrdering = if (ctx.FIRST != null) { - NullsFirst - } else if (ctx.LAST != null) { - NullsLast - } else { - direction.defaultNullOrdering - } - SortOrder(expression(ctx.expression), direction, nullOrdering, Seq.empty) - } - - /** - * Create a typed Literal expression. A typed literal has the following SQL syntax: - * {{{ - * [TYPE] '[VALUE]' - * }}} - * Currently Date, Timestamp, Interval and Binary typed literals are supported. - */ - override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) { - val value = string(ctx.STRING) - val valueType = ctx.identifier.getText.toUpperCase(Locale.ROOT) - - def toLiteral[T](f: UTF8String => Option[T], t: DataType): Literal = { - f(UTF8String.fromString(value)).map(Literal(_, t)).getOrElse { - throw QueryParsingErrors.cannotParseValueTypeError(valueType, value, ctx) - } - } - - def constructTimestampLTZLiteral(value: String): Literal = { - val zoneId = getZoneId(conf.sessionLocalTimeZone) - val specialTs = convertSpecialTimestamp(value, zoneId).map(Literal(_, TimestampType)) - specialTs.getOrElse(toLiteral(stringToTimestamp(_, zoneId), TimestampType)) - } - - try { - valueType match { - case "DATE" => - val zoneId = getZoneId(conf.sessionLocalTimeZone) - val specialDate = convertSpecialDate(value, zoneId).map(Literal(_, DateType)) - specialDate.getOrElse(toLiteral(stringToDate, DateType)) - case "TIMESTAMP_NTZ" if isTesting => - convertSpecialTimestampNTZ(value, getZoneId(conf.sessionLocalTimeZone)) - .map(Literal(_, TimestampNTZType)) - .getOrElse(toLiteral(stringToTimestampWithoutTimeZone, TimestampNTZType)) - case "TIMESTAMP_LTZ" if isTesting => - constructTimestampLTZLiteral(value) - case "TIMESTAMP" => - SQLConf.get.timestampType match { - case TimestampNTZType => - convertSpecialTimestampNTZ(value, getZoneId(conf.sessionLocalTimeZone)) - .map(Literal(_, TimestampNTZType)) - .getOrElse { - val containsTimeZonePart = - DateTimeUtils.parseTimestampString(UTF8String.fromString(value))._2.isDefined - // If the input string contains time zone part, return a timestamp with local time - // zone literal. - if (containsTimeZonePart) { - constructTimestampLTZLiteral(value) - } else { - toLiteral(stringToTimestampWithoutTimeZone, TimestampNTZType) - } - } - - case TimestampType => - constructTimestampLTZLiteral(value) - } - - case "INTERVAL" => - val interval = - try { - IntervalUtils.stringToInterval(UTF8String.fromString(value)) - } catch { - case e: IllegalArgumentException => - val ex = QueryParsingErrors.cannotParseIntervalValueError(value, ctx) - ex.setStackTrace(e.getStackTrace) - throw ex - } - if (!conf.legacyIntervalEnabled) { - val units = value - .split("\\s") - .map(_.toLowerCase(Locale.ROOT).stripSuffix("s")) - .filter(s => s != "interval" && s.matches("[a-z]+")) - constructMultiUnitsIntervalLiteral(ctx, interval, units) - } else { - Literal(interval, CalendarIntervalType) - } - case "X" => - val padding = if (value.length % 2 != 0) "0" else "" - try { - Literal(Hex.decodeHex(padding + value)) - } catch { - case _: DecoderException => - throw new IllegalArgumentException( - s"contains illegal character for hexBinary: $padding$value"); - } - case other => - throw QueryParsingErrors.literalValueTypeUnsupportedError(other, ctx) - } - } catch { - case e: IllegalArgumentException => - throw QueryParsingErrors.parsingValueTypeError(e, valueType, ctx) - } - } - - /** - * Create a NULL literal expression. - */ - override def visitNullLiteral(ctx: NullLiteralContext): Literal = withOrigin(ctx) { - Literal(null) - } - - /** - * Create a Boolean literal expression. - */ - override def visitBooleanLiteral(ctx: BooleanLiteralContext): Literal = withOrigin(ctx) { - if (ctx.getText.toBoolean) { - Literal.TrueLiteral - } else { - Literal.FalseLiteral - } - } - - /** - * Create an integral literal expression. The code selects the most narrow integral type - * possible, either a BigDecimal, a Long or an Integer is returned. - */ - override def visitIntegerLiteral(ctx: IntegerLiteralContext): Literal = withOrigin(ctx) { - BigDecimal(ctx.getText) match { - case v if v.isValidInt => - Literal(v.intValue) - case v if v.isValidLong => - Literal(v.longValue) - case v => Literal(v.underlying()) - } - } - - /** - * Create a decimal literal for a regular decimal number. - */ - override def visitDecimalLiteral(ctx: DecimalLiteralContext): Literal = withOrigin(ctx) { - Literal(BigDecimal(ctx.getText).underlying()) - } - - /** - * Create a decimal literal for a regular decimal number or a scientific decimal number. - */ - override def visitLegacyDecimalLiteral( - ctx: LegacyDecimalLiteralContext): Literal = withOrigin(ctx) { - Literal(BigDecimal(ctx.getText).underlying()) - } - - /** - * Create a double literal for number with an exponent, e.g. 1E-30 - */ - override def visitExponentLiteral(ctx: ExponentLiteralContext): Literal = { - numericLiteral( - ctx, - ctx.getText, /* exponent values don't have a suffix */ - Double.MinValue, - Double.MaxValue, - DoubleType.simpleString)(_.toDouble) - } - - /** Create a numeric literal expression. */ - private def numericLiteral( - ctx: NumberContext, - rawStrippedQualifier: String, - minValue: BigDecimal, - maxValue: BigDecimal, - typeName: String)(converter: String => Any): Literal = withOrigin(ctx) { - try { - val rawBigDecimal = BigDecimal(rawStrippedQualifier) - if (rawBigDecimal < minValue || rawBigDecimal > maxValue) { - throw QueryParsingErrors.invalidNumericLiteralRangeError( - rawStrippedQualifier, - minValue, - maxValue, - typeName, - ctx) - } - Literal(converter(rawStrippedQualifier)) - } catch { - case e: NumberFormatException => - throw new ParseException(e.getMessage, ctx) - } - } - - /** - * Create a Byte Literal expression. - */ - override def visitTinyIntLiteral(ctx: TinyIntLiteralContext): Literal = { - val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) - numericLiteral(ctx, rawStrippedQualifier, Byte.MinValue, Byte.MaxValue, ByteType.simpleString)( - _.toByte) - } - - /** - * Create a Short Literal expression. - */ - override def visitSmallIntLiteral(ctx: SmallIntLiteralContext): Literal = { - val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) - numericLiteral( - ctx, - rawStrippedQualifier, - Short.MinValue, - Short.MaxValue, - ShortType.simpleString)(_.toShort) - } - - /** - * Create a Long Literal expression. - */ - override def visitBigIntLiteral(ctx: BigIntLiteralContext): Literal = { - val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) - numericLiteral(ctx, rawStrippedQualifier, Long.MinValue, Long.MaxValue, LongType.simpleString)( - _.toLong) - } - - /** - * Create a Float Literal expression. - */ - override def visitFloatLiteral(ctx: FloatLiteralContext): Literal = { - val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) - numericLiteral( - ctx, - rawStrippedQualifier, - Float.MinValue, - Float.MaxValue, - FloatType.simpleString)(_.toFloat) - } - - /** - * Create a Double Literal expression. - */ - override def visitDoubleLiteral(ctx: DoubleLiteralContext): Literal = { - val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) - numericLiteral( - ctx, - rawStrippedQualifier, - Double.MinValue, - Double.MaxValue, - DoubleType.simpleString)(_.toDouble) - } - - /** - * Create a BigDecimal Literal expression. - */ - override def visitBigDecimalLiteral(ctx: BigDecimalLiteralContext): Literal = { - val raw = ctx.getText.substring(0, ctx.getText.length - 2) - try { - Literal(BigDecimal(raw).underlying()) - } catch { - case e: AnalysisException => - throw new ParseException(e.message, ctx) - } - } - - /** - * Create a String literal expression. - */ - override def visitStringLiteral(ctx: StringLiteralContext): Literal = withOrigin(ctx) { - Literal(createString(ctx)) - } - - /** - * Create a String from a string literal context. This supports multiple consecutive string - * literals, these are concatenated, for example this expression "'hello' 'world'" will be - * converted into "helloworld". - * - * Special characters can be escaped by using Hive/C-style escaping. - */ - private def createString(ctx: StringLiteralContext): String = { - if (conf.escapedStringLiterals) { - ctx.STRING().asScala.map(stringWithoutUnescape).mkString - } else { - ctx.STRING().asScala.map(string).mkString - } - } - - /** - * Create an [[UnresolvedRelation]] from a multi-part identifier context. - */ - private def createUnresolvedRelation( - ctx: MultipartIdentifierContext): UnresolvedRelation = withOrigin(ctx) { - UnresolvedRelation(visitMultipartIdentifier(ctx)) - } - - /** - * Create an [[UnresolvedTable]] from a multi-part identifier context. - */ - private def createUnresolvedTable( - ctx: MultipartIdentifierContext, - commandName: String, - relationTypeMismatchHint: Option[String] = None): UnresolvedTable = withOrigin(ctx) { - UnresolvedTable(visitMultipartIdentifier(ctx), commandName, relationTypeMismatchHint) - } - - /** - * Create an [[UnresolvedView]] from a multi-part identifier context. - */ - private def createUnresolvedView( - ctx: MultipartIdentifierContext, - commandName: String, - allowTemp: Boolean = true, - relationTypeMismatchHint: Option[String] = None): UnresolvedView = withOrigin(ctx) { - UnresolvedView(visitMultipartIdentifier(ctx), commandName, allowTemp, relationTypeMismatchHint) - } - - /** - * Create an [[UnresolvedTableOrView]] from a multi-part identifier context. - */ - private def createUnresolvedTableOrView( - ctx: MultipartIdentifierContext, - commandName: String, - allowTempView: Boolean = true): UnresolvedTableOrView = withOrigin(ctx) { - UnresolvedTableOrView(visitMultipartIdentifier(ctx), commandName, allowTempView) - } - - /** - * Construct an [[Literal]] from [[CalendarInterval]] and - * units represented as a [[Seq]] of [[String]]. - */ - private def constructMultiUnitsIntervalLiteral( - ctx: ParserRuleContext, - calendarInterval: CalendarInterval, - units: Seq[String]): Literal = { - val yearMonthFields = Set.empty[Byte] - val dayTimeFields = Set.empty[Byte] - for (unit <- units) { - if (YearMonthIntervalType.stringToField.contains(unit)) { - yearMonthFields += YearMonthIntervalType.stringToField(unit) - } else if (DayTimeIntervalType.stringToField.contains(unit)) { - dayTimeFields += DayTimeIntervalType.stringToField(unit) - } else if (unit == "week") { - dayTimeFields += DayTimeIntervalType.DAY - } else { - assert(unit == "millisecond" || unit == "microsecond") - dayTimeFields += DayTimeIntervalType.SECOND - } - } - if (yearMonthFields.nonEmpty) { - if (dayTimeFields.nonEmpty) { - val literalStr = source(ctx) - throw QueryParsingErrors.mixedIntervalUnitsError(literalStr, ctx) - } - Literal( - calendarInterval.months, - YearMonthIntervalType(yearMonthFields.min, yearMonthFields.max)) - } else { - Literal( - IntervalUtils.getDuration(calendarInterval, TimeUnit.MICROSECONDS), - DayTimeIntervalType(dayTimeFields.min, dayTimeFields.max)) - } - } - - /** - * Create a [[CalendarInterval]] or ANSI interval literal expression. - * Two syntaxes are supported: - * - multiple unit value pairs, for instance: interval 2 months 2 days. - * - from-to unit, for instance: interval '1-2' year to month. - */ - override def visitInterval(ctx: IntervalContext): Literal = withOrigin(ctx) { - val calendarInterval = parseIntervalLiteral(ctx) - if (ctx.errorCapturingUnitToUnitInterval != null && !conf.legacyIntervalEnabled) { - // Check the `to` unit to distinguish year-month and day-time intervals because - // `CalendarInterval` doesn't have enough info. For instance, new CalendarInterval(0, 0, 0) - // can be derived from INTERVAL '0-0' YEAR TO MONTH as well as from - // INTERVAL '0 00:00:00' DAY TO SECOND. - val fromUnit = - ctx.errorCapturingUnitToUnitInterval.body.from.getText.toLowerCase(Locale.ROOT) - val toUnit = ctx.errorCapturingUnitToUnitInterval.body.to.getText.toLowerCase(Locale.ROOT) - if (toUnit == "month") { - assert(calendarInterval.days == 0 && calendarInterval.microseconds == 0) - val start = YearMonthIntervalType.stringToField(fromUnit) - Literal(calendarInterval.months, YearMonthIntervalType(start, YearMonthIntervalType.MONTH)) - } else { - assert(calendarInterval.months == 0) - val micros = IntervalUtils.getDuration(calendarInterval, TimeUnit.MICROSECONDS) - val start = DayTimeIntervalType.stringToField(fromUnit) - val end = DayTimeIntervalType.stringToField(toUnit) - Literal(micros, DayTimeIntervalType(start, end)) - } - } else if (ctx.errorCapturingMultiUnitsInterval != null && !conf.legacyIntervalEnabled) { - val units = - ctx.errorCapturingMultiUnitsInterval.body.unit.asScala.map( - _.getText.toLowerCase(Locale.ROOT).stripSuffix("s")).toSeq - constructMultiUnitsIntervalLiteral(ctx, calendarInterval, units) - } else { - Literal(calendarInterval, CalendarIntervalType) - } - } - - /** - * Create a [[CalendarInterval]] object - */ - protected def parseIntervalLiteral(ctx: IntervalContext): CalendarInterval = withOrigin(ctx) { - if (ctx.errorCapturingMultiUnitsInterval != null) { - val innerCtx = ctx.errorCapturingMultiUnitsInterval - if (innerCtx.unitToUnitInterval != null) { - throw QueryParsingErrors.moreThanOneFromToUnitInIntervalLiteralError( - innerCtx.unitToUnitInterval) - } - visitMultiUnitsInterval(innerCtx.multiUnitsInterval) - } else if (ctx.errorCapturingUnitToUnitInterval != null) { - val innerCtx = ctx.errorCapturingUnitToUnitInterval - if (innerCtx.error1 != null || innerCtx.error2 != null) { - val errorCtx = if (innerCtx.error1 != null) innerCtx.error1 else innerCtx.error2 - throw QueryParsingErrors.moreThanOneFromToUnitInIntervalLiteralError(errorCtx) - } - visitUnitToUnitInterval(innerCtx.body) - } else { - throw QueryParsingErrors.invalidIntervalLiteralError(ctx) - } - } - - /** - * Creates a [[CalendarInterval]] with multiple unit value pairs, e.g. 1 YEAR 2 DAYS. - */ - override def visitMultiUnitsInterval(ctx: MultiUnitsIntervalContext): CalendarInterval = { - withOrigin(ctx) { - val units = ctx.unit.asScala - val values = ctx.intervalValue().asScala - try { - assert(units.length == values.length) - val kvs = units.indices.map { i => - val u = units(i).getText - val v = if (values(i).STRING() != null) { - val value = string(values(i).STRING()) - // SPARK-32840: For invalid cases, e.g. INTERVAL '1 day 2' hour, - // INTERVAL 'interval 1' day, we need to check ahead before they are concatenated with - // units and become valid ones, e.g. '1 day 2 hour'. - // Ideally, we only ensure the value parts don't contain any units here. - if (value.exists(Character.isLetter)) { - throw QueryParsingErrors.invalidIntervalFormError(value, ctx) - } - if (values(i).MINUS() == null) { - value - } else if (value.startsWith("-")) { - value.replaceFirst("-", "") - } else { - s"-$value" - } - } else { - values(i).getText - } - UTF8String.fromString(" " + v + " " + u) - } - IntervalUtils.stringToInterval(UTF8String.concat(kvs: _*)) - } catch { - case i: IllegalArgumentException => - val e = new ParseException(i.getMessage, ctx) - e.setStackTrace(i.getStackTrace) - throw e - } - } - } - - /** - * Creates a [[CalendarInterval]] with from-to unit, e.g. '2-1' YEAR TO MONTH. - */ - override def visitUnitToUnitInterval(ctx: UnitToUnitIntervalContext): CalendarInterval = { - withOrigin(ctx) { - val value = Option(ctx.intervalValue.STRING).map(string).map { interval => - if (ctx.intervalValue().MINUS() == null) { - interval - } else if (interval.startsWith("-")) { - interval.replaceFirst("-", "") - } else { - s"-$interval" - } - }.getOrElse { - throw QueryParsingErrors.invalidFromToUnitValueError(ctx.intervalValue) - } - try { - val from = ctx.from.getText.toLowerCase(Locale.ROOT) - val to = ctx.to.getText.toLowerCase(Locale.ROOT) - (from, to) match { - case ("year", "month") => - IntervalUtils.fromYearMonthString(value) - case ("day", "hour") | ("day", "minute") | ("day", "second") | ("hour", "minute") | - ("hour", "second") | ("minute", "second") => - IntervalUtils.fromDayTimeString( - value, - DayTimeIntervalType.stringToField(from), - DayTimeIntervalType.stringToField(to)) - case _ => - throw QueryParsingErrors.fromToIntervalUnsupportedError(from, to, ctx) - } - } catch { - // Handle Exceptions thrown by CalendarInterval - case e: IllegalArgumentException => - val pe = new ParseException(e.getMessage, ctx) - pe.setStackTrace(e.getStackTrace) - throw pe - } - } - } - - /* ******************************************************************************************** - * DataType parsing - * ******************************************************************************************** */ - - /** - * Resolve/create a primitive type. - */ - override def visitPrimitiveDataType(ctx: PrimitiveDataTypeContext): DataType = withOrigin(ctx) { - val dataType = ctx.identifier.getText.toLowerCase(Locale.ROOT) - (dataType, ctx.INTEGER_VALUE().asScala.toList) match { - case ("boolean", Nil) => BooleanType - case ("tinyint" | "byte", Nil) => ByteType - case ("smallint" | "short", Nil) => ShortType - case ("int" | "integer", Nil) => IntegerType - case ("bigint" | "long", Nil) => LongType - case ("float" | "real", Nil) => FloatType - case ("double", Nil) => DoubleType - case ("date", Nil) => DateType - case ("timestamp", Nil) => SQLConf.get.timestampType - // SPARK-36227: Remove TimestampNTZ type support in Spark 3.2 with minimal code changes. - case ("timestamp_ntz", Nil) if isTesting => TimestampNTZType - case ("timestamp_ltz", Nil) if isTesting => TimestampType - case ("string", Nil) => StringType - case ("character" | "char", length :: Nil) => CharType(length.getText.toInt) - case ("varchar", length :: Nil) => VarcharType(length.getText.toInt) - case ("binary", Nil) => BinaryType - case ("decimal" | "dec" | "numeric", Nil) => DecimalType.USER_DEFAULT - case ("decimal" | "dec" | "numeric", precision :: Nil) => - DecimalType(precision.getText.toInt, 0) - case ("decimal" | "dec" | "numeric", precision :: scale :: Nil) => - DecimalType(precision.getText.toInt, scale.getText.toInt) - case ("void", Nil) => NullType - case ("interval", Nil) => CalendarIntervalType - case (dt, params) => - val dtStr = if (params.nonEmpty) s"$dt(${params.mkString(",")})" else dt - throw QueryParsingErrors.dataTypeUnsupportedError(dtStr, ctx) - } - } - - override def visitYearMonthIntervalDataType(ctx: YearMonthIntervalDataTypeContext): DataType = { - val startStr = ctx.from.getText.toLowerCase(Locale.ROOT) - val start = YearMonthIntervalType.stringToField(startStr) - if (ctx.to != null) { - val endStr = ctx.to.getText.toLowerCase(Locale.ROOT) - val end = YearMonthIntervalType.stringToField(endStr) - if (end <= start) { - throw QueryParsingErrors.fromToIntervalUnsupportedError(startStr, endStr, ctx) - } - YearMonthIntervalType(start, end) - } else { - YearMonthIntervalType(start) - } - } - - override def visitDayTimeIntervalDataType(ctx: DayTimeIntervalDataTypeContext): DataType = { - val startStr = ctx.from.getText.toLowerCase(Locale.ROOT) - val start = DayTimeIntervalType.stringToField(startStr) - if (ctx.to != null) { - val endStr = ctx.to.getText.toLowerCase(Locale.ROOT) - val end = DayTimeIntervalType.stringToField(endStr) - if (end <= start) { - throw QueryParsingErrors.fromToIntervalUnsupportedError(startStr, endStr, ctx) - } - DayTimeIntervalType(start, end) - } else { - DayTimeIntervalType(start) - } - } - - /** - * Create a complex DataType. Arrays, Maps and Structures are supported. - */ - override def visitComplexDataType(ctx: ComplexDataTypeContext): DataType = withOrigin(ctx) { - ctx.complex.getType match { - case MixedFormatSqlExtendParser.ARRAY => - ArrayType(typedVisit(ctx.dataType(0))) - case MixedFormatSqlExtendParser.MAP => - MapType(typedVisit(ctx.dataType(0)), typedVisit(ctx.dataType(1))) - case MixedFormatSqlExtendParser.STRUCT => - StructType(Option(ctx.complexColTypeList).toSeq.flatMap(visitComplexColTypeList)) - } - } - - /** - * Create top level table schema. - */ - protected def createSchema(ctx: ColTypeListContext): StructType = { - StructType(Option(ctx).toSeq.flatMap(visitColTypeList)) - } - - /** - * Create a [[StructType]] from a number of column definitions. - */ - override def visitColTypeList(ctx: ColTypeListContext): Seq[StructField] = withOrigin(ctx) { - ctx.colType().asScala.map(visitColType).toSeq - } - - /** - * Create a top level [[StructField]] from a column definition. - */ - override def visitColType(ctx: ColTypeContext): StructField = withOrigin(ctx) { - import ctx._ - - val builder = new MetadataBuilder - // Add comment to metadata - Option(commentSpec()).map(visitCommentSpec).foreach { - builder.putString("comment", _) - } - - StructField( - name = colName.getText, - dataType = typedVisit[DataType](ctx.dataType), - nullable = NULL == null, - metadata = builder.build()) - } - - /** - * Create a [[StructType]] from a sequence of [[StructField]]s. - */ - protected def createStructType(ctx: ComplexColTypeListContext): StructType = { - StructType(Option(ctx).toSeq.flatMap(visitComplexColTypeList)) - } - - /** - * Create a [[StructType]] from a number of column definitions. - */ - override def visitComplexColTypeList( - ctx: ComplexColTypeListContext): Seq[StructField] = withOrigin(ctx) { - ctx.complexColType().asScala.map(visitComplexColType).toSeq - } - - /** - * Create a [[StructField]] from a column definition. - */ - override def visitComplexColType(ctx: ComplexColTypeContext): StructField = withOrigin(ctx) { - import ctx._ - val structField = StructField( - name = identifier.getText, - dataType = typedVisit(dataType()), - nullable = NULL == null) - Option(commentSpec).map(visitCommentSpec).map(structField.withComment).getOrElse(structField) - } - - /** - * Create a location string. - */ - override def visitLocationSpec(ctx: LocationSpecContext): String = withOrigin(ctx) { - string(ctx.STRING) - } - - /** - * Create an optional location string. - */ - protected def visitLocationSpecList(ctx: java.util.List[LocationSpecContext]): Option[String] = { - ctx.asScala.headOption.map(visitLocationSpec) - } - - /** - * Create a comment string. - */ - override def visitCommentSpec(ctx: CommentSpecContext): String = withOrigin(ctx) { - string(ctx.STRING) - } - - /** - * Create an optional comment string. - */ - protected def visitCommentSpecList(ctx: java.util.List[CommentSpecContext]): Option[String] = { - ctx.asScala.headOption.map(visitCommentSpec) - } - - /** - * Create a [[BucketSpec]]. - */ - override def visitBucketSpec(ctx: BucketSpecContext): BucketSpec = withOrigin(ctx) { - BucketSpec( - ctx.INTEGER_VALUE.getText.toInt, - visitIdentifierList(ctx.identifierList), - Option(ctx.orderedIdentifierList) - .toSeq - .flatMap(_.orderedIdentifier.asScala) - .map { orderedIdCtx => - Option(orderedIdCtx.ordering).map(_.getText).foreach { dir => - if (dir.toLowerCase(Locale.ROOT) != "asc") { - operationNotAllowed(s"Column ordering must be ASC, was '$dir'", ctx) - } - } - - orderedIdCtx.ident.getText - }) - } - - /** - * Convert a property list into a key-value map. - * This should be called through [[visitPropertyKeyValues]] or [[visitPropertyKeys]]. - */ - override def visitTablePropertyList( - ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) { - val properties = ctx.tableProperty.asScala.map { property => - val key = visitTablePropertyKey(property.key) - val value = visitTablePropertyValue(property.value) - key -> value - } - // Check for duplicate property names. - checkDuplicateKeys(properties.toSeq, ctx) - properties.toMap - } - - /** - * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified. - */ - def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = { - val props = visitTablePropertyList(ctx) - val badKeys = props.collect { case (key, null) => key } - if (badKeys.nonEmpty) { - operationNotAllowed( - s"Values must be specified for key(s): ${badKeys.mkString("[", ",", "]")}", - ctx) - } - props - } - - /** - * Parse a list of keys from a [[TablePropertyListContext]], assuming no values are specified. - */ - def visitPropertyKeys(ctx: TablePropertyListContext): Seq[String] = { - val props = visitTablePropertyList(ctx) - val badKeys = props.filter { case (_, v) => v != null }.keys - if (badKeys.nonEmpty) { - operationNotAllowed( - s"Values should not be specified for key(s): ${badKeys.mkString("[", ",", "]")}", - ctx) - } - props.keys.toSeq - } - - /** - * A table property key can either be String or a collection of dot separated elements. This - * function extracts the property key based on whether its a string literal or a table property - * identifier. - */ - override def visitTablePropertyKey(key: TablePropertyKeyContext): String = { - if (key.STRING != null) { - string(key.STRING) - } else { - key.getText - } - } - - /** - * A table property value can be String, Integer, Boolean or Decimal. This function extracts - * the property value based on whether its a string, integer, boolean or decimal literal. - */ - override def visitTablePropertyValue(value: TablePropertyValueContext): String = { - if (value == null) { - null - } else if (value.STRING != null) { - string(value.STRING) - } else if (value.booleanValue != null) { - value.getText.toLowerCase(Locale.ROOT) - } else { - value.getText - } - } - - /** - * Type to keep track of a table header: (identifier, isTemporary, ifNotExists, isExternal). - */ - type TableHeader = (Seq[String], Boolean, Boolean, Boolean) - - /** - * Validate a create table statement and return the [[TableIdentifier]]. - */ - override def visitCreateTableHeader( - ctx: CreateTableHeaderContext): TableHeader = withOrigin(ctx) { - val temporary = ctx.TEMPORARY != null - val ifNotExists = ctx.EXISTS != null - if (temporary && ifNotExists) { - operationNotAllowed("CREATE TEMPORARY TABLE ... IF NOT EXISTS", ctx) - } - val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText).toSeq - (multipartIdentifier, temporary, ifNotExists, ctx.EXTERNAL != null) - } - - /** - * Parse a qualified name to a multipart name. - */ - override def visitQualifiedName(ctx: QualifiedNameContext): Seq[String] = withOrigin(ctx) { - ctx.identifier.asScala.map(_.getText).toSeq - } - - /** - * Parse a list of transforms or columns. - */ - override def visitPartitionFieldList( - ctx: PartitionFieldListContext): (Seq[Transform], Seq[StructField]) = withOrigin(ctx) { - val (transforms, columns) = ctx.fields.asScala.map { - case transform: PartitionTransformContext => - (Some(visitPartitionTransform(transform)), None) - case field: PartitionColumnContext => - (None, Some(visitColType(field.colType))) - }.unzip - - (transforms.flatten.toSeq, columns.flatten.toSeq) - } - - override def visitPartitionTransform( - ctx: PartitionTransformContext): Transform = withOrigin(ctx) { - def getFieldReference( - ctx: ApplyTransformContext, - arg: V2Expression): FieldReference = { - lazy val name: String = ctx.identifier.getText - arg match { - case ref: FieldReference => - ref - case nonRef => - throw QueryParsingErrors.partitionTransformNotExpectedError(name, nonRef.describe, ctx) - } - } - - def getSingleFieldReference( - ctx: ApplyTransformContext, - arguments: Seq[V2Expression]): FieldReference = { - lazy val name: String = ctx.identifier.getText - if (arguments.size > 1) { - throw QueryParsingErrors.tooManyArgumentsForTransformError(name, ctx) - } else if (arguments.isEmpty) { - throw new IllegalStateException(s"Not enough arguments for transform $name") - } else { - getFieldReference(ctx, arguments.head) - } - } - - ctx.transform match { - case identityCtx: IdentityTransformContext => - IdentityTransform(FieldReference(typedVisit[Seq[String]](identityCtx.qualifiedName))) - - case applyCtx: ApplyTransformContext => - val arguments = applyCtx.argument.asScala.map(visitTransformArgument).toSeq - - applyCtx.identifier.getText match { - case "bucket" => - val numBuckets: Int = arguments.head match { - case LiteralValue(shortValue, ShortType) => - shortValue.asInstanceOf[Short].toInt - case LiteralValue(intValue, IntegerType) => - intValue.asInstanceOf[Int] - case LiteralValue(longValue, LongType) => - longValue.asInstanceOf[Long].toInt - case lit => - throw QueryParsingErrors.invalidBucketsNumberError(lit.describe, applyCtx) - } - - val fields = arguments.tail.map(arg => getFieldReference(applyCtx, arg)) - - BucketTransform(LiteralValue(numBuckets, IntegerType), fields) - - case "years" => - YearsTransform(getSingleFieldReference(applyCtx, arguments)) - - case "months" => - MonthsTransform(getSingleFieldReference(applyCtx, arguments)) - - case "days" => - DaysTransform(getSingleFieldReference(applyCtx, arguments)) - - case "hours" => - HoursTransform(getSingleFieldReference(applyCtx, arguments)) - - case name => - ApplyTransform(name, arguments) - } - } - } - - /** - * Parse an argument to a transform. An argument may be a field reference (qualified name) or - * a value literal. - */ - override def visitTransformArgument(ctx: TransformArgumentContext): V2Expression = { - withOrigin(ctx) { - val reference = Option(ctx.qualifiedName) - .map(typedVisit[Seq[String]]) - .map(FieldReference(_)) - val literal = Option(ctx.constant) - .map(typedVisit[Literal]) - .map(lit => LiteralValue(lit.value, lit.dataType)) - reference.orElse(literal) - .getOrElse(throw new IllegalStateException("Invalid transform argument")) - } - } - - private def cleanNamespaceProperties( - properties: Map[String, String], - ctx: ParserRuleContext): Map[String, String] = withOrigin(ctx) { - import SupportsNamespaces._ - val legacyOn = conf.getConf(SQLConf.LEGACY_PROPERTY_NON_RESERVED) - properties.filter { - case (PROP_LOCATION, _) if !legacyOn => - throw QueryParsingErrors.cannotCleanReservedNamespacePropertyError( - PROP_LOCATION, - ctx, - "please use the LOCATION clause to specify it") - case (PROP_LOCATION, _) => false - case (PROP_OWNER, _) if !legacyOn => - throw QueryParsingErrors.cannotCleanReservedNamespacePropertyError( - PROP_OWNER, - ctx, - "it will be set to the current user") - case (PROP_OWNER, _) => false - case _ => true - } - } - - def cleanTableProperties( - ctx: ParserRuleContext, - properties: Map[String, String]): Map[String, String] = { - import TableCatalog._ - val legacyOn = conf.getConf(SQLConf.LEGACY_PROPERTY_NON_RESERVED) - properties.filter { - case (PROP_PROVIDER, _) if !legacyOn => - throw QueryParsingErrors.cannotCleanReservedTablePropertyError( - PROP_PROVIDER, - ctx, - "please use the USING clause to specify it") - case (PROP_PROVIDER, _) => false - case (PROP_LOCATION, _) if !legacyOn => - throw QueryParsingErrors.cannotCleanReservedTablePropertyError( - PROP_LOCATION, - ctx, - "please use the LOCATION clause to specify it") - case (PROP_LOCATION, _) => false - case (PROP_OWNER, _) if !legacyOn => - throw QueryParsingErrors.cannotCleanReservedTablePropertyError( - PROP_OWNER, - ctx, - "it will be set to the current user") - case (PROP_OWNER, _) => false - case (PROP_EXTERNAL, _) if !legacyOn => - throw QueryParsingErrors.cannotCleanReservedTablePropertyError( - PROP_EXTERNAL, - ctx, - "please use CREATE EXTERNAL TABLE") - case (PROP_EXTERNAL, _) => false - // It's safe to set whatever table comment, so we don't make it a reserved table property. - case (PROP_COMMENT, _) => true - case (k, _) => - val isReserved = CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(k) - if (!legacyOn && isReserved) { - throw QueryParsingErrors.cannotCleanReservedTablePropertyError( - k, - ctx, - "please remove it from the TBLPROPERTIES list.") - } - !isReserved - } - } - - def cleanTableOptions( - ctx: ParserRuleContext, - options: Map[String, String], - location: Option[String]): (Map[String, String], Option[String]) = { - var path = location - val filtered = cleanTableProperties(ctx, options).filter { - case (k, v) if k.equalsIgnoreCase("path") && path.nonEmpty => - throw QueryParsingErrors.duplicatedTablePathsFoundError(path.get, v, ctx) - case (k, v) if k.equalsIgnoreCase("path") => - path = Some(v) - false - case _ => true - } - (filtered, path) - } - - /** - * Create a [[SerdeInfo]] for creating tables. - * - * Format: STORED AS (name | INPUTFORMAT input_format OUTPUTFORMAT output_format) - */ - override def visitCreateFileFormat(ctx: CreateFileFormatContext): SerdeInfo = withOrigin(ctx) { - (ctx.fileFormat, ctx.storageHandler) match { - // Expected format: INPUTFORMAT input_format OUTPUTFORMAT output_format - case (c: TableFileFormatContext, null) => - SerdeInfo(formatClasses = Some(FormatClasses(string(c.inFmt), string(c.outFmt)))) - // Expected format: SEQUENCEFILE | TEXTFILE | RCFILE | ORC | PARQUET | AVRO - case (c: GenericFileFormatContext, null) => - SerdeInfo(storedAs = Some(c.identifier.getText)) - case (null, storageHandler) => - operationNotAllowed("STORED BY", ctx) - case _ => - throw QueryParsingErrors.storedAsAndStoredByBothSpecifiedError(ctx) - } - } - - /** - * Create a [[SerdeInfo]] used for creating tables. - * - * Example format: - * {{{ - * SERDE serde_name [WITH SERDEPROPERTIES (k1=v1, k2=v2, ...)] - * }}} - * - * OR - * - * {{{ - * DELIMITED [FIELDS TERMINATED BY char [ESCAPED BY char]] - * [COLLECTION ITEMS TERMINATED BY char] - * [MAP KEYS TERMINATED BY char] - * [LINES TERMINATED BY char] - * [NULL DEFINED AS char] - * }}} - */ - def visitRowFormat(ctx: RowFormatContext): SerdeInfo = withOrigin(ctx) { - ctx match { - case serde: RowFormatSerdeContext => visitRowFormatSerde(serde) - case delimited: RowFormatDelimitedContext => visitRowFormatDelimited(delimited) - } - } - - /** - * Create SERDE row format name and properties pair. - */ - override def visitRowFormatSerde(ctx: RowFormatSerdeContext): SerdeInfo = withOrigin(ctx) { - import ctx._ - SerdeInfo( - serde = Some(string(name)), - serdeProperties = Option(tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) - } - - /** - * Create a delimited row format properties object. - */ - override def visitRowFormatDelimited( - ctx: RowFormatDelimitedContext): SerdeInfo = withOrigin(ctx) { - // Collect the entries if any. - def entry(key: String, value: Token): Seq[(String, String)] = { - Option(value).toSeq.map(x => key -> string(x)) - } - - // TODO we need proper support for the NULL format. - val entries = - entry("field.delim", ctx.fieldsTerminatedBy) ++ - entry("serialization.format", ctx.fieldsTerminatedBy) ++ - entry("escape.delim", ctx.escapedBy) ++ - // The following typo is inherited from Hive... - entry("colelction.delim", ctx.collectionItemsTerminatedBy) ++ - entry("mapkey.delim", ctx.keysTerminatedBy) ++ - Option(ctx.linesSeparatedBy).toSeq.map { token => - val value = string(token) - validate( - value == "\n", - s"LINES TERMINATED BY only supports newline '\\n' right now: $value", - ctx) - "line.delim" -> value - } - SerdeInfo(serdeProperties = entries.toMap) - } - - /** - * Throw a [[ParseException]] if the user specified incompatible SerDes through ROW FORMAT - * and STORED AS. - * - * The following are allowed. Anything else is not: - * ROW FORMAT SERDE ... STORED AS [SEQUENCEFILE | RCFILE | TEXTFILE] - * ROW FORMAT DELIMITED ... STORED AS TEXTFILE - * ROW FORMAT ... STORED AS INPUTFORMAT ... OUTPUTFORMAT ... - */ - protected def validateRowFormatFileFormat( - rowFormatCtx: RowFormatContext, - createFileFormatCtx: CreateFileFormatContext, - parentCtx: ParserRuleContext): Unit = { - if (rowFormatCtx == null || createFileFormatCtx == null) { - return - } - (rowFormatCtx, createFileFormatCtx.fileFormat) match { - case (_, ffTable: TableFileFormatContext) => // OK - case (rfSerde: RowFormatSerdeContext, ffGeneric: GenericFileFormatContext) => - ffGeneric.identifier.getText.toLowerCase(Locale.ROOT) match { - case ("sequencefile" | "textfile" | "rcfile") => // OK - case fmt => - operationNotAllowed( - s"ROW FORMAT SERDE is incompatible with format '$fmt', which also specifies a serde", - parentCtx) - } - case (rfDelimited: RowFormatDelimitedContext, ffGeneric: GenericFileFormatContext) => - ffGeneric.identifier.getText.toLowerCase(Locale.ROOT) match { - case "textfile" => // OK - case fmt => operationNotAllowed( - s"ROW FORMAT DELIMITED is only compatible with 'textfile', not '$fmt'", - parentCtx) - } - case _ => - // should never happen - def str(ctx: ParserRuleContext): String = { - (0 until ctx.getChildCount).map { i => ctx.getChild(i).getText }.mkString(" ") - } - - operationNotAllowed( - s"Unexpected combination of ${str(rowFormatCtx)} and ${str(createFileFormatCtx)}", - parentCtx) - } - } - - protected def validateRowFormatFileFormat( - rowFormatCtx: Seq[RowFormatContext], - createFileFormatCtx: Seq[CreateFileFormatContext], - parentCtx: ParserRuleContext): Unit = { - if (rowFormatCtx.size == 1 && createFileFormatCtx.size == 1) { - validateRowFormatFileFormat(rowFormatCtx.head, createFileFormatCtx.head, parentCtx) - } - } - - protected def getSerdeInfo( - rowFormatCtx: Seq[RowFormatContext], - createFileFormatCtx: Seq[CreateFileFormatContext], - ctx: ParserRuleContext): Option[SerdeInfo] = { - validateRowFormatFileFormat(rowFormatCtx, createFileFormatCtx, ctx) - val rowFormatSerdeInfo = rowFormatCtx.map(visitRowFormat) - val fileFormatSerdeInfo = createFileFormatCtx.map(visitCreateFileFormat) - (fileFormatSerdeInfo ++ rowFormatSerdeInfo).reduceLeftOption((l, r) => l.merge(r)) - } - - private def partitionExpressions( - partTransforms: Seq[Transform], - partCols: Seq[StructField], - ctx: ParserRuleContext): Seq[Transform] = { - if (partTransforms.nonEmpty) { - if (partCols.nonEmpty) { - val references = partTransforms.map(_.describe()).mkString(", ") - val columns = partCols - .map(field => s"${field.name} ${field.dataType.simpleString}") - .mkString(", ") - operationNotAllowed( - s"""PARTITION BY: Cannot mix partition expressions and partition columns: - |Expressions: $references - |Columns: $columns""".stripMargin, - ctx) - - } - partTransforms - } else { - // columns were added to create the schema. convert to column references - partCols.map { column => - IdentityTransform(FieldReference(Seq(column.name))) - } - } - } - - override def visitColPosition(ctx: ColPositionContext): ColumnPosition = { - ctx.position.getType match { - case MixedFormatSqlExtendParser.FIRST => ColumnPosition.first() - case MixedFormatSqlExtendParser.AFTER => ColumnPosition.after(ctx.afterCol.getText) - } - } - -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/parser/QueryParsingErrors.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/parser/QueryParsingErrors.scala deleted file mode 100644 index bb6e38a247..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/amoro/parser/QueryParsingErrors.scala +++ /dev/null @@ -1,247 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.amoro.parser - -import org.antlr.v4.runtime.ParserRuleContext -import org.apache.spark.sql.catalyst.parser.ParseException - -import org.apache.amoro.spark.sql.parser.MixedFormatSqlExtendParser._ - -/** - * Object for grouping all error messages of the query parsing. - * Currently it includes all ParseException. - */ -private[sql] object QueryParsingErrors { - - def columnAliasInOperationNotAllowedError(op: String, ctx: TableAliasContext): Throwable = { - new ParseException(s"Columns aliases are not allowed in $op.", ctx.identifierList()) - } - - def combinationQueryResultClausesUnsupportedError(ctx: QueryOrganizationContext): Throwable = { - new ParseException( - "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", - ctx) - } - - def distributeByUnsupportedError(ctx: QueryOrganizationContext): Throwable = { - new ParseException("DISTRIBUTE BY is not supported", ctx) - } - - def transformNotSupportQuantifierError(ctx: ParserRuleContext): Throwable = { - new ParseException("TRANSFORM does not support DISTINCT/ALL in inputs", ctx) - } - - def transformWithSerdeUnsupportedError(ctx: ParserRuleContext): Throwable = { - new ParseException("TRANSFORM with serde is only supported in hive mode", ctx) - } - - def lateralWithPivotInFromClauseNotAllowedError(ctx: FromClauseContext): Throwable = { - new ParseException("LATERAL cannot be used together with PIVOT in FROM clause", ctx) - } - - def lateralJoinWithNaturalJoinUnsupportedError(ctx: ParserRuleContext): Throwable = { - new ParseException("LATERAL join with NATURAL join is not supported", ctx) - } - - def lateralJoinWithUsingJoinUnsupportedError(ctx: ParserRuleContext): Throwable = { - new ParseException("LATERAL join with USING join is not supported", ctx) - } - - def unsupportedLateralJoinTypeError(ctx: ParserRuleContext, joinType: String): Throwable = { - new ParseException(s"Unsupported LATERAL join type $joinType", ctx) - } - - def invalidLateralJoinRelationError(ctx: RelationPrimaryContext): Throwable = { - new ParseException(s"LATERAL can only be used with subquery", ctx) - } - - def repetitiveWindowDefinitionError(name: String, ctx: WindowClauseContext): Throwable = { - new ParseException(s"The definition of window '$name' is repetitive", ctx) - } - - def invalidWindowReferenceError(name: String, ctx: WindowClauseContext): Throwable = { - new ParseException(s"Window reference '$name' is not a window specification", ctx) - } - - def cannotResolveWindowReferenceError(name: String, ctx: WindowClauseContext): Throwable = { - new ParseException(s"Cannot resolve window reference '$name'", ctx) - } - - def naturalCrossJoinUnsupportedError(ctx: RelationContext): Throwable = { - new ParseException("NATURAL CROSS JOIN is not supported", ctx) - } - - def emptyInputForTableSampleError(ctx: ParserRuleContext): Throwable = { - new ParseException("TABLESAMPLE does not accept empty inputs.", ctx) - } - - def tableSampleByBytesUnsupportedError(msg: String, ctx: SampleMethodContext): Throwable = { - new ParseException(s"TABLESAMPLE($msg) is not supported", ctx) - } - - def invalidByteLengthLiteralError(bytesStr: String, ctx: SampleByBytesContext): Throwable = { - new ParseException( - s"$bytesStr is not a valid byte length literal, " + - "expected syntax: DIGIT+ ('B' | 'K' | 'M' | 'G')", - ctx) - } - - def invalidFromToUnitValueError(ctx: IntervalValueContext): Throwable = { - new ParseException("The value of from-to unit must be a string", ctx) - } - - def storedAsAndStoredByBothSpecifiedError(ctx: CreateFileFormatContext): Throwable = { - new ParseException("Expected either STORED AS or STORED BY, not both", ctx) - } - - def invalidEscapeStringError(ctx: PredicateContext): Throwable = { - new ParseException("Invalid escape string. Escape string must contain only one character.", ctx) - } - - def trimOptionUnsupportedError(trimOption: Int, ctx: TrimContext): Throwable = { - new ParseException( - "Function trim doesn't support with " + - s"type $trimOption. Please use BOTH, LEADING or TRAILING as trim type", - ctx) - } - - def invalidIntervalFormError(value: String, ctx: MultiUnitsIntervalContext): Throwable = { - new ParseException( - "Can only use numbers in the interval value part for" + - s" multiple unit value pairs interval form, but got invalid value: $value", - ctx) - } - - def functionNameUnsupportedError(functionName: String, ctx: ParserRuleContext): Throwable = { - new ParseException(s"Unsupported function name '$functionName'", ctx) - } - - def cannotParseValueTypeError( - valueType: String, - value: String, - ctx: TypeConstructorContext): Throwable = { - new ParseException(s"Cannot parse the $valueType value: $value", ctx) - } - - def cannotParseIntervalValueError(value: String, ctx: TypeConstructorContext): Throwable = { - new ParseException(s"Cannot parse the INTERVAL value: $value", ctx) - } - - def literalValueTypeUnsupportedError( - valueType: String, - ctx: TypeConstructorContext): Throwable = { - new ParseException(s"Literals of type '$valueType' are currently not supported.", ctx) - } - - def parsingValueTypeError( - e: IllegalArgumentException, - valueType: String, - ctx: TypeConstructorContext): Throwable = { - val message = Option(e.getMessage).getOrElse(s"Exception parsing $valueType") - new ParseException(message, ctx) - } - - def invalidNumericLiteralRangeError( - rawStrippedQualifier: String, - minValue: BigDecimal, - maxValue: BigDecimal, - typeName: String, - ctx: NumberContext): Throwable = { - new ParseException( - s"Numeric literal $rawStrippedQualifier does not " + - s"fit in range [$minValue, $maxValue] for type $typeName", - ctx) - } - - def moreThanOneFromToUnitInIntervalLiteralError(ctx: ParserRuleContext): Throwable = { - new ParseException("Can only have a single from-to unit in the interval literal syntax", ctx) - } - - def invalidIntervalLiteralError(ctx: IntervalContext): Throwable = { - new ParseException("at least one time unit should be given for interval literal", ctx) - } - - def fromToIntervalUnsupportedError( - from: String, - to: String, - ctx: ParserRuleContext): Throwable = { - new ParseException(s"Intervals FROM $from TO $to are not supported.", ctx) - } - - def mixedIntervalUnitsError(literal: String, ctx: ParserRuleContext): Throwable = { - new ParseException(s"Cannot mix year-month and day-time fields: $literal", ctx) - } - - def dataTypeUnsupportedError(dataType: String, ctx: PrimitiveDataTypeContext): Throwable = { - new ParseException(s"DataType $dataType is not supported.", ctx) - } - - def partitionTransformNotExpectedError( - name: String, - describe: String, - ctx: ApplyTransformContext): Throwable = { - new ParseException(s"Expected a column reference for transform $name: $describe", ctx) - } - - def tooManyArgumentsForTransformError(name: String, ctx: ApplyTransformContext): Throwable = { - new ParseException(s"Too many arguments for transform $name", ctx) - } - - def invalidBucketsNumberError(describe: String, ctx: ApplyTransformContext): Throwable = { - new ParseException(s"Invalid number of buckets: $describe", ctx) - } - - def cannotCleanReservedNamespacePropertyError( - property: String, - ctx: ParserRuleContext, - msg: String): Throwable = { - new ParseException(s"$property is a reserved namespace property, $msg.", ctx) - } - - def cannotCleanReservedTablePropertyError( - property: String, - ctx: ParserRuleContext, - msg: String): Throwable = { - new ParseException(s"$property is a reserved table property, $msg.", ctx) - } - - def duplicatedTablePathsFoundError( - pathOne: String, - pathTwo: String, - ctx: ParserRuleContext): Throwable = { - new ParseException( - s"Duplicated table paths found: '$pathOne' and '$pathTwo'. LOCATION" + - s" and the case insensitive key 'path' in OPTIONS are all used to indicate the custom" + - s" table path, you can only specify one of them.", - ctx) - } - - def operationNotAllowedError(message: String, ctx: ParserRuleContext): Throwable = { - new ParseException(s"Operation not allowed: $message", ctx) - } - - def duplicateCteDefinitionNamesError(duplicateNames: String, ctx: CtesContext): Throwable = { - new ParseException(s"CTE definition can't have duplicate names: $duplicateNames.", ctx) - } - - def invalidGroupingSetError(element: String, ctx: GroupingAnalyticsContext): Throwable = { - new ParseException(s"Empty set in $element grouping sets is not supported.", ctx) - } - -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/catalyst/analysis/MixedFormatAssignmentAlignmentSupport.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/catalyst/analysis/MixedFormatAssignmentAlignmentSupport.scala deleted file mode 100644 index eb06caaea5..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/catalyst/analysis/MixedFormatAssignmentAlignmentSupport.scala +++ /dev/null @@ -1,200 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.analysis - -import scala.collection.compat.immutable.ArraySeq -import scala.collection.mutable - -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.SQLConfHelper -import org.apache.spark.sql.catalyst.expressions.{Alias, AnsiCast, Cast, CreateNamedStruct, Expression, GetStructField, Literal, NamedExpression} -import org.apache.spark.sql.catalyst.expressions.AssignmentUtils._ -import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan} -import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy -import org.apache.spark.sql.types.{DataType, StructField, StructType} - -trait MixedFormatAssignmentAlignmentSupport extends CastSupport { - - self: SQLConfHelper => - - private case class ColumnUpdate(ref: Seq[String], expr: Expression) - - /** - * Aligns assignments to match table columns. - *

- * This method processes and reorders given assignments so that each target column gets - * an expression it should be set to. If a column does not have a matching assignment, - * it will be set to its current value. For example, if one passes a table with columns c1, c2 - * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1. - *

- * This method also handles updates to nested columns. If there is an assignment to a particular - * nested field, this method will construct a new struct with one field updated - * preserving other fields that have not been modified. For example, if one passes a table with - * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1, - * this method will return c1 = c1, c2 = struct(c2.n1, 1). - * - * @param table a target table - * @param assignments assignments to align - * @return aligned assignments that match table columns - */ - protected def alignAssignments( - table: LogicalPlan, - assignments: Seq[Assignment]): Seq[Assignment] = { - - val columnUpdates = assignments.map(a => ColumnUpdate(toAssignmentRef(a.key), a.value)) - val outputExprs = applyUpdates(table.output, columnUpdates) - outputExprs.zip(table.output).map { - case (expr, attr) => handleCharVarcharLimits(Assignment(attr, expr)) - } - } - - private def applyUpdates( - cols: Seq[NamedExpression], - updates: Seq[ColumnUpdate], - resolver: Resolver = conf.resolver, - namePrefix: Seq[String] = Nil): Seq[Expression] = { - - // iterate through columns at the current level and find which column updates match - cols.map { col => - // find matches for this column or any of its children - val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name)) - prefixMatchedUpdates match { - // if there is no exact match and no match for children, return the column as is - case updates if updates.isEmpty => - col - - // if there is an exact match, return the assigned expression - case Seq(update) if isExactMatch(update, col, resolver) => - castIfNeeded(col, update.expr, resolver) - - // if there are matches only for children - case updates if !hasExactMatch(updates, col, resolver) => - col.dataType match { - case StructType(fields) => - // build field expressions - val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) => - Alias(GetStructField(col, ordinal, Some(field.name)), field.name)() - } - - // recursively apply this method on nested fields - val newUpdates = updates.map(u => u.copy(ref = u.ref.tail)) - val updatedFieldExprs = applyUpdates( - ArraySeq.unsafeWrapArray(fieldExprs), - newUpdates, - resolver, - namePrefix :+ col.name) - - // construct a new struct with updated field expressions - toNamedStruct(ArraySeq.unsafeWrapArray(fields), updatedFieldExprs) - - case otherType => - val colName = (namePrefix :+ col.name).mkString(".") - throw new AnalysisException( - "Updating nested fields is only supported for StructType " + - s"but $colName is of type $otherType", - Array.empty[String]) - } - - // if there are conflicting updates, throw an exception - // there are two illegal scenarios: - // - multiple updates to the same column - // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c) - case updates if hasExactMatch(updates, col, resolver) => - val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString(".")) - throw new AnalysisException( - "Updates are in conflict for these columns: " + - conflictingCols.distinct.mkString(", "), - Array.empty[String]) - } - } - } - - private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = { - val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) => - Seq(Literal(field.name), expr) - } - CreateNamedStruct(namedStructExprs) - } - - private def hasExactMatch( - updates: Seq[ColumnUpdate], - col: NamedExpression, - resolver: Resolver): Boolean = { - - updates.exists(assignment => isExactMatch(assignment, col, resolver)) - } - - private def isExactMatch( - update: ColumnUpdate, - col: NamedExpression, - resolver: Resolver): Boolean = { - - update.ref match { - case Seq(namePart) if resolver(namePart, col.name) => true - case _ => false - } - } - - protected def castIfNeeded( - tableAttr: NamedExpression, - expr: Expression, - resolver: Resolver): Expression = { - - val storeAssignmentPolicy = conf.storeAssignmentPolicy - - // run the type check and catch type errors - storeAssignmentPolicy match { - case StoreAssignmentPolicy.STRICT | StoreAssignmentPolicy.ANSI => - if (expr.nullable && !tableAttr.nullable) { - throw new AnalysisException( - s"Cannot write nullable values to non-null column '${tableAttr.name}'", - Array.empty[String]) - } - - // use byName = true to catch cases when struct field names don't match - // e.g. a struct with fields (a, b) is assigned as a struct with fields (a, c) or (b, a) - val errors = new mutable.ArrayBuffer[String]() - val canWrite = DataType.canWrite( - expr.dataType, - tableAttr.dataType, - byName = true, - resolver, - tableAttr.name, - storeAssignmentPolicy, - err => errors += err) - - if (!canWrite) { - throw new AnalysisException( - s"Cannot write incompatible data:\n- ${errors.mkString("\n- ")}", - Array.empty[String]) - } - - case _ => // OK - } - - storeAssignmentPolicy match { - case _ if tableAttr.dataType.sameType(expr.dataType) => - expr - case StoreAssignmentPolicy.ANSI => - AnsiCast(expr, tableAttr.dataType, Option(conf.sessionLocalTimeZone)) - case _ => - Cast(expr, tableAttr.dataType, Option(conf.sessionLocalTimeZone)) - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MixedFormatExtendedDataSourceV2Strategy.scala b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MixedFormatExtendedDataSourceV2Strategy.scala deleted file mode 100644 index d4f52f226d..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MixedFormatExtendedDataSourceV2Strategy.scala +++ /dev/null @@ -1,218 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.v2 - -import scala.jdk.CollectionConverters._ - -import org.apache.iceberg.spark.Spark3Util -import org.apache.iceberg.spark.SparkCatalog -import org.apache.iceberg.spark.SparkSessionCatalog -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.Strategy -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.catalyst.expressions.PredicateHelper -import org.apache.spark.sql.catalyst.plans.logical.AddPartitionField -import org.apache.spark.sql.catalyst.plans.logical.Call -import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceBranch -import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceTag -import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable -import org.apache.spark.sql.catalyst.plans.logical.DropBranch -import org.apache.spark.sql.catalyst.plans.logical.DropIdentifierFields -import org.apache.spark.sql.catalyst.plans.logical.DropPartitionField -import org.apache.spark.sql.catalyst.plans.logical.DropTag -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.plans.logical.MergeRows -import org.apache.spark.sql.catalyst.plans.logical.NoStatsUnaryNode -import org.apache.spark.sql.catalyst.plans.logical.ReplaceData -import org.apache.spark.sql.catalyst.plans.logical.ReplacePartitionField -import org.apache.spark.sql.catalyst.plans.logical.SetIdentifierFields -import org.apache.spark.sql.catalyst.plans.logical.SetWriteDistributionAndOrdering -import org.apache.spark.sql.catalyst.plans.logical.WriteDelta -import org.apache.spark.sql.connector.catalog.Identifier -import org.apache.spark.sql.connector.catalog.TableCatalog -import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.datasources.DataSourceStrategy - -import org.apache.amoro.spark.{SparkUnifiedCatalog, SparkUnifiedSessionCatalog} - -/** - * refer apache iceberg project - * spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/ - * datasources/v2/ExtendedDataSourceV2Strategy.scala - * - * @param spark - */ -case class MixedFormatExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy - with PredicateHelper { - - import DataSourceV2Implicits._ - - override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case c @ Call(procedure, args) => - val input = buildInternalRow(args) - CallExec(c.output, procedure, input) :: Nil - - case AddPartitionField(IcebergCatalogAndIdentifier(catalog, ident), transform, name) => - AddPartitionFieldExec(catalog, ident, transform, name) :: Nil - - case CreateOrReplaceBranch( - IcebergCatalogAndIdentifier(catalog, ident), - branch, - branchOptions, - create, - replace, - ifNotExists) => - CreateOrReplaceBranchExec( - catalog, - ident, - branch, - branchOptions, - create, - replace, - ifNotExists) :: Nil - - case CreateOrReplaceTag( - IcebergCatalogAndIdentifier(catalog, ident), - tag, - tagOptions, - create, - replace, - ifNotExists) => - CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, create, replace, ifNotExists) :: Nil - - case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) => - DropBranchExec(catalog, ident, branch, ifExists) :: Nil - - case DropTag(IcebergCatalogAndIdentifier(catalog, ident), tag, ifExists) => - DropTagExec(catalog, ident, tag, ifExists) :: Nil - - case DropPartitionField(IcebergCatalogAndIdentifier(catalog, ident), transform) => - DropPartitionFieldExec(catalog, ident, transform) :: Nil - - case ReplacePartitionField( - IcebergCatalogAndIdentifier(catalog, ident), - transformFrom, - transformTo, - name) => - ReplacePartitionFieldExec(catalog, ident, transformFrom, transformTo, name) :: Nil - - case SetIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) => - SetIdentifierFieldsExec(catalog, ident, fields) :: Nil - - case DropIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) => - DropIdentifierFieldsExec(catalog, ident, fields) :: Nil - - case SetWriteDistributionAndOrdering( - IcebergCatalogAndIdentifier(catalog, ident), - distributionMode, - ordering) => - SetWriteDistributionAndOrderingExec(catalog, ident, distributionMode, ordering) :: Nil - - case ReplaceData(_: DataSourceV2Relation, query, r: DataSourceV2Relation, Some(write)) => - // refresh the cache using the original relation - ReplaceDataExec(planLater(query), refreshCache(r), write) :: Nil - - case WriteDelta(_: DataSourceV2Relation, query, r: DataSourceV2Relation, projs, Some(write)) => - // refresh the cache using the original relation - WriteDeltaExec(planLater(query), refreshCache(r), projs, write) :: Nil - - case MergeRows( - isSourceRowPresent, - isTargetRowPresent, - matchedConditions, - matchedOutputs, - notMatchedConditions, - notMatchedOutputs, - targetOutput, - rowIdAttrs, - performCardinalityCheck, - emitNotMatchedTargetRows, - output, - child) => - MergeRowsExec( - isSourceRowPresent, - isTargetRowPresent, - matchedConditions, - matchedOutputs, - notMatchedConditions, - notMatchedOutputs, - targetOutput, - rowIdAttrs, - performCardinalityCheck, - emitNotMatchedTargetRows, - output, - planLater(child)) :: Nil - - case DeleteFromIcebergTable(DataSourceV2ScanRelation(r, _, output), condition, None) => - // the optimizer has already checked that this delete can be handled using a metadata operation - val deleteCond = condition.getOrElse(Literal.TrueLiteral) - val predicates = splitConjunctivePredicates(deleteCond) - val normalizedPredicates = DataSourceStrategy.normalizeExprs(predicates, output) - val filters = normalizedPredicates.flatMap { pred => - val filter = DataSourceStrategy.translateFilter(pred, supportNestedPredicatePushdown = true) - if (filter.isEmpty) { - throw QueryCompilationErrors.cannotTranslateExpressionToSourceFilterError(pred) - } - filter - }.toArray - DeleteFromTableExec(r.table.asDeletable, filters, refreshCache(r)) :: Nil - - case NoStatsUnaryNode(child) => - planLater(child) :: Nil - - case _ => Nil - } - - private def buildInternalRow(exprs: Seq[Expression]): InternalRow = { - val values = new Array[Any](exprs.size) - for (index <- exprs.indices) { - values(index) = exprs(index).eval() - } - new GenericInternalRow(values) - } - - private def refreshCache(r: DataSourceV2Relation)(): Unit = { - spark.sharedState.cacheManager.recacheByPlan(spark, r) - } - - /** - * support UnifiedSessionCatalog - */ - private object IcebergCatalogAndIdentifier { - def unapply(identifier: Seq[String]): Option[(TableCatalog, Identifier)] = { - val catalogAndIdentifier = Spark3Util.catalogAndIdentifier(spark, identifier.asJava) - catalogAndIdentifier.catalog match { - case icebergCatalog: SparkCatalog => - Some((icebergCatalog, catalogAndIdentifier.identifier)) - case icebergCatalog: SparkSessionCatalog[_] => - Some((icebergCatalog, catalogAndIdentifier.identifier)) - case icebergCatalog: SparkUnifiedCatalog => - Some((icebergCatalog, catalogAndIdentifier.identifier)) - case icebergCatalog: SparkUnifiedSessionCatalog[_] => - Some((icebergCatalog, catalogAndIdentifier.identifier)) - case _ => - None - } - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/TestUnifiedCatalog.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/TestUnifiedCatalog.java deleted file mode 100644 index 1cbafa88d5..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/TestUnifiedCatalog.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.spark.test.unified.UnifiedCatalogTestSuites; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; - -public class TestUnifiedCatalog extends UnifiedCatalogTestSuites { - - @ParameterizedTest - @MethodSource - public void testTableFormats(TableFormat format, boolean sessionCatalog) { - super.testTableFormats(format, sessionCatalog); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/TestImpalaParquet.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/TestImpalaParquet.java deleted file mode 100644 index 02d31d5197..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/TestImpalaParquet.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test; - -import org.apache.amoro.hive.HMSMockServer; -import org.apache.amoro.shade.guava32.com.google.common.collect.Iterators; -import org.apache.amoro.spark.reader.SparkParquetReaders; -import org.apache.amoro.spark.test.utils.ResourceInputFile; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.MappedField; -import org.apache.iceberg.mapping.NameMapping; -import org.apache.iceberg.parquet.AdaptHiveParquet; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.catalyst.InternalRow; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.util.HashMap; - -public class TestImpalaParquet { - - InputFile targetInputParquet; - - @BeforeEach - public void setup() { - targetInputParquet = - ResourceInputFile.newFile(HMSMockServer.class.getClassLoader(), "string_is_bytes.parquet"); - } - - @Test - public void sparkRead() { - NameMapping mapping = NameMapping.of(MappedField.of(1, "str")); - Schema schema = new Schema(Types.NestedField.of(1, true, "str", Types.StringType.get())); - - AdaptHiveParquet.ReadBuilder builder = - AdaptHiveParquet.read(targetInputParquet) - .project(schema) - .withNameMapping(mapping) - .createReaderFunc( - fileSchema -> SparkParquetReaders.buildReader(schema, fileSchema, new HashMap<>())) - .caseSensitive(false); - CloseableIterator iterator = builder.build().iterator(); - while (iterator.hasNext()) { - InternalRow next = (InternalRow) iterator.next(); - Assertions.assertEquals("hello parquet", next.getString(0)); - } - } - - @Test - public void genericFilter() { - NameMapping mapping = NameMapping.of(MappedField.of(1, "str")); - Schema schema = new Schema(Types.NestedField.of(1, true, "str", Types.StringType.get())); - AdaptHiveParquet.ReadBuilder builder = - AdaptHiveParquet.read(targetInputParquet) - .project(schema) - .withNameMapping(mapping) - .filter(Expressions.in("str", "aa")) - .createReaderFunc( - fileSchema -> SparkParquetReaders.buildReader(schema, fileSchema, new HashMap<>())) - .caseSensitive(false); - CloseableIterator iterator = builder.build().iterator(); - Assertions.assertEquals(0, Iterators.size(iterator)); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/api/TestKeyedTableDataFrameAPI.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/api/TestKeyedTableDataFrameAPI.java deleted file mode 100644 index a63c161f29..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/api/TestKeyedTableDataFrameAPI.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.api; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.iceberg.Schema; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.types.StructType; -import org.junit.Assert; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.List; -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestKeyedTableDataFrameAPI extends MixedTableTestBase { - final Schema schema = - new Schema( - Types.NestedField.of(1, false, "id", Types.IntegerType.get()), - Types.NestedField.of(2, false, "data", Types.StringType.get()), - Types.NestedField.of(3, false, "day", Types.StringType.get())); - Dataset df; - - public static Stream testV2ApiKeyedTable() { - return Stream.of(Arguments.of(TableFormat.MIXED_HIVE), Arguments.of(TableFormat.MIXED_ICEBERG)); - } - - @DisplayName("Test `test V2 Api for KeyedTable`") - @ParameterizedTest - @MethodSource - public void testV2ApiKeyedTable(TableFormat format) throws Exception { - String tablePath = target().catalog + "." + target().database + "." + target().table; - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id int, data string, day string , primary key (id)) using " - + provider(format) - + " partitioned by (day)"; - - sql(sqlText); - - // test overwrite partitions - StructType structType = SparkSchemaUtil.convert(schema); - df = - spark() - .createDataFrame( - Lists.newArrayList( - RowFactory.create(1, "aaa", "aaa"), - RowFactory.create(2, "bbb", "bbb"), - RowFactory.create(3, "ccc", "ccc")), - structType); - df.writeTo(tablePath).overwritePartitions(); - - df = spark().read().table(tablePath); - Assertions.assertEquals(3, df.count()); - - df = - spark() - .createDataFrame( - Lists.newArrayList( - RowFactory.create(4, "aaa", "ccc"), - RowFactory.create(5, "bbb", "ddd"), - RowFactory.create(6, "ccc", "eee")), - structType); - df.writeTo(tablePath).overwritePartitions(); - df = spark().read().table(tablePath); - Assertions.assertEquals(5, df.count()); - } - - public static Stream testKeyedTableDataFrameApi() { - return Stream.of(Arguments.of(TableFormat.MIXED_HIVE), Arguments.of(TableFormat.MIXED_ICEBERG)); - } - - @DisplayName("Test `test DataFrameApi for KeyedTable`") - @ParameterizedTest - @MethodSource - public void testKeyedTableDataFrameApi(TableFormat format) throws Exception { - String tablePath = target().catalog + "." + target().database + "." + target().table; - StructType structType = SparkSchemaUtil.convert(schema); - // test create - df = - spark() - .createDataFrame( - Lists.newArrayList( - RowFactory.create(1, "aaa", "aaa"), - RowFactory.create(2, "bbb", "bbb"), - RowFactory.create(3, "ccc", "ccc")), - structType); - df.write() - .format(provider(format)) - .partitionBy("day") - .option("primary.keys", "id") - .save(tablePath); - List rows = sql("desc " + target().database + "." + target().table).collectAsList(); - assertTableDesc(rows, Lists.newArrayList("id"), Lists.newArrayList("day")); - df = spark().read().table(tablePath); - Assertions.assertEquals(3, df.count()); - - // test overwrite dynamic - df = - spark() - .createDataFrame( - Lists.newArrayList( - RowFactory.create(4, "aaa", "aaa"), - RowFactory.create(5, "aaa", "bbb"), - RowFactory.create(6, "aaa", "ccc")), - structType); - df.write() - .format(provider(format)) - .partitionBy("day") - .option("overwrite-mode", "dynamic") - .mode(SaveMode.Overwrite) - .save(tablePath); - df = spark().read().format("arctic").load(tablePath); - Assert.assertEquals(3, df.count()); - - df = - spark() - .createDataFrame( - Lists.newArrayList( - RowFactory.create(4, "aaa", "ccc"), - RowFactory.create(5, "bbb", "ddd"), - RowFactory.create(6, "ccc", "eee")), - structType); - df.writeTo(tablePath).overwritePartitions(); - df = spark().read().table(tablePath); - Assertions.assertEquals(5, df.count()); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/api/TestUnKeyedTableDataFrameAPI.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/api/TestUnKeyedTableDataFrameAPI.java deleted file mode 100644 index 508de67a6c..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/api/TestUnKeyedTableDataFrameAPI.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.api; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.iceberg.Schema; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.Column; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.types.StructType; -import org.junit.Assert; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestUnKeyedTableDataFrameAPI extends MixedTableTestBase { - - final Schema schema = - new Schema( - Types.NestedField.of(1, false, "id", Types.IntegerType.get()), - Types.NestedField.of(2, false, "data", Types.StringType.get()), - Types.NestedField.of(3, false, "day", Types.StringType.get())); - Dataset df; - - public static Stream testV2ApiUnkeyedTable() { - return Stream.of(Arguments.of(TableFormat.MIXED_HIVE), Arguments.of(TableFormat.MIXED_ICEBERG)); - } - - @DisplayName("Test `test V2 Api for UnkeyedTable`") - @ParameterizedTest - @MethodSource - public void testV2ApiUnkeyedTable(TableFormat format) throws Exception { - String tablePath = target().catalog + "." + target().database + "." + target().table; - StructType structType = SparkSchemaUtil.convert(schema); - - // create test - df = - spark() - .createDataFrame( - Lists.newArrayList( - RowFactory.create(1, "aaa", "aaa"), - RowFactory.create(2, "bbb", "bbb"), - RowFactory.create(3, "ccc", "ccc")), - structType); - df.writeTo(tablePath).partitionedBy(new Column("day")).create(); - df = spark().read().table(tablePath); - Assert.assertEquals(3, df.count()); - - // append test - df = - spark() - .createDataFrame( - Lists.newArrayList( - RowFactory.create(4, "aaa", "aaa"), - RowFactory.create(5, "bbb", "bbb"), - RowFactory.create(6, "ccc", "ccc")), - structType); - df.writeTo(tablePath).append(); - df = spark().read().table(tablePath); - Assertions.assertEquals(6, df.count()); - - // overwritePartition test - df = - spark() - .createDataFrame( - Lists.newArrayList( - RowFactory.create(10, "ccc", "ccc"), - RowFactory.create(11, "ddd", "ddd"), - RowFactory.create(12, "eee", "eee")), - structType); - df.writeTo(tablePath).overwritePartitions(); - df = spark().read().table(tablePath); - Assertions.assertEquals(7, df.count()); - } - - public static Stream testV1ApiUnkeyedTable() { - return Stream.of(Arguments.of(TableFormat.MIXED_HIVE), Arguments.of(TableFormat.MIXED_ICEBERG)); - } - - @DisplayName("Test `test V1 Api for UnkeyedTable`") - @ParameterizedTest - @MethodSource - public void testV1ApiUnkeyedTable(TableFormat format) { - String tablePath = target().catalog + "." + target().database + "." + target().table; - StructType structType = SparkSchemaUtil.convert(schema); - - // test create - df = - spark() - .createDataFrame( - Lists.newArrayList( - RowFactory.create(1, "aaa", "aaa"), - RowFactory.create(2, "bbb", "bbb"), - RowFactory.create(3, "ccc", "ccc")), - structType); - df.write().format("arctic").partitionBy("day").save(tablePath); - df = spark().read().format("arctic").load(tablePath); - Assertions.assertEquals(3, df.count()); - - // test overwrite dynamic - df = - spark() - .createDataFrame( - Lists.newArrayList( - RowFactory.create(4, "aaa", "ccc"), - RowFactory.create(5, "bbb", "ccc"), - RowFactory.create(6, "ccc", "ccc")), - structType); - df.write() - .format("arctic") - .partitionBy("day") - .option("overwrite-mode", "dynamic") - .mode(SaveMode.Overwrite) - .save(tablePath); - df = spark().read().format("arctic").load(tablePath); - Assertions.assertEquals(5, df.count()); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/catalog/TestSessionCatalog.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/catalog/TestSessionCatalog.java deleted file mode 100644 index 5785abec30..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/catalog/TestSessionCatalog.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.catalog; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableMap; -import org.apache.amoro.spark.table.MixedSparkTable; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.SparkTestContext; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -import java.util.Map; - -public class TestSessionCatalog extends MixedTableTestBase { - - public static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get()), - Types.NestedField.required(3, "pt", Types.StringType.get())); - public static final PrimaryKeySpec PK_SPEC = - PrimaryKeySpec.builderFor(SCHEMA).addColumn("id").build(); - public static final PartitionSpec PT_SPEC = - PartitionSpec.builderFor(SCHEMA).identity("pt").build(); - - @Override - protected Map sparkSessionConfig() { - return ImmutableMap.of( - "spark.sql.catalog.spark_catalog", - SparkTestContext.SESSION_CATALOG_IMPL, - "spark.sql.catalog.spark_catalog.url", - CONTEXT.amsCatalogUrl(TableFormat.MIXED_ICEBERG)); - } - - @Test - public void testLoadTables() throws NoSuchTableException { - createTarget(SCHEMA, builder -> builder.withPrimaryKeySpec(PK_SPEC).withPartitionSpec(PT_SPEC)); - - TableCatalog sessionCatalog = - (TableCatalog) spark().sessionState().catalogManager().catalog(SPARK_SESSION_CATALOG); - - Table table = sessionCatalog.loadTable(target().toSparkIdentifier()); - Assertions.assertTrue(table instanceof MixedSparkTable); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestAlterTableColumnSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestAlterTableColumnSQL.java deleted file mode 100644 index 3dd63f75ba..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestAlterTableColumnSQL.java +++ /dev/null @@ -1,312 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.AnalysisException; -import org.junit.Assert; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestAlterTableColumnSQL extends MixedTableTestBase { - - public static Stream testAddColumn() { - return Stream.of( - Arguments.of( - TableFormat.MIXED_HIVE, - "", - Types.StructType.of( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional( - 4, - "point", - Types.StructType.of( - Types.NestedField.required(5, "x", Types.DoubleType.get()), - Types.NestedField.required(6, "y", Types.DoubleType.get()))), - Types.NestedField.optional(3, "ts", Types.StringType.get()))), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "", - Types.StructType.of( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.StringType.get()), - Types.NestedField.optional( - 4, - "point", - Types.StructType.of( - Types.NestedField.required(5, "x", Types.DoubleType.get()), - Types.NestedField.required(6, "y", Types.DoubleType.get()))))), - Arguments.of( - TableFormat.MIXED_HIVE, - ", PRIMARY KEY(id)", - Types.StructType.of( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional( - 4, - "point", - Types.StructType.of( - Types.NestedField.required(5, "x", Types.DoubleType.get()), - Types.NestedField.required(6, "y", Types.DoubleType.get()))), - Types.NestedField.optional(3, "ts", Types.StringType.get()))), - Arguments.of( - TableFormat.MIXED_ICEBERG, - ", PRIMARY KEY(id)", - Types.StructType.of( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.StringType.get()), - Types.NestedField.optional( - 4, - "point", - Types.StructType.of( - Types.NestedField.required(5, "x", Types.DoubleType.get()), - Types.NestedField.required(6, "y", Types.DoubleType.get())))))); - } - - @DisplayName("Test `add column`") - @ParameterizedTest - @MethodSource() - public void testAddColumn( - TableFormat format, String primaryKeyDDL, Types.StructType expectedSchema) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id bigint, data string, ts string " - + primaryKeyDDL - + " ) using " - + provider(format) - + " PARTITIONED BY (ts)"; - sql(sqlText); - sql( - "ALTER TABLE " - + target().database - + "." - + target().table - + " ADD COLUMN point struct"); - - Assertions.assertEquals( - expectedSchema, loadTable().schema().asStruct(), "Schema should match expected"); - } - - public static Stream testDropColumn() { - return Stream.of( - Arguments.of( - TableFormat.MIXED_ICEBERG, - ", PRIMARY KEY(id)", - Types.StructType.of( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.optional(3, "ts", Types.StringType.get()))), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "", - Types.StructType.of( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(3, "ts", Types.StringType.get())))); - } - - @DisplayName("Test `drop column`") - @ParameterizedTest - @MethodSource() - public void testDropColumn( - TableFormat format, String primaryKeyDDL, Types.StructType expectedSchema) { - - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id bigint, data string, ts string " - + primaryKeyDDL - + " ) using " - + provider(format) - + " PARTITIONED BY (ts)"; - sql(sqlText); - sql("ALTER TABLE " + target().database + "." + target().table + " DROP COLUMN data"); - - Assertions.assertEquals( - expectedSchema, loadTable().schema().asStruct(), "Schema should match expected"); - } - - /** TODO: are arguments could be simplify? TODO: Test coverage is not enough. */ - public static Stream testAlterColumn() { - return Stream.of( - Arguments.of( - " id COMMENT 'Record id'", - "", - Types.StructType.of( - Types.NestedField.optional(1, "id", Types.LongType.get(), "Record id"), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.TimestampType.withZone()), - Types.NestedField.optional(4, "count", Types.IntegerType.get()))), - Arguments.of( - " count TYPE bigint", - "", - Types.StructType.of( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.TimestampType.withZone()), - Types.NestedField.optional(4, "count", Types.LongType.get()))), - Arguments.of( - " data DROP NOT NULL", - "", - Types.StructType.of( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.TimestampType.withZone()), - Types.NestedField.optional(4, "count", Types.IntegerType.get()))), - Arguments.of(" data SET NOT NULL", "", null), - Arguments.of( - " count AFTER id", - "", - Types.StructType.of( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(4, "count", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.TimestampType.withZone()))), - Arguments.of( - " id COMMENT 'Record id'", - ", PRIMARY KEY(id)", - Types.StructType.of( - Types.NestedField.required(1, "id", Types.LongType.get(), "Record id"), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.TimestampType.withZone()), - Types.NestedField.optional(4, "count", Types.IntegerType.get()))), - Arguments.of( - " count TYPE bigint", - ", PRIMARY KEY(id)", - Types.StructType.of( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.TimestampType.withZone()), - Types.NestedField.optional(4, "count", Types.LongType.get()))), - Arguments.of( - " data DROP NOT NULL", - ", PRIMARY KEY(id)", - Types.StructType.of( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.TimestampType.withZone()), - Types.NestedField.optional(4, "count", Types.IntegerType.get()))), - Arguments.of(" data SET NOT NULL", ", PRIMARY KEY(id)", null), - Arguments.of( - " count AFTER id", - ", PRIMARY KEY(id)", - Types.StructType.of( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.optional(4, "count", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.TimestampType.withZone())))); - } - - @DisplayName("Test `alter column`") - @ParameterizedTest - @MethodSource() - @EnableCatalogSelect.SelectCatalog(use = MIXED_ICEBERG_CATALOG) - public void testAlterColumn( - String alterText, String primaryKeyDDL, Types.StructType expectedSchema) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id bigint, data string, ts timestamp, count int " - + primaryKeyDDL - + " ) using " - + provider(TableFormat.MIXED_ICEBERG) - + " PARTITIONED BY (ts)"; - sql(sqlText); - if (expectedSchema != null) { - sql("ALTER TABLE " + target().database + "." + target().table + " ALTER COLUMN " + alterText); - Assertions.assertEquals( - expectedSchema, loadTable().schema().asStruct(), "Schema should match expected"); - } else { - Assert.assertThrows( - AnalysisException.class, - () -> - sql( - "ALTER TABLE " - + target().database - + "." - + target().table - + " ALTER COLUMN " - + alterText)); - } - } - - public static Stream testAlterTableProperties() { - return Stream.of( - Arguments.of( - TableFormat.MIXED_ICEBERG, "", " SET TBLPROPERTIES ('test.props' = 'val')", "val"), - Arguments.of( - TableFormat.MIXED_HIVE, "", " SET TBLPROPERTIES ('test.props' = 'val')", "val"), - Arguments.of(TableFormat.MIXED_ICEBERG, "", " UNSET TBLPROPERTIES ('test.props')", null), - Arguments.of(TableFormat.MIXED_HIVE, "", " UNSET TBLPROPERTIES ('test.props')", null), - Arguments.of( - TableFormat.MIXED_ICEBERG, - ", PRIMARY KEY(id)", - " SET TBLPROPERTIES ('test.props' = 'val')", - "val"), - Arguments.of( - TableFormat.MIXED_HIVE, - ", PRIMARY KEY(id)", - " SET TBLPROPERTIES ('test.props' = 'val')", - "val"), - Arguments.of( - TableFormat.MIXED_ICEBERG, - ", PRIMARY KEY(id)", - " UNSET TBLPROPERTIES ('test.props')", - null), - Arguments.of( - TableFormat.MIXED_HIVE, - ", PRIMARY KEY(id)", - " UNSET TBLPROPERTIES ('test.props')", - null)); - } - - @DisplayName("Test `alter table properties`") - @ParameterizedTest - @MethodSource() - public void testAlterTableProperties( - TableFormat format, String primaryKeyDDL, String alterText, String expectedProperties) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id bigint, data string, ts string " - + primaryKeyDDL - + " ) using " - + provider(format) - + " PARTITIONED BY (ts)"; - sql(sqlText); - sql("ALTER TABLE " + target().database + "." + target().table + alterText); - Assertions.assertEquals(expectedProperties, loadTable().properties().get("test.props")); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestCreateTableAsSelect.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestCreateTableAsSelect.java deleted file mode 100644 index 04b61a76f3..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestCreateTableAsSelect.java +++ /dev/null @@ -1,336 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableMap; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.mixed.SparkSQLProperties; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.amoro.spark.test.utils.Asserts; -import org.apache.amoro.spark.test.utils.DataComparator; -import org.apache.amoro.spark.test.utils.TableFiles; -import org.apache.amoro.spark.test.utils.TestTableUtil; -import org.apache.amoro.spark.test.utils.TestTables; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestCreateTableAsSelect extends MixedTableTestBase { - - public static final Schema SIMPLE_SOURCE_SCHEMA = TestTables.MixedIceberg.NO_PK_PT.schema; - public static final List SIMPLE_SOURCE_DATA = - TestTables.MixedIceberg.PK_PT.newDateGen().records(10); - - public static Stream testTimestampZoneHandle() { - return Stream.of( - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PRIMARY KEY(id, pt)", - true, - Types.TimestampType.withoutZone()), - Arguments.of(TableFormat.MIXED_ICEBERG, "", false, Types.TimestampType.withZone()), - Arguments.of( - TableFormat.MIXED_HIVE, "PRIMARY KEY(id, pt)", true, Types.TimestampType.withoutZone()), - Arguments.of(TableFormat.MIXED_HIVE, "", false, Types.TimestampType.withoutZone())); - } - - @ParameterizedTest - @MethodSource - public void testTimestampZoneHandle( - TableFormat format, - String primaryKeyDDL, - boolean timestampWithoutZone, - Types.TimestampType expectType) { - createViewSource(SIMPLE_SOURCE_SCHEMA, SIMPLE_SOURCE_DATA); - spark() - .conf() - .set( - SparkSQLProperties.USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES, timestampWithoutZone); - - String sqlText = - "CREATE TABLE " - + target() - + " " - + primaryKeyDDL - + " USING " - + provider(format) - + " AS SELECT * FROM " - + source(); - sql(sqlText); - - MixedTable table = loadTable(); - Types.NestedField f = table.schema().findField("ts"); - Asserts.assertType(expectType, f.type()); - } - - private static PartitionSpec.Builder ptBuilder() { - return PartitionSpec.builderFor(SIMPLE_SOURCE_SCHEMA); - } - - public static Stream testSchemaAndData() { - PrimaryKeySpec keyIdPtSpec = - PrimaryKeySpec.builderFor(SIMPLE_SOURCE_SCHEMA).addColumn("id").addColumn("pt").build(); - PrimaryKeySpec keyIdSpec = - PrimaryKeySpec.builderFor(SIMPLE_SOURCE_SCHEMA).addColumn("id").build(); - - return Stream.of( - Arguments.of( - TableFormat.MIXED_HIVE, - "PRIMARY KEY(id, pt)", - "PARTITIONED BY(pt)", - keyIdPtSpec, - ptBuilder().identity("pt").build()), - Arguments.of( - TableFormat.MIXED_HIVE, - "PRIMARY KEY(id, pt)", - "", - keyIdPtSpec, - PartitionSpec.unpartitioned()), - Arguments.of( - TableFormat.MIXED_HIVE, - "", - "PARTITIONED BY(pt)", - PrimaryKeySpec.noPrimaryKey(), - ptBuilder().identity("pt").build()), - Arguments.of( - TableFormat.MIXED_HIVE, - "", - "", - PrimaryKeySpec.noPrimaryKey(), - PartitionSpec.unpartitioned()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PRIMARY KEY(id, pt)", - "", - keyIdPtSpec, - PartitionSpec.unpartitioned()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "", - "PARTITIONED BY(pt,id)", - PrimaryKeySpec.noPrimaryKey(), - ptBuilder().identity("pt").identity("id").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PRIMARY KEY(id)", - "PARTITIONED BY(years(ts))", - keyIdSpec, - ptBuilder().year("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PRIMARY KEY(id)", - "PARTITIONED BY(months(ts))", - keyIdSpec, - ptBuilder().month("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PRIMARY KEY(id)", - "PARTITIONED BY(days(ts))", - keyIdSpec, - ptBuilder().day("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PRIMARY KEY(id)", - "PARTITIONED BY(date(ts))", - keyIdSpec, - ptBuilder().day("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PRIMARY KEY(id)", - "PARTITIONED BY(hours(ts))", - keyIdSpec, - ptBuilder().hour("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PRIMARY KEY(id)", - "PARTITIONED BY(date_hour(ts))", - keyIdSpec, - ptBuilder().hour("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PRIMARY KEY(id)", - "PARTITIONED BY(bucket(10, id))", - keyIdSpec, - ptBuilder().bucket("id", 10).build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PRIMARY KEY(id)", - "PARTITIONED BY(truncate(10, data))", - keyIdSpec, - ptBuilder().truncate("data", 10).build())); - } - - @ParameterizedTest - @MethodSource - public void testSchemaAndData( - TableFormat format, - String primaryKeyDDL, - String partitionDDL, - PrimaryKeySpec keySpec, - PartitionSpec ptSpec) { - spark().conf().set("spark.sql.session.timeZone", "UTC"); - createViewSource(SIMPLE_SOURCE_SCHEMA, SIMPLE_SOURCE_DATA); - - spark().conf().set(SparkSQLProperties.USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES, true); - - String sqlText = - "CREATE TABLE " - + target() - + " " - + primaryKeyDDL - + " USING " - + provider(format) - + " " - + partitionDDL - + " AS SELECT * FROM " - + source(); - sql(sqlText); - - Schema expectSchema = TestTableUtil.toSchemaWithPrimaryKey(SIMPLE_SOURCE_SCHEMA, keySpec); - expectSchema = TestTableUtil.timestampToWithoutZone(expectSchema); - - MixedTable table = loadTable(); - Asserts.assertPartition(ptSpec, table.spec()); - Assertions.assertEquals(keySpec.primaryKeyExisted(), table.isKeyedTable()); - if (table.isKeyedTable()) { - Asserts.assertPrimaryKey(keySpec, table.asKeyedTable().primaryKeySpec()); - } - Asserts.assertType(expectSchema.asStruct(), table.schema().asStruct()); - TableFiles files = TestTableUtil.files(table); - Asserts.assertAllFilesInBaseStore(files); - - if (TableFormat.MIXED_HIVE.equals(format)) { - Table hiveTable = loadHiveTable(); - Asserts.assertHiveColumns(expectSchema, ptSpec, hiveTable.getSd().getCols()); - Asserts.assertHivePartition(ptSpec, hiveTable.getPartitionKeys()); - Asserts.assertAllFilesInHiveLocation(files, hiveTable.getSd().getLocation()); - } - - List records = TestTableUtil.tableRecords(table); - DataComparator.build(SIMPLE_SOURCE_DATA, records) - .ignoreOrder(Comparator.comparing(r -> (Integer) r.get(0))) - .assertRecordsEqual(); - } - - public static Stream testSourceDuplicateCheck() { - List duplicateSource = Lists.newArrayList(SIMPLE_SOURCE_DATA); - duplicateSource.add(SIMPLE_SOURCE_DATA.get(0)); - - return Stream.of( - Arguments.of(TableFormat.MIXED_ICEBERG, SIMPLE_SOURCE_DATA, "PRIMARY KEY(id, pt)", false), - Arguments.of(TableFormat.MIXED_ICEBERG, SIMPLE_SOURCE_DATA, "", false), - Arguments.of(TableFormat.MIXED_ICEBERG, duplicateSource, "", false), - Arguments.of(TableFormat.MIXED_ICEBERG, duplicateSource, "PRIMARY KEY(id, pt)", true), - Arguments.of(TableFormat.MIXED_HIVE, SIMPLE_SOURCE_DATA, "PRIMARY KEY(id, pt)", false), - Arguments.of(TableFormat.MIXED_HIVE, SIMPLE_SOURCE_DATA, "", false), - Arguments.of(TableFormat.MIXED_HIVE, duplicateSource, "", false), - Arguments.of(TableFormat.MIXED_HIVE, duplicateSource, "PRIMARY KEY(id, pt)", true)); - } - - @ParameterizedTest(name = "{index} {0} {2} {3}") - @MethodSource - public void testSourceDuplicateCheck( - TableFormat format, - List sourceData, - String primaryKeyDDL, - boolean duplicateCheckFailed) { - spark().conf().set(SparkSQLProperties.CHECK_SOURCE_DUPLICATES_ENABLE, "true"); - createViewSource(SIMPLE_SOURCE_SCHEMA, sourceData); - String sqlText = - "CREATE TABLE " - + target() - + " " - + primaryKeyDDL - + " USING " - + provider(format) - + " " - + " AS SELECT * FROM " - + source(); - - boolean exceptionCatched = false; - try { - sql(sqlText); - } catch (Exception e) { - exceptionCatched = true; - } - - Assertions.assertEquals(duplicateCheckFailed, exceptionCatched); - } - - public static Stream testAdditionProperties() { - String propertiesDDL = "TBLPROPERTIES('k1'='v1', 'k2'='v2')"; - Map expectProperties = ImmutableMap.of("k1", "v1", "k2", "v2"); - Map emptyProperties = Collections.emptyMap(); - return Stream.of( - Arguments.of(TableFormat.MIXED_ICEBERG, "PRIMARY KEY(id, pt)", "", emptyProperties), - Arguments.of( - TableFormat.MIXED_ICEBERG, "PRIMARY KEY(id, pt)", propertiesDDL, expectProperties), - Arguments.of(TableFormat.MIXED_ICEBERG, "", propertiesDDL, expectProperties), - Arguments.of(TableFormat.MIXED_HIVE, "PRIMARY KEY(id, pt)", "", emptyProperties), - Arguments.of( - TableFormat.MIXED_HIVE, "PRIMARY KEY(id, pt)", propertiesDDL, expectProperties), - Arguments.of(TableFormat.MIXED_HIVE, "", propertiesDDL, expectProperties)); - } - - @ParameterizedTest - @MethodSource - public void testAdditionProperties( - TableFormat format, - String primaryKeyDDL, - String propertiesDDL, - Map expectProperties) { - createViewSource(SIMPLE_SOURCE_SCHEMA, SIMPLE_SOURCE_DATA); - String sqlText = - "CREATE TABLE " - + target() - + " " - + primaryKeyDDL - + " USING " - + provider(format) - + " PARTITIONED BY (pt) " - + propertiesDDL - + " AS SELECT * FROM " - + source(); - sql(sqlText); - MixedTable table = loadTable(); - Map tableProperties = table.properties(); - Asserts.assertHashMapContainExpect(expectProperties, tableProperties); - } - - // TODO: test optimize write for ctas. - -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestCreateTableLikeSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestCreateTableLikeSQL.java deleted file mode 100644 index 1572315af8..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestCreateTableLikeSQL.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableMap; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.mixed.SparkSQLProperties; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.TestIdentifier; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.amoro.spark.test.utils.Asserts; -import org.apache.amoro.spark.test.utils.TestTable; -import org.apache.amoro.spark.test.utils.TestTables; -import org.apache.amoro.table.MixedTable; -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.List; -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestCreateTableLikeSQL extends MixedTableTestBase { - - public static Stream testTimestampZoneHandle() { - return Stream.of( - Arguments.of(TableFormat.MIXED_ICEBERG, false, Types.TimestampType.withZone()), - Arguments.of(TableFormat.MIXED_ICEBERG, true, Types.TimestampType.withoutZone()), - Arguments.of(TableFormat.MIXED_HIVE, false, Types.TimestampType.withoutZone()), - Arguments.of(TableFormat.MIXED_HIVE, true, Types.TimestampType.withoutZone())); - } - - @DisplayName("TestSQL: CREATE TABLE LIKE handle timestamp type in new table.") - @ParameterizedTest - @MethodSource - public void testTimestampZoneHandle( - TableFormat format, boolean newTableTimestampWithoutZone, Type expectTimestampType) { - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "ts", Types.TimestampType.withZone())); - createMixedFormatSource(schema, x -> {}); - - spark() - .conf() - .set( - SparkSQLProperties.USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES, - newTableTimestampWithoutZone); - - sql("CREATE TABLE " + target() + " LIKE " + source() + " USING " + provider(format)); - - MixedTable table = loadTable(); - Types.NestedField tsField = table.schema().findField("ts"); - Asserts.assertType(expectTimestampType, tsField.type()); - } - - public static Stream testCreateTableLikeHiveTable() { - return Stream.of( - Arguments.of(TestTables.MixedHive.NO_PK_NO_PT), - Arguments.of(TestTables.MixedHive.NO_PK_PT)); - } - - @DisplayName("Test SQL: CREATE TABLE LIKE hive table") - @ParameterizedTest - @MethodSource - @EnableCatalogSelect.SelectCatalog(use = SPARK_SESSION_CATALOG) - public void testCreateTableLikeHiveTable(TestTable source) { - createHiveSource(source.hiveSchema, source.hivePartitions, ImmutableMap.of("k1", "v1")); - - String sqlText = "CREATE TABLE " + target() + " LIKE " + source() + " USING arctic"; - sql(sqlText); - MixedTable table = loadTable(); - Asserts.assertType(source.schema.asStruct(), table.schema().asStruct()); - Asserts.assertPartition(source.ptSpec, table.spec()); - // CREATE TABLE LIKE do not copy properties. - Assertions.assertFalse(table.properties().containsKey("k1")); - } - - public static Stream testCreateTableLikeDataLakeTable() { - List tables = - Lists.newArrayList( - TestTables.MixedHive.NO_PK_NO_PT, - TestTables.MixedHive.NO_PK_PT, - TestTables.MixedHive.PK_NO_PT, - TestTables.MixedHive.PK_PT, - TestTables.MixedIceberg.NO_PK_NO_PT, - TestTables.MixedIceberg.PK_NO_PT, - TestTables.MixedIceberg.PK_PT, - TestTables.MixedIceberg.NO_PK_PT); - return tables.stream().map(t -> Arguments.of(t.format, t)); - } - - @DisplayName("Test SQL: CREATE TABLE LIKE data-lake table") - @ParameterizedTest - @MethodSource - public void testCreateTableLikeDataLakeTable(TableFormat format, TestTable source) { - MixedTable expect = - createMixedFormatSource( - source.schema, - builder -> - builder - .withPartitionSpec(source.ptSpec) - .withPrimaryKeySpec(source.keySpec) - .withProperty("k1", "v1")); - - spark().conf().set(SparkSQLProperties.USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES, true); - String sqlText = - "CREATE TABLE " + target() + " LIKE " + source() + " USING " + provider(format); - sql(sqlText); - - MixedTable table = loadTable(); - Asserts.assertType(expect.schema().asStruct(), table.schema().asStruct()); - Asserts.assertPartition(expect.spec(), table.spec()); - Assertions.assertEquals(expect.isKeyedTable(), table.isKeyedTable()); - // CREATE TABLE LIKE do not copy properties. - Assertions.assertFalse(table.properties().containsKey("k1")); - if (expect.isKeyedTable()) { - Asserts.assertPrimaryKey( - expect.asKeyedTable().primaryKeySpec(), table.asKeyedTable().primaryKeySpec()); - } - } - - public static Stream testCreateTableWithoutProviderInSessionCatalog() { - return Stream.of(Arguments.of("", false), Arguments.of("USING arctic", true)); - } - - @DisplayName("TestSQL: CREATE TABLE LIKE without USING ARCTIC") - @ParameterizedTest(name = "{index} provider = {0} ") - @MethodSource - @EnableCatalogSelect.SelectCatalog(use = SPARK_SESSION_CATALOG) - public void testCreateTableWithoutProviderInSessionCatalog( - String provider, boolean expectCreate) { - TestTable source = TestTables.MixedHive.PK_PT; - createHiveSource(source.hiveSchema, source.hivePartitions); - - sql("CREATE TABLE " + target() + " LIKE " + source() + " " + provider); - Assertions.assertEquals(expectCreate, tableExists()); - if (!expectCreate) { - // not an mixed-format table. - TestIdentifier target = target(); - CONTEXT.dropHiveTable(target.database, target.table); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestCreateTableSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestCreateTableSQL.java deleted file mode 100644 index 99af2f5d7e..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestCreateTableSQL.java +++ /dev/null @@ -1,312 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableMap; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.mixed.SparkSQLProperties; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.amoro.spark.test.utils.Asserts; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.Map; -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestCreateTableSQL extends MixedTableTestBase { - - public static Stream testTimestampHandleInCreateTable() { - return Stream.of( - Arguments.arguments(TableFormat.MIXED_HIVE, true, Types.TimestampType.withoutZone()), - Arguments.arguments(TableFormat.MIXED_HIVE, false, Types.TimestampType.withoutZone()), - Arguments.arguments(TableFormat.MIXED_ICEBERG, true, Types.TimestampType.withoutZone()), - Arguments.arguments(TableFormat.MIXED_ICEBERG, false, Types.TimestampType.withZone())); - } - - @DisplayName("Test `use-timestamp-without-zone-in-new-tables`") - @ParameterizedTest - @MethodSource - public void testTimestampHandleInCreateTable( - TableFormat format, boolean usingTimestampWithoutZone, Types.TimestampType expectType) { - spark() - .conf() - .set( - SparkSQLProperties.USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES, - usingTimestampWithoutZone); - - String sqlText = - "CREATE TABLE " - + target() - + "(\n" - + "id INT, \n" - + "ts TIMESTAMP \n) using " - + provider(format); - sql(sqlText); - MixedTable actual = loadTable(); - Type actualType = actual.schema().findField("ts").type(); - Assertions.assertEquals(expectType, actualType); - } - - public static Stream testPrimaryKeyNotNullConstraint() { - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, "INT", "", false), - Arguments.of(TableFormat.MIXED_HIVE, "INT NOT NULL", "", true), - Arguments.of(TableFormat.MIXED_HIVE, "INT", ", PRIMARY KEY(id)", true), - Arguments.of(TableFormat.MIXED_HIVE, "INT NOT NULL", ", PRIMARY KEY(id)", true), - Arguments.of(TableFormat.MIXED_ICEBERG, "INT", "", false), - Arguments.of(TableFormat.MIXED_ICEBERG, "INT NOT NULL", "", true), - Arguments.of(TableFormat.MIXED_ICEBERG, "INT", ", PRIMARY KEY(id)", true), - Arguments.of(TableFormat.MIXED_ICEBERG, "INT NOT NULL", ", PRIMARY KEY(id)", true)); - } - - @DisplayName("Test auto add `NOT NULL` for primary key") - @ParameterizedTest - @MethodSource - public void testPrimaryKeyNotNullConstraint( - TableFormat format, String idFieldTypeDDL, String primaryKeyDDL, boolean expectRequired) { - String sqlText = - "CREATE TABLE " - + target() - + "(\n" - + "id " - + idFieldTypeDDL - + ",\n" - + "DATA string " - + primaryKeyDDL - + "\n" - + ") using " - + provider(format); - - sql(sqlText); - Schema actualSchema = loadTable().schema(); - Types.NestedField idField = actualSchema.findField("id"); - Assertions.assertEquals(idField.isRequired(), expectRequired); - } - - public static Stream testPrimaryKeySpecExist() { - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", true), - Arguments.of(TableFormat.MIXED_HIVE, "", false), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", true), - Arguments.of(TableFormat.MIXED_ICEBERG, "", false)); - } - - @DisplayName("Test PRIMARY KEY spec exists.") - @ParameterizedTest - @MethodSource() - public void testPrimaryKeySpecExist( - TableFormat format, String primaryKeyDDL, boolean expectKeyedTable) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id int, data string " - + primaryKeyDDL - + " ) using " - + provider(format); - sql(sqlText); - MixedTable actualTable = loadTable(); - Assertions.assertEquals(actualTable.isKeyedTable(), expectKeyedTable); - if (expectKeyedTable) { - PrimaryKeySpec keySpec = actualTable.asKeyedTable().primaryKeySpec(); - Assertions.assertEquals(1, keySpec.fields().size()); - Assertions.assertTrue(keySpec.fieldNames().contains("id")); - } - } - - static Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.IntegerType.get()), - Types.NestedField.required(1, "data", Types.StringType.get()), - Types.NestedField.required(2, "ts", Types.TimestampType.withoutZone()), - Types.NestedField.required(3, "pt", Types.StringType.get())); - - public static Stream testPartitionSpec() { - - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, "", PartitionSpec.unpartitioned()), - Arguments.of( - TableFormat.MIXED_HIVE, - "PARTITIONED BY (pt)", - PartitionSpec.builderFor(schema).identity("pt").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PARTITIONED BY (years(ts))", - PartitionSpec.builderFor(schema).year("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PARTITIONED BY (months(ts))", - PartitionSpec.builderFor(schema).month("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PARTITIONED BY (days(ts))", - PartitionSpec.builderFor(schema).day("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PARTITIONED BY (date(ts))", - PartitionSpec.builderFor(schema).day("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PARTITIONED BY (hours(ts))", - PartitionSpec.builderFor(schema).hour("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PARTITIONED BY (date_hour(ts))", - PartitionSpec.builderFor(schema).hour("ts").build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PARTITIONED BY (bucket(4, id))", - PartitionSpec.builderFor(schema).bucket("id", 4).build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PARTITIONED BY (truncate(10, data))", - PartitionSpec.builderFor(schema).truncate("data", 10).build()), - Arguments.of( - TableFormat.MIXED_ICEBERG, - "PARTITIONED BY (truncate(10, id))", - PartitionSpec.builderFor(schema).truncate("id", 10).build())); - } - - @DisplayName("Test PartitionSpec is right") - @ParameterizedTest - @MethodSource() - public void testPartitionSpec(TableFormat format, String partitionDDL, PartitionSpec expectSpec) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id int, " - + "data string, " - + "ts timestamp, " - + "pt string, " - + " PRIMARY KEY(id) ) using " - + provider(format) - + " " - + partitionDDL; - - sql(sqlText); - - MixedTable actualTable = loadTable(); - Asserts.assertPartition(expectSpec, actualTable.spec()); - if (TableFormat.MIXED_HIVE.equals(format)) { - Table hiveTable = loadHiveTable(); - Asserts.assertHivePartition(expectSpec, hiveTable.getPartitionKeys()); - } - } - - public static Stream testSchemaAndProperties() { - String structDDL = - "id INT,\n" - + "data string NOT NULL,\n" - + "point struct,\n" - + "maps map,\n" - + "arrays array,\n" - + "pt string "; - Types.NestedField id = Types.NestedField.optional(1, "id", Types.IntegerType.get()); - Types.NestedField data = Types.NestedField.required(2, "data", Types.StringType.get()); - Types.NestedField point = - Types.NestedField.optional( - 3, - "point", - Types.StructType.of( - Types.NestedField.required(4, "x", Types.DoubleType.get()), - Types.NestedField.required(5, "y", Types.DoubleType.get()))); - Types.NestedField map = - Types.NestedField.optional( - 6, - "maps", - Types.MapType.ofOptional(7, 8, Types.StringType.get(), Types.StringType.get())); - Types.NestedField array = - Types.NestedField.optional( - 9, "arrays", Types.ListType.ofOptional(10, Types.StringType.get())); - Types.NestedField pt = Types.NestedField.optional(11, "pt", Types.StringType.get()); - - return Stream.of( - Arguments.of( - TableFormat.MIXED_ICEBERG, - structDDL, - "TBLPROPERTIES('key'='value1', 'catalog'='INTERNAL')", - new Schema(Lists.newArrayList(id, data, point, map, array, pt)), - ImmutableMap.of("key", "value1", "catalog", "INTERNAL")), - Arguments.of( - TableFormat.MIXED_ICEBERG, - structDDL + ", PRIMARY KEY(id)", - "TBLPROPERTIES('key'='value1', 'catalog'='INTERNAL')", - new Schema(Lists.newArrayList(id.asRequired(), data, point, map, array, pt)), - ImmutableMap.of("key", "value1", "catalog", "INTERNAL")), - Arguments.of( - TableFormat.MIXED_HIVE, - structDDL, - "tblproperties('key'='value1', 'catalog'='hive')", - new Schema(Lists.newArrayList(id, data, point, map, array, pt)), - ImmutableMap.of("key", "value1", "catalog", "hive")), - Arguments.of( - TableFormat.MIXED_HIVE, - structDDL + ", PRIMARY KEY(id)", - "tblproperties('key'='value1', 'catalog'='hive')", - new Schema(Lists.newArrayList(id.asRequired(), data, point, map, array, pt)), - ImmutableMap.of("key", "value1", "catalog", "hive"))); - } - - @DisplayName("Test primary key, schema and properties") - @ParameterizedTest - @MethodSource - public void testSchemaAndProperties( - TableFormat format, - String structDDL, - String propertiesDDL, - Schema expectSchema, - Map expectProperties) { - spark().conf().set(SparkSQLProperties.USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES, false); - String sqlText = - "CREATE TABLE " - + target() - + "(" - + structDDL - + ") using " - + provider(format) - + " " - + propertiesDDL; - sql(sqlText); - - MixedTable tbl = loadTable(); - - Asserts.assertType(expectSchema.asStruct(), tbl.schema().asStruct()); - Asserts.assertHashMapContainExpect(expectProperties, tbl.properties()); - if (TableFormat.MIXED_HIVE.equals(format)) { - Table hiveTable = loadHiveTable(); - Asserts.assertHiveColumns( - expectSchema, PartitionSpec.unpartitioned(), hiveTable.getSd().getCols()); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestDeleteFromSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestDeleteFromSQL.java deleted file mode 100644 index 546da93a64..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestDeleteFromSQL.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestDeleteFromSQL extends MixedTableTestBase { - public static Stream testDelete() { - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", " where id = 3"), - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", ""), - Arguments.of(TableFormat.MIXED_HIVE, "", ""), - Arguments.of(TableFormat.MIXED_HIVE, "", " where id = 3"), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", " where id = 3"), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", ""), - Arguments.of(TableFormat.MIXED_ICEBERG, "", ""), - Arguments.of(TableFormat.MIXED_ICEBERG, "", " where id = 3")); - } - - @DisplayName("Test `test delete`") - @ParameterizedTest - @MethodSource - public void testDelete(TableFormat format, String primaryKeyDDL, String filter) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id int, data string " - + primaryKeyDDL - + " ) using " - + provider(format); - sql(sqlText); - sql( - "insert into " - + target().database - + "." - + target().table - + " values (1, 'a'), (2, 'b'), (3, 'c')"); - sql("delete from " + target().database + "." + target().table + filter); - Dataset sql = sql("select * from " + target().database + "." + target().table); - if (filter.isEmpty()) { - Assertions.assertEquals(0, sql.collectAsList().size()); - } else { - Assertions.assertEquals(2, sql.collectAsList().size()); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestDescSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestDescSQL.java deleted file mode 100644 index 2ecf24aec8..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestDescSQL.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.spark.sql.Row; -import org.junit.jupiter.api.Assumptions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestDescSQL extends MixedTableTestBase { - - public static Stream testDescTable() { - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", ""), - Arguments.of(TableFormat.MIXED_HIVE, "", ""), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", ""), - Arguments.of(TableFormat.MIXED_ICEBERG, "", ""), - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", " PARTITIONED BY (day)"), - Arguments.of(TableFormat.MIXED_HIVE, "", " PARTITIONED BY (day)"), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", " PARTITIONED BY (day)"), - Arguments.of(TableFormat.MIXED_ICEBERG, "", " PARTITIONED BY (day)")); - } - - @DisplayName("Test `test describe table`") - @ParameterizedTest - @MethodSource - public void testDescTable(TableFormat format, String primaryKeyDDL, String partitionDDL) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id int, data string, day string " - + primaryKeyDDL - + " ) using " - + provider(format) - + partitionDDL; - sql(sqlText); - List rows = sql("desc " + target().database + "." + target().table).collectAsList(); - List primaryKeys = new ArrayList<>(); - List partitions = new ArrayList<>(); - if (!primaryKeyDDL.isEmpty()) { - primaryKeys.add("id"); - } - if (!partitionDDL.isEmpty()) { - partitions.add("day"); - } - assertTableDesc(rows, primaryKeys, partitions); - - List rows2 = - sql("desc extended " + target().database + "." + target().table).collectAsList(); - assertTableDesc(rows2, primaryKeys, partitions); - - Assumptions.assumeFalse(spark().version().startsWith("3.1")); - rows = sql("show create table " + target()).collectAsList(); - assertShowCreateTable(rows, target(), loadTable()); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestDropPartitionSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestDropPartitionSQL.java deleted file mode 100644 index a18f1a04c1..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestDropPartitionSQL.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.amoro.spark.test.utils.TableFiles; -import org.apache.amoro.spark.test.utils.TestTableUtil; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestDropPartitionSQL extends MixedTableTestBase { - - public static Stream testDropPartition() { - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)"), - Arguments.of(TableFormat.MIXED_HIVE, ""), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)"), - Arguments.of(TableFormat.MIXED_ICEBERG, "")); - } - - @DisplayName("Test `test drop partiton`") - @ParameterizedTest - @MethodSource - public void testDropPartition(TableFormat format, String primaryKeyDDL) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id int, data string, day string " - + primaryKeyDDL - + " ) using " - + provider(format) - + " PARTITIONED BY (day)"; - sql(sqlText); - sql( - "insert into " - + target().database - + "." - + target().table - + " values (1, 'a', 'a'), (2, 'b', 'b'), (3, 'c', 'c')"); - sql( - "alter table " - + target().database - + "." - + target().table - + " drop if exists partition (day='c')"); - Dataset sql = sql("select * from " + target().database + "." + target().table); - TableFiles files = TestTableUtil.files(loadTable()); - if (primaryKeyDDL.isEmpty()) { - Assertions.assertEquals(2, files.baseDataFiles.size()); - } else { - Assertions.assertEquals(2, files.changeInsertFiles.size()); - } - Assertions.assertEquals(2, sql.collectAsList().size()); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestInsertIntoSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestInsertIntoSQL.java deleted file mode 100644 index e4fbdae184..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestInsertIntoSQL.java +++ /dev/null @@ -1,273 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.data.ChangeAction; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.mixed.SparkSQLProperties; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.amoro.spark.test.utils.DataComparator; -import org.apache.amoro.spark.test.utils.ExpectResultUtil; -import org.apache.amoro.spark.test.utils.RecordGenerator; -import org.apache.amoro.spark.test.utils.TestTableUtil; -import org.apache.amoro.table.MetadataColumns; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.amoro.table.TableProperties; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.Comparator; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -/** - * test 1. upsert not enabled test 2. upsert enabled test 3. upsert source duplicate check test 4. - * upsert optimize write - */ -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestInsertIntoSQL extends MixedTableTestBase { - - static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get()), - Types.NestedField.required(3, "pt", Types.StringType.get())); - - static final PrimaryKeySpec ID_PRIMARY_KEY_SPEC = - PrimaryKeySpec.builderFor(SCHEMA).addColumn("id").build(); - - static final PartitionSpec PT_SPEC = PartitionSpec.builderFor(SCHEMA).identity("pt").build(); - - List base = - Lists.newArrayList( - RecordGenerator.newRecord(SCHEMA, 1, "aaa", "AAA"), - RecordGenerator.newRecord(SCHEMA, 2, "bbb", "AAA"), - RecordGenerator.newRecord(SCHEMA, 3, "ccc", "AAA"), - RecordGenerator.newRecord(SCHEMA, 4, "ddd", "AAA"), - RecordGenerator.newRecord(SCHEMA, 5, "eee", "BBB"), - RecordGenerator.newRecord(SCHEMA, 6, "fff", "BBB"), - RecordGenerator.newRecord(SCHEMA, 7, "ggg", "BBB"), - RecordGenerator.newRecord(SCHEMA, 8, "hhh", "BBB")); - List source = - Lists.newArrayList( - RecordGenerator.newRecord(SCHEMA, 1, "xxx", "AAA"), - RecordGenerator.newRecord(SCHEMA, 2, "xxx", "AAA"), - RecordGenerator.newRecord(SCHEMA, 7, "xxx", "BBB"), - RecordGenerator.newRecord(SCHEMA, 8, "xxx", "BBB"), - RecordGenerator.newRecord(SCHEMA, 9, "xxx", "CCC"), - RecordGenerator.newRecord(SCHEMA, 10, "xxx", "CCC")); - List duplicateSource = - Lists.newArrayList( - RecordGenerator.newRecord(SCHEMA, 1, "xxx", "AAA"), - RecordGenerator.newRecord(SCHEMA, 2, "xxx", "AAA"), - RecordGenerator.newRecord(SCHEMA, 2, "xxx", "BBB")); - - Comparator pkComparator = Comparator.comparing(r -> r.get(0, Integer.class)); - - Comparator dataComparator = Comparator.comparing(r -> r.get(1, String.class)); - - Comparator changeActionComparator = - Comparator.comparing(r -> (String) r.getField(MetadataColumns.CHANGE_ACTION_NAME)); - Comparator comparator = pkComparator.thenComparing(dataComparator); - - public static Stream testNoUpsert() { - return Stream.of( - Arguments.of(MIXED_HIVE, SCHEMA, ID_PRIMARY_KEY_SPEC, PT_SPEC), - Arguments.of(MIXED_HIVE, SCHEMA, NO_PRIMARY_KEY, PT_SPEC), - Arguments.of(MIXED_HIVE, SCHEMA, ID_PRIMARY_KEY_SPEC, UNPARTITIONED), - Arguments.of(MIXED_HIVE, SCHEMA, NO_PRIMARY_KEY, UNPARTITIONED), - Arguments.of(MIXED_ICEBERG, SCHEMA, ID_PRIMARY_KEY_SPEC, PT_SPEC), - Arguments.of(MIXED_ICEBERG, SCHEMA, NO_PRIMARY_KEY, PT_SPEC), - Arguments.of(MIXED_ICEBERG, SCHEMA, ID_PRIMARY_KEY_SPEC, UNPARTITIONED), - Arguments.of(MIXED_ICEBERG, SCHEMA, NO_PRIMARY_KEY, UNPARTITIONED)) - .flatMap( - e -> { - List parquet = Lists.newArrayList(e.get()); - parquet.add(FileFormat.PARQUET); - List orc = Lists.newArrayList(e.get()); - orc.add(FileFormat.ORC); - return Stream.of(Arguments.of(parquet.toArray()), Arguments.of(orc.toArray())); - }); - } - - @DisplayName("TestSQL: INSERT INTO table without upsert") - @ParameterizedTest - @MethodSource - public void testNoUpsert( - TableFormat format, - Schema schema, - PrimaryKeySpec keySpec, - PartitionSpec ptSpec, - FileFormat fileFormat) { - MixedTable table = - createTarget( - schema, - tableBuilder -> - tableBuilder - .withPrimaryKeySpec(keySpec) - .withProperty(TableProperties.UPSERT_ENABLED, "false") - .withProperty(TableProperties.CHANGE_FILE_FORMAT, fileFormat.name()) - .withProperty(TableProperties.BASE_FILE_FORMAT, fileFormat.name()) - .withPartitionSpec(ptSpec)); - - createViewSource(schema, source); - - TestTableUtil.writeToBase(table, base); - sql("INSERT INTO " + target() + " SELECT * FROM " + source()); - table.refresh(); - - // mor result - List results = TestTableUtil.tableRecords(table); - List expects = Lists.newArrayList(); - expects.addAll(base); - expects.addAll(source); - - DataComparator.build(expects, results).ignoreOrder(comparator).assertRecordsEqual(); - } - - public static Stream testUpsert() { - return Stream.of( - Arguments.of(MIXED_HIVE, SCHEMA, ID_PRIMARY_KEY_SPEC, PT_SPEC), - Arguments.of(MIXED_HIVE, SCHEMA, NO_PRIMARY_KEY, PT_SPEC), - Arguments.of(MIXED_HIVE, SCHEMA, ID_PRIMARY_KEY_SPEC, UNPARTITIONED), - Arguments.of(MIXED_HIVE, SCHEMA, NO_PRIMARY_KEY, UNPARTITIONED), - Arguments.of(MIXED_ICEBERG, SCHEMA, ID_PRIMARY_KEY_SPEC, PT_SPEC), - Arguments.of(MIXED_ICEBERG, SCHEMA, NO_PRIMARY_KEY, PT_SPEC), - Arguments.of(MIXED_ICEBERG, SCHEMA, ID_PRIMARY_KEY_SPEC, UNPARTITIONED), - Arguments.of(MIXED_ICEBERG, SCHEMA, NO_PRIMARY_KEY, UNPARTITIONED)) - .flatMap( - e -> { - List parquet = Lists.newArrayList(e.get()); - parquet.add(FileFormat.PARQUET); - List orc = Lists.newArrayList(e.get()); - orc.add(FileFormat.ORC); - return Stream.of(Arguments.of(parquet.toArray()), Arguments.of(orc.toArray())); - }); - } - - @DisplayName("TestSQL: INSERT INTO table with upsert enabled") - @ParameterizedTest - @MethodSource - public void testUpsert( - TableFormat format, - Schema schema, - PrimaryKeySpec keySpec, - PartitionSpec ptSpec, - FileFormat fileFormat) { - MixedTable table = - createTarget( - schema, - tableBuilder -> - tableBuilder - .withPrimaryKeySpec(keySpec) - .withProperty(TableProperties.UPSERT_ENABLED, "true") - .withProperty(TableProperties.CHANGE_FILE_FORMAT, fileFormat.name()) - .withProperty(TableProperties.BASE_FILE_FORMAT, fileFormat.name()) - .withPartitionSpec(ptSpec)); - createViewSource(schema, source); - - TestTableUtil.writeToBase(table, base); - sql("INSERT INTO " + target() + " SELECT * FROM " + source()); - - List expects; - if (keySpec.primaryKeyExisted()) { - expects = ExpectResultUtil.upsertResult(base, source, r -> r.get(0, Integer.class)); - } else { - expects = Lists.newArrayList(base); - expects.addAll(source); - } - - table.refresh(); - List results = TestTableUtil.tableRecords(table); - DataComparator.build(expects, results).ignoreOrder(comparator).assertRecordsEqual(); - - if (table.isKeyedTable()) { - List deletes = - ExpectResultUtil.upsertDeletes(base, source, r -> r.get(0, Integer.class)); - - List expectChanges = - deletes.stream() - .map( - r -> - TestTableUtil.extendMetadataValue( - r, MetadataColumns.CHANGE_ACTION_FIELD, ChangeAction.DELETE.name())) - .collect(Collectors.toList()); - - source.stream() - .map( - r -> - TestTableUtil.extendMetadataValue( - r, MetadataColumns.CHANGE_ACTION_FIELD, ChangeAction.INSERT.name())) - .forEach(expectChanges::add); - - List changes = TestTableUtil.changeRecordsWithAction(table.asKeyedTable()); - - DataComparator.build(expectChanges, changes) - .ignoreOrder(pkComparator.thenComparing(changeActionComparator)) - .assertRecordsEqual(); - } - } - - public static Stream testDuplicateSourceCheck() { - return Stream.of( - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, true, true), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY, true, false), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, false, false), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, true, true), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, true, false), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, false, false)); - } - - @DisplayName("TestSQL: INSERT INTO duplicate source check") - @ParameterizedTest(name = "{index} {0} {1} source-is-duplicate: {2} expect-exception: {3}") - @MethodSource - public void testDuplicateSourceCheck( - TableFormat format, - PrimaryKeySpec keySpec, - boolean duplicateSource, - boolean expectException) { - spark().conf().set(SparkSQLProperties.CHECK_SOURCE_DUPLICATES_ENABLE, "true"); - createTarget(SCHEMA, tableBuilder -> tableBuilder.withPrimaryKeySpec(keySpec)); - - List source = duplicateSource ? this.duplicateSource : this.source; - createViewSource(SCHEMA, source); - - boolean getException = false; - - try { - sql("INSERT INTO " + target() + " SELECT * FROM " + source()); - } catch (Exception e) { - getException = true; - } - Assertions.assertEquals(expectException, getException, "expect exception assert failed."); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestInsertOverwriteSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestInsertOverwriteSQL.java deleted file mode 100644 index 613dd45c22..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestInsertOverwriteSQL.java +++ /dev/null @@ -1,451 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.data.ChangeAction; -import org.apache.amoro.hive.table.SupportHive; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.mixed.SparkSQLProperties; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.amoro.spark.test.utils.Asserts; -import org.apache.amoro.spark.test.utils.DataComparator; -import org.apache.amoro.spark.test.utils.ExpectResultUtil; -import org.apache.amoro.spark.test.utils.RecordGenerator; -import org.apache.amoro.spark.test.utils.TableFiles; -import org.apache.amoro.spark.test.utils.TestTableUtil; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.amoro.table.TableProperties; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.time.LocalDateTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.List; -import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.Stream; - -/** - * 1. dynamic insert overwrite 2. static insert overwrite 3. for un-partitioned table 3. duplicate - * check for insert overwrite 4. optimize write is work for insert overwrite - */ -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestInsertOverwriteSQL extends MixedTableTestBase { - - static final String OVERWRITE_MODE_KEY = "spark.sql.sources.partitionOverwriteMode"; - static final String DYNAMIC = "DYNAMIC"; - static final String STATIC = "STATIC"; - - static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get()), - Types.NestedField.required(3, "pt", Types.StringType.get())); - - static final PrimaryKeySpec ID_PRIMARY_KEY_SPEC = - PrimaryKeySpec.builderFor(SCHEMA).addColumn("id").build(); - - static final PartitionSpec PT_SPEC = PartitionSpec.builderFor(SCHEMA).identity("pt").build(); - - List base = - Lists.newArrayList( - RecordGenerator.newRecord(SCHEMA, 1, "aaa", "AAA"), - RecordGenerator.newRecord(SCHEMA, 2, "bbb", "AAA"), - RecordGenerator.newRecord(SCHEMA, 3, "ccc", "BBB"), - RecordGenerator.newRecord(SCHEMA, 4, "ddd", "BBB"), - RecordGenerator.newRecord(SCHEMA, 5, "eee", "CCC"), - RecordGenerator.newRecord(SCHEMA, 6, "fff", "CCC")); - - List change = - Lists.newArrayList( - RecordGenerator.newRecord(SCHEMA, 7, "ggg", "DDD"), - RecordGenerator.newRecord(SCHEMA, 8, "hhh", "DDD"), - RecordGenerator.newRecord(SCHEMA, 9, "jjj", "AAA"), - RecordGenerator.newRecord(SCHEMA, 10, "kkk", "AAA")); - - List source = - Lists.newArrayList( - RecordGenerator.newRecord(SCHEMA, 1, "xxx", "AAA"), - RecordGenerator.newRecord(SCHEMA, 2, "xxx", "AAA"), - RecordGenerator.newRecord(SCHEMA, 11, "xxx", "DDD"), - RecordGenerator.newRecord(SCHEMA, 12, "xxx", "DDD"), - RecordGenerator.newRecord(SCHEMA, 13, "xxx", "EEE"), - RecordGenerator.newRecord(SCHEMA, 14, "xxx", "EEE")); - - private MixedTable table; - private List target; - private List initFiles; - - private void initTargetTable(PrimaryKeySpec keySpec, PartitionSpec ptSpec) { - table = - createTarget( - SCHEMA, builder -> builder.withPartitionSpec(ptSpec).withPrimaryKeySpec(keySpec)); - initFiles = TestTableUtil.writeToBase(table, base); - target = Lists.newArrayList(base); - - if (keySpec.primaryKeyExisted()) { - List changeFiles = - TestTableUtil.writeToChange(table.asKeyedTable(), change, ChangeAction.INSERT); - initFiles.addAll(changeFiles); - target.addAll(change); - } - - createViewSource(SCHEMA, source); - } - - private void assertFileLayout(TableFormat format) { - TableFiles files = TestTableUtil.files(table); - Set initFileSet = - initFiles.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); - files = files.removeFiles(initFileSet); - - Asserts.assertAllFilesInBaseStore(files); - if (MIXED_HIVE == format) { - String hiveLocation = ((SupportHive) table).hiveLocation(); - Asserts.assertAllFilesInHiveLocation(files, hiveLocation); - } - } - - @BeforeEach - void cleanVars() { - this.table = null; - this.target = Lists.newArrayList(); - this.initFiles = Lists.newArrayList(); - } - - public static Stream testDynamic() { - return Stream.of( - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY)); - } - - @DisplayName("TestSQL: INSERT OVERWRITE dynamic mode") - @ParameterizedTest() - @MethodSource - public void testDynamic(TableFormat format, PrimaryKeySpec keySpec) { - spark().conf().set(OVERWRITE_MODE_KEY, DYNAMIC); - - initTargetTable(keySpec, PT_SPEC); - - sql("INSERT OVERWRITE " + target() + " SELECT * FROM " + source()); - - table.refresh(); - List expects = - ExpectResultUtil.dynamicOverwriteResult(target, source, r -> r.getField("pt")); - List actual = TestTableUtil.tableRecords(table); - DataComparator.build(expects, actual).ignoreOrder("id").assertRecordsEqual(); - - assertFileLayout(format); - } - - private static Record setPtValue(Record r, String value) { - Record record = r.copy(); - record.setField("pt", value); - return record; - } - - public static Stream testStatic() { - Function alwaysTrue = r -> true; - Function deleteAAA = r -> "AAA".equals(r.getField("pt")); - Function deleteDDD = r -> "DDD".equals(r.getField("pt")); - - Function noTrans = Function.identity(); - Function ptAAA = r -> setPtValue(r, "AAA"); - Function ptDDD = r -> setPtValue(r, "DDD"); - - return Stream.of( - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, "", "*", alwaysTrue, noTrans), - Arguments.arguments( - MIXED_ICEBERG, - ID_PRIMARY_KEY_SPEC, - "PARTITION(pt = 'AAA')", - "id, data", - deleteAAA, - ptAAA), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, "", "*", alwaysTrue, noTrans), - Arguments.arguments( - MIXED_ICEBERG, NO_PRIMARY_KEY, "PARTITION(pt = 'DDD')", "id, data", deleteDDD, ptDDD), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, "", "*", alwaysTrue, noTrans), - Arguments.arguments( - MIXED_HIVE, ID_PRIMARY_KEY_SPEC, "PARTITION(pt = 'AAA')", "id, data", deleteAAA, ptAAA), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY, "", "*", alwaysTrue, noTrans), - Arguments.arguments( - MIXED_HIVE, NO_PRIMARY_KEY, "PARTITION(pt = 'DDD')", "id, data", deleteDDD, ptDDD)); - } - - @DisplayName("TestSQL: INSERT OVERWRITE static mode") - @ParameterizedTest(name = "{index} {0} {1} {2} SELECT {3}") - @MethodSource - public void testStatic( - TableFormat format, - PrimaryKeySpec keySpec, - String ptFilter, - String sourceProject, - Function deleteFilter, - Function sourceTrans) { - spark().conf().set(OVERWRITE_MODE_KEY, STATIC); - initTargetTable(keySpec, PT_SPEC); - - sql( - "INSERT OVERWRITE " - + target() - + " " - + ptFilter - + " SELECT " - + sourceProject - + " FROM " - + source()); - table.refresh(); - - List expects = - target.stream().filter(r -> !deleteFilter.apply(r)).collect(Collectors.toList()); - - source.stream().map(sourceTrans).forEach(expects::add); - - List actual = TestTableUtil.tableRecords(table); - DataComparator.build(expects, actual).ignoreOrder("pt", "id").assertRecordsEqual(); - - assertFileLayout(format); - } - - public static Stream testUnPartitioned() { - - return Stream.of( - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, DYNAMIC), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, STATIC), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, DYNAMIC), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, DYNAMIC), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, DYNAMIC), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, STATIC), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY, DYNAMIC), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY, DYNAMIC)); - } - - @DisplayName("TestSQL: INSERT OVERWRITE un-partitioned") - @ParameterizedTest(name = "{index} {0} {1} partitionOverwriteMode={2}") - @MethodSource - public void testUnPartitioned(TableFormat format, PrimaryKeySpec keySpec, String mode) { - spark().conf().set(OVERWRITE_MODE_KEY, mode); - initTargetTable(keySpec, PartitionSpec.unpartitioned()); - - sql("INSERT OVERWRITE " + target() + " SELECT * FROM " + source()); - - table.refresh(); - List expects = Lists.newArrayList(source); - List actual = TestTableUtil.tableRecords(table); - DataComparator.build(expects, actual).ignoreOrder("pt", "id").assertRecordsEqual(); - - assertFileLayout(format); - } - - private static final Schema hiddenPartitionSchema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "ts", Types.TimestampType.withZone())); - private static final OffsetDateTime EPOCH = - LocalDateTime.of(2000, 1, 1, 0, 0, 0).atOffset(ZoneOffset.UTC); - private static final List hiddenPartitionSource = - IntStream.range(0, 10) - .boxed() - .map(i -> RecordGenerator.newRecord(hiddenPartitionSchema, i, EPOCH.plusDays(i))) - .collect(Collectors.toList()); - - private static PartitionSpec.Builder ptBuilder() { - return PartitionSpec.builderFor(hiddenPartitionSchema); - } - - public static Stream testHiddenPartitions() { - return Stream.of( - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, ptBuilder().year("ts").build()), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, ptBuilder().month("ts").build()), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, ptBuilder().day("ts").build()), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, ptBuilder().hour("ts").build()), - Arguments.arguments( - MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, ptBuilder().bucket("id", 8).build()), - Arguments.arguments( - MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, ptBuilder().truncate("id", 10).build()), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, ptBuilder().year("ts").build()), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, ptBuilder().month("ts").build()), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, ptBuilder().day("ts").build()), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, ptBuilder().hour("ts").build()), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, ptBuilder().bucket("id", 8).build()), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, ptBuilder().truncate("id", 10).build())); - } - - @DisplayName("TestSQL: INSERT OVERWRITE hidden partition optimize write") - @ParameterizedTest() - @MethodSource - public void testHiddenPartitions( - TableFormat format, PrimaryKeySpec keySpec, PartitionSpec ptSpec) { - spark().conf().set(OVERWRITE_MODE_KEY, DYNAMIC); - spark().conf().set(SparkSQLProperties.OPTIMIZE_WRITE_ENABLED, "true"); - - this.table = - createTarget( - hiddenPartitionSchema, - builder -> builder.withPrimaryKeySpec(keySpec).withPartitionSpec(ptSpec)); - createViewSource(hiddenPartitionSchema, hiddenPartitionSource); - this.initFiles = Lists.newArrayList(); - - sql("INSERT OVERWRITE " + target() + " SELECT * FROM " + source()); - - table.refresh(); - assertFileLayout(format); - } - - public static Stream testOptimizeWrite() { - return Stream.of( - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, PT_SPEC, STATIC, 4, true), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, UNPARTITIONED, STATIC, 4, true), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, PT_SPEC, STATIC, 4, true), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, UNPARTITIONED, STATIC, 4, true), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, PT_SPEC, STATIC, 1, true), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, PT_SPEC, STATIC, 4, false), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, PT_SPEC, DYNAMIC, 4, true), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, UNPARTITIONED, DYNAMIC, 4, true), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, PT_SPEC, DYNAMIC, 4, true), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, UNPARTITIONED, DYNAMIC, 4, true), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, PT_SPEC, DYNAMIC, 1, true), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, PT_SPEC, DYNAMIC, 4, false), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, PT_SPEC, STATIC, 4, true), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, UNPARTITIONED, STATIC, 4, true), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY, PT_SPEC, STATIC, 4, true), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY, UNPARTITIONED, STATIC, 4, true), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, PT_SPEC, STATIC, 1, true), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, PT_SPEC, STATIC, 4, false), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, PT_SPEC, DYNAMIC, 4, true), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, UNPARTITIONED, DYNAMIC, 4, true), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY, PT_SPEC, DYNAMIC, 4, true), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY, UNPARTITIONED, DYNAMIC, 4, true), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, PT_SPEC, DYNAMIC, 1, true), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, PT_SPEC, DYNAMIC, 4, false)); - } - - @DisplayName("TestSQL: INSERT OVERWRITE optimize write works") - @ParameterizedTest() - @MethodSource - public void testOptimizeWrite( - TableFormat format, - PrimaryKeySpec keySpec, - PartitionSpec ptSpec, - String mode, - int bucket, - boolean optimizeWriteEnable) { - spark().conf().set(SparkSQLProperties.OPTIMIZE_WRITE_ENABLED, optimizeWriteEnable); - spark().conf().set(OVERWRITE_MODE_KEY, mode); - - this.table = - createTarget( - SCHEMA, - builder -> - builder - .withPrimaryKeySpec(keySpec) - .withProperty( - TableProperties.BASE_FILE_INDEX_HASH_BUCKET, String.valueOf(bucket)) - .withPartitionSpec(ptSpec)); - - String[] ptValues = {"AAA", "BBB", "CCC", "DDD"}; - List source = - IntStream.range(1, 100) - .boxed() - .map( - i -> - RecordGenerator.newRecord( - SCHEMA, i, "index" + i, ptValues[i % ptValues.length])) - .collect(Collectors.toList()); - createViewSource(SCHEMA, source); - - sql("INSERT OVERWRITE " + target() + " SELECT * FROM " + source()); - - boolean shouldOptimized = - optimizeWriteEnable && (keySpec.primaryKeyExisted() || ptSpec.isPartitioned()); - - if (shouldOptimized) { - table.refresh(); - TableFiles files = TestTableUtil.files(table); - int expectFiles = ExpectResultUtil.expectOptimizeWriteFileCount(source, table, bucket); - - Assertions.assertEquals(expectFiles, files.baseDataFiles.size()); - } - } - - public static Arguments[] testSourceDuplicateCheck() { - return new Arguments[] { - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, PT_SPEC, STATIC, true, true), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, PT_SPEC, STATIC, true, false), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, PT_SPEC, DYNAMIC, true, true), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY, PT_SPEC, DYNAMIC, true, false), - Arguments.arguments(MIXED_ICEBERG, ID_PRIMARY_KEY_SPEC, PT_SPEC, DYNAMIC, false, false), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, PT_SPEC, STATIC, true, true), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY, PT_SPEC, STATIC, true, false), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, PT_SPEC, DYNAMIC, true, true), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY, PT_SPEC, DYNAMIC, true, false), - Arguments.arguments(MIXED_HIVE, ID_PRIMARY_KEY_SPEC, PT_SPEC, DYNAMIC, false, false), - }; - } - - @DisplayName("TestSQL: INSERT OVERWRITE duplicate check source") - @ParameterizedTest() - @MethodSource - public void testSourceDuplicateCheck( - TableFormat format, - PrimaryKeySpec keySpec, - PartitionSpec ptSpec, - String mode, - boolean duplicateSource, - boolean expectChecked) { - spark().conf().set(OVERWRITE_MODE_KEY, mode); - spark().conf().set(SparkSQLProperties.CHECK_SOURCE_DUPLICATES_ENABLE, true); - - table = - createTarget( - SCHEMA, builder -> builder.withPartitionSpec(ptSpec).withPrimaryKeySpec(keySpec)); - List sourceData = Lists.newArrayList(this.base); - if (duplicateSource) { - sourceData.addAll(this.source); - } - createViewSource(SCHEMA, sourceData); - - boolean failed = false; - try { - sql("INSERT OVERWRITE " + target() + " SELECT * FROM " + source()); - } catch (Exception e) { - failed = true; - } - Assertions.assertEquals(expectChecked, failed); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestMergeIntoSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestMergeIntoSQL.java deleted file mode 100644 index 84ea61ba86..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestMergeIntoSQL.java +++ /dev/null @@ -1,325 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.data.ChangeAction; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.amoro.spark.test.utils.DataComparator; -import org.apache.amoro.spark.test.utils.ExpectResultUtil; -import org.apache.amoro.spark.test.utils.RecordGenerator; -import org.apache.amoro.spark.test.utils.TestTableUtil; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.List; -import java.util.function.Function; -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestMergeIntoSQL extends MixedTableTestBase { - - private static final Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get()), - Types.NestedField.required(3, "fdata", Types.FloatType.get()), - Types.NestedField.required(4, "ddata", Types.DoubleType.get()), - Types.NestedField.required(5, "pt", Types.StringType.get())); - - private static final PrimaryKeySpec pk = - PrimaryKeySpec.builderFor(schema).addColumn("id").build(); - - private static final List base = - Lists.newArrayList( - RecordGenerator.newRecord(schema, 1, "a", 1.1f, 1.1D, "001"), - RecordGenerator.newRecord(schema, 2, "b", 1.1f, 1.1D, "002")); - private static final List change = - Lists.newArrayList( - RecordGenerator.newRecord(schema, 3, "c", 1.1f, 1.1D, "001"), - RecordGenerator.newRecord(schema, 4, "d", 1.1f, 1.1D, "002")); - - private static final List source = - Lists.newArrayList( - RecordGenerator.newRecord(schema, 1, "s1", 1.1f, 1.1D, "001"), - RecordGenerator.newRecord(schema, 2, "s2", 1.1f, 1.1D, "002"), - RecordGenerator.newRecord(schema, 5, "s5", 1.1f, 1.1D, "001"), - RecordGenerator.newRecord(schema, 6, "s6", 1.1f, 1.1D, "003")); - - private final List target = Lists.newArrayList(); - - public void setupTest(PrimaryKeySpec keySpec) { - MixedTable table = createTarget(schema, builder -> builder.withPrimaryKeySpec(keySpec)); - target.addAll(base); - target.addAll(change); - - if (table.isKeyedTable()) { - TestTableUtil.writeToBase(table, base); - TestTableUtil.writeToChange(table.asKeyedTable(), change, ChangeAction.INSERT); - } else { - TestTableUtil.writeToBase(table, target); - } - } - - public static Stream args() { - return Stream.of( - Arguments.arguments(MIXED_ICEBERG, pk), - Arguments.arguments(MIXED_ICEBERG, NO_PRIMARY_KEY), - Arguments.arguments(MIXED_HIVE, pk), - Arguments.arguments(MIXED_HIVE, NO_PRIMARY_KEY)); - } - - @DisplayName("SQL: MERGE INTO for all actions with condition") - @ParameterizedTest - @MethodSource("args") - public void testAllAction(TableFormat format, PrimaryKeySpec keySpec) { - setupTest(keySpec); - createViewSource(schema, source); - - sql( - "MERGE INTO " - + target() - + " AS t USING " - + source() - + " AS s ON t.id == s.id " - + "WHEN MATCHED AND t.id = 1 THEN DELETE " - + "WHEN MATCHED AND t.id = 2 THEN UPDATE SET * " - + "WHEN NOT MATCHED AND s.id != 5 THEN INSERT (t.data, t.pt, t.id, t.fdata,t.ddata) values ( s.data, s.pt, 1000,1.1,1.1)"); - - List expects = - ExpectResultUtil.expectMergeResult(target, source, r -> r.getField("id")) - .whenMatched((t, s) -> t.getField("id").equals(1), (t, s) -> null) - .whenMatched((t, s) -> t.getField("id").equals(2), (t, s) -> s) - .whenNotMatched( - s -> !s.getField("id").equals(5), - s -> { - s.setField("id", 1000); - s.setField("fdata", 1.1f); - s.setField("ddata", 1.1D); - return s; - }) - .results(); - - MixedTable table = loadTable(); - List actual = TestTableUtil.tableRecords(table); - DataComparator.build(expects, actual).ignoreOrder("id").assertRecordsEqual(); - } - - @DisplayName("SQL: MERGE INTO for all actions with condition") - @ParameterizedTest - @MethodSource("args") - public void testSetExactValue(TableFormat format, PrimaryKeySpec keySpec) { - setupTest(keySpec); - createViewSource(schema, source); - - sql( - "MERGE INTO " - + target() - + " AS t USING " - + source() - + " AS s ON t.id == s.id " - + "WHEN MATCHED AND t.id = 2 THEN UPDATE SET t.data = 'ccc', t.fdata = 1.1, t.ddata = 1.1"); - - List expects = - ExpectResultUtil.expectMergeResult(target, source, r -> r.getField("id")) - .whenMatched( - (t, s) -> t.getField("id").equals(2), - (t, s) -> { - t.setField("data", "ccc"); - return t; - }) - .results(); - - MixedTable table = loadTable(); - List actual = TestTableUtil.tableRecords(table); - DataComparator.build(expects, actual).ignoreOrder("id").assertRecordsEqual(); - } - - @DisplayName("SQL: MERGE INTO for all actions with target no data") - @ParameterizedTest - @MethodSource("args") - public void testEmptyTarget(TableFormat format, PrimaryKeySpec keySpec) { - MixedTable table = createTarget(schema, builder -> builder.withPrimaryKeySpec(keySpec)); - createViewSource(schema, source); - - sql( - "MERGE INTO " - + target() - + " AS t USING " - + source() - + " AS s ON t.id == s.id " - + "WHEN MATCHED AND t.id = 1 THEN DELETE " - + "WHEN MATCHED AND t.id = 2 THEN UPDATE SET * " - + "WHEN NOT MATCHED THEN INSERT *"); - - table.refresh(); - List expects = Lists.newArrayList(source); - List actual = TestTableUtil.tableRecords(table); - DataComparator.build(expects, actual).ignoreOrder("id").assertRecordsEqual(); - } - - @DisplayName("SQL: MERGE INTO for all actions without condition") - @ParameterizedTest - @MethodSource("args") - public void testActionWithoutCondition(TableFormat format, PrimaryKeySpec keySpec) { - setupTest(keySpec); - createViewSource(schema, source); - - sql( - "MERGE INTO " - + target() - + " AS t USING " - + source() - + " AS s ON t.id == s.id " - + "WHEN MATCHED THEN UPDATE SET * " - + "WHEN NOT MATCHED THEN INSERT *"); - - List expects = - ExpectResultUtil.expectMergeResult(target, source, r -> r.getField("id")) - .whenMatched((t, s) -> true, (t, s) -> s) - .whenNotMatched(s -> true, Function.identity()) - .results(); - - MixedTable table = loadTable(); - List actual = TestTableUtil.tableRecords(table); - DataComparator.build(expects, actual).ignoreOrder("id").assertRecordsEqual(); - } - - @DisplayName("SQL: MERGE INTO for only delete actions") - @ParameterizedTest - @MethodSource("args") - public void testOnlyDeletes(TableFormat format, PrimaryKeySpec keySpec) { - setupTest(keySpec); - createViewSource(schema, source); - sql( - "MERGE INTO " - + target() - + " AS t USING " - + source() - + " AS s ON t.id == s.id " - + "WHEN MATCHED THEN DELETE "); - - List expects = - ExpectResultUtil.expectMergeResult(target, source, r -> r.getField("id")) - .whenMatched((t, s) -> true, (t, s) -> null) - .results(); - - MixedTable table = loadTable(); - List actual = TestTableUtil.tableRecords(table); - DataComparator.build(expects, actual).ignoreOrder("id").assertRecordsEqual(); - } - - @DisplayName("SQL: MERGE INTO for explicit column ") - @ParameterizedTest - @MethodSource("args") - public void testExplicitColumn(TableFormat format, PrimaryKeySpec keySpec) { - setupTest(keySpec); - createViewSource(schema, source); - - sql( - "MERGE INTO " - + target() - + " AS t USING " - + source() - + " AS s ON t.id == s.id " - + "WHEN MATCHED THEN UPDATE SET t.id = s.id, t.data = s.pt, t.pt = s.pt " - + "WHEN NOT MATCHED THEN INSERT (t.data, t.pt, t.id, t.fdata,t.ddata) values ( s.pt, s.pt, s.id,s.fdata,s.ddata) "); - - Function dataAsPt = - s -> { - Record r = s.copy(); - r.setField("data", s.getField("pt")); - return r; - }; - List expects = - ExpectResultUtil.expectMergeResult(target, source, r -> r.getField("id")) - .whenMatched((t, s) -> true, (t, s) -> dataAsPt.apply(s)) - .whenNotMatched(s -> true, dataAsPt) - .results(); - - MixedTable table = loadTable(); - List actual = TestTableUtil.tableRecords(table); - DataComparator.build(expects, actual).ignoreOrder("id").assertRecordsEqual(); - } - - public static Stream formatArgs() { - return Stream.of(MIXED_HIVE, MIXED_ICEBERG); - } - - @DisplayName("SQL: MERGE INTO failed if join on non primary key") - @ParameterizedTest - @MethodSource("formatArgs") - public void testFailedForNonPrimaryKeyMerge(TableFormat format) { - setupTest(pk); - createViewSource(schema, source); - - boolean catched = false; - try { - sql( - "MERGE INTO " - + target() - + " AS t USING " - + source() - + " AS s ON t.pt == s.id " - + "WHEN MATCHED THEN UPDATE SET t.id = s.id, t.data = s.pt, t.pt = s.pt " - + "WHEN NOT MATCHED THEN INSERT (t.data, t.pt, t.id,t.fdata,t.ddata) values ( s.pt, s.data, s.id,s.fdata,s.ddata) "); - } catch (Exception e) { - catched = true; - } - Assertions.assertTrue(catched); - } - - @DisplayName("SQL: MERGE INTO failed if source has duplicate join key") - @ParameterizedTest - @MethodSource("formatArgs") - public void testFailedWhenDuplicateJoinKey(TableFormat format) { - setupTest(pk); - List source = - Lists.newArrayList( - RecordGenerator.newRecord(schema, 1, "s1", 1.1f, 2.2D, "001"), - RecordGenerator.newRecord(schema, 1, "s2", 1.1f, 2.2D, "001")); - createViewSource(schema, source); - - boolean catched = false; - try { - sql( - "MERGE INTO " - + target() - + " AS t USING " - + source() - + " AS s ON t.id == s.id " - + "WHEN MATCHED THEN UPDATE SET t.id = s.id, t.data = s.pt, t.pt = s.pt " - + "WHEN NOT MATCHED THEN INSERT (t.data, t.pt, t.id, t.fdata, t.ddata) values ( s.data, s.pt, s.id, s.fdata, s.ddata) "); - } catch (Exception e) { - catched = true; - } - Assertions.assertTrue(catched); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestMixedFormatSessionCatalog.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestMixedFormatSessionCatalog.java deleted file mode 100644 index 546030b3c2..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestMixedFormatSessionCatalog.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.properties.HiveTableProperties; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; -import org.apache.amoro.spark.mixed.SparkSQLProperties; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.utils.RecordGenerator; -import org.apache.amoro.spark.test.utils.TestTableUtil; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.thrift.TException; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; -import org.junit.platform.commons.util.StringUtils; - -import java.util.List; -import java.util.Map; -import java.util.stream.Stream; - -public class TestMixedFormatSessionCatalog extends MixedTableTestBase { - - Dataset rs; - - @Override - public Dataset sql(String sqlText) { - rs = super.sql(sqlText); - return rs; - } - - public static Stream testCreateTable() { - return Stream.of( - Arguments.arguments("arctic", true, ""), - Arguments.arguments("arctic", false, "pt"), - Arguments.arguments("arctic", true, "pt"), - Arguments.arguments("parquet", false, "pt"), - Arguments.arguments("parquet", false, "dt string")); - } - - @ParameterizedTest(name = "{index} USING {0} WITH PK {1} PARTITIONED BY ({2})") - @MethodSource - public void testCreateTable(String provider, boolean pk, String pt) { - - String sqlText = "CREATE TABLE " + target() + "(" + " id INT, data string, pt string "; - if (pk) { - sqlText += ", PRIMARY KEY(id)"; - } - sqlText += ") USING " + provider; - - if (StringUtils.isNotBlank(pt)) { - sqlText += " PARTITIONED BY (" + pt + ")"; - } - - sql(sqlText); - - if ("arctic".equalsIgnoreCase(provider)) { - Assertions.assertTrue(tableExists()); - } - - Table hiveTable = loadHiveTable(); - Assertions.assertNotNull(hiveTable); - } - - static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get()), - Types.NestedField.required(3, "pt", Types.StringType.get())); - - List source = - Lists.newArrayList( - RecordGenerator.newRecord(SCHEMA, 1, "111", "AAA"), - RecordGenerator.newRecord(SCHEMA, 2, "222", "AAA"), - RecordGenerator.newRecord(SCHEMA, 3, "333", "DDD"), - RecordGenerator.newRecord(SCHEMA, 4, "444", "DDD"), - RecordGenerator.newRecord(SCHEMA, 5, "555", "EEE"), - RecordGenerator.newRecord(SCHEMA, 6, "666", "EEE")); - - public static Stream testCreateTableAsSelect() { - return Stream.of( - Arguments.arguments("arctic", true, "", true), - Arguments.arguments("arctic", false, "pt", true), - Arguments.arguments("arctic", true, "pt", false), - Arguments.arguments("parquet", false, "pt", false), - Arguments.arguments("parquet", false, "", false)); - } - - @ParameterizedTest(name = "{index} USING {0} WITH PK {1} PARTITIONED BY ({2}) check: {3}") - @MethodSource - public void testCreateTableAsSelect( - String provider, boolean pk, String pt, boolean duplicateCheck) { - spark().conf().set(SparkSQLProperties.CHECK_SOURCE_DUPLICATES_ENABLE, duplicateCheck); - createViewSource(SCHEMA, source); - String sqlText = "CREATE TABLE " + target(); - if (pk) { - sqlText += " PRIMARY KEY (id, pt) "; - } - sqlText += " USING " + provider + " "; - if (StringUtils.isNotBlank(pt)) { - sqlText += " PARTITIONED BY (" + pt + ")"; - } - sqlText += " AS SELECT * FROM " + source(); - - sql(sqlText); - if ("arctic".equalsIgnoreCase(provider)) { - Assertions.assertTrue(tableExists()); - - if (duplicateCheck && pk) { - // LogicalPlan query = Asserts.lookupPlan(qe.optimizedPlan(), - // QueryWithConstraintCheckPlan.class); - // TODO: duplicate query check - // Assertions.assertNotNull(query); - } - } - - Table hiveTable = loadHiveTable(); - Assertions.assertNotNull(hiveTable); - } - - @Test - public void testLoadLegacyTable() { - createTarget( - SCHEMA, - c -> c.withPrimaryKeySpec(PrimaryKeySpec.builderFor(SCHEMA).addColumn("id").build())); - createViewSource(SCHEMA, source); - Table hiveTable = loadHiveTable(); - Map properties = Maps.newHashMap(hiveTable.getParameters()); - properties.remove(HiveTableProperties.MIXED_TABLE_FLAG); - properties.put(HiveTableProperties.AMORO_TABLE_FLAG_LEGACY, "true"); - hiveTable.setParameters(properties); - try { - CONTEXT - .getHiveClient() - .alter_table(hiveTable.getDbName(), hiveTable.getTableName(), hiveTable); - } catch (TException e) { - throw new RuntimeException(e); - } - - sql("insert into " + target() + " select * from " + source()); - MixedTable table = loadTable(); - List changes = TestTableUtil.changeRecordsWithAction(table.asKeyedTable()); - Assertions.assertTrue(changes.size() > 0); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestSelectSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestSelectSQL.java deleted file mode 100644 index a788d77754..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestSelectSQL.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.data.ChangeAction; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.amoro.spark.test.utils.DataComparator; -import org.apache.amoro.spark.test.utils.RecordGenerator; -import org.apache.amoro.spark.test.utils.TestTable; -import org.apache.amoro.spark.test.utils.TestTableUtil; -import org.apache.amoro.spark.test.utils.TestTables; -import org.apache.amoro.table.KeyedTable; -import org.apache.amoro.table.MetadataColumns; -import org.apache.amoro.table.TableProperties; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.Comparator; -import java.util.LinkedList; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestSelectSQL extends MixedTableTestBase { - - public static Stream testKeyedTableQuery() { - List tests = - Lists.newArrayList( - TestTables.MixedIceberg.PK_PT, - TestTables.MixedIceberg.PK_NO_PT, - TestTables.MixedHive.PK_PT, - TestTables.MixedHive.PK_NO_PT); - return tests.stream() - .map(t -> Arguments.of(t.format, t)) - .flatMap( - e -> { - List parquet = Lists.newArrayList(e.get()); - parquet.add(FileFormat.PARQUET); - List orc = Lists.newArrayList(e.get()); - orc.add(FileFormat.ORC); - return Stream.of(Arguments.of(parquet.toArray()), Arguments.of(orc.toArray())); - }); - } - - @ParameterizedTest - @MethodSource - public void testKeyedTableQuery(TableFormat format, TestTable table, FileFormat fileFormat) { - createTarget( - table.schema, - builder -> - builder - .withPrimaryKeySpec(table.keySpec) - .withProperty(TableProperties.CHANGE_FILE_FORMAT, fileFormat.name()) - .withProperty(TableProperties.BASE_FILE_FORMAT, fileFormat.name())); - - KeyedTable tbl = loadTable().asKeyedTable(); - RecordGenerator dataGen = table.newDateGen(); - - List base = dataGen.records(10); - TestTableUtil.writeToBase(tbl, base); - LinkedList expects = Lists.newLinkedList(base); - - // insert some record in change - List changeInsert = dataGen.records(5); - - // insert some delete in change(delete base records) - List changeDelete = Lists.newArrayList(); - IntStream.range(0, 3).boxed().forEach(i -> changeDelete.add(expects.pollFirst())); - - // insert some delete in change(delete change records) - expects.addAll(changeInsert); - - IntStream.range(0, 2).boxed().forEach(i -> changeDelete.add(expects.pollLast())); - - // insert some delete in change(delete non exists records) - changeDelete.addAll(dataGen.records(3)); - - TestTableUtil.writeToChange(tbl.asKeyedTable(), changeInsert, ChangeAction.INSERT); - TestTableUtil.writeToChange(tbl.asKeyedTable(), changeDelete, ChangeAction.DELETE); - // reload table; - LinkedList expectChange = Lists.newLinkedList(changeInsert); - expectChange.addAll(changeDelete); - - // Assert MOR - Dataset ds = sql("SELECT * FROM " + target() + " ORDER BY id"); - List actual = - ds.collectAsList().stream() - .map(r -> TestTableUtil.rowToRecord(r, table.schema.asStruct())) - .collect(Collectors.toList()); - expects.sort(Comparator.comparing(r -> r.get(0, Integer.class))); - - DataComparator.build(expects, actual).assertRecordsEqual(); - - ds = sql("SELECT * FROM " + target() + ".change" + " ORDER BY ID"); - List changeActual = ds.collectAsList(); - Assertions.assertEquals(expectChange.size(), changeActual.size()); - - Schema changeSchema = MetadataColumns.appendChangeStoreMetadataColumns(table.schema); - changeActual.stream() - .map(r -> TestTableUtil.rowToRecord(r, changeSchema.asStruct())) - .forEach( - r -> { - Assertions.assertNotNull(r.getField(MetadataColumns.CHANGE_ACTION_NAME)); - Assertions.assertTrue( - ((Long) r.getField(MetadataColumns.TRANSACTION_ID_FILED_NAME)) > 0); - }); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestTruncateSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestTruncateSQL.java deleted file mode 100644 index 4f6ecd709d..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestTruncateSQL.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.amoro.spark.test.utils.TableFiles; -import org.apache.amoro.spark.test.utils.TestTableUtil; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.stream.Stream; - -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestTruncateSQL extends MixedTableTestBase { - - public static Stream testTruncateTable() { - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", ""), - Arguments.of(TableFormat.MIXED_HIVE, "", ""), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", ""), - Arguments.of(TableFormat.MIXED_ICEBERG, "", ""), - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", " PARTITIONED BY (day)"), - Arguments.of(TableFormat.MIXED_HIVE, "", " PARTITIONED BY (day)"), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", " PARTITIONED BY (day)"), - Arguments.of(TableFormat.MIXED_ICEBERG, "", " PARTITIONED BY (day)")); - } - - @DisplayName("Test `test truncate table`") - @ParameterizedTest - @MethodSource - public void testTruncateTable(TableFormat format, String primaryKeyDDL, String partitionDDL) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id int, data string, day string " - + primaryKeyDDL - + " ) using " - + provider(format) - + partitionDDL; - sql(sqlText); - sql( - "insert into " - + target().database - + "." - + target().table - + " values (1, 'a', 'a'), (2, 'b', 'b'), (3, 'c', 'c')"); - TableFiles files = TestTableUtil.files(loadTable()); - Assertions.assertEquals(3, files.totalFileCount()); - - sql("truncate table " + target().database + "." + target().table); - Dataset sql = sql("select * from " + target().database + "." + target().table); - Assertions.assertEquals(0, sql.collectAsList().size()); - files = TestTableUtil.files(loadTable()); - Assertions.assertEquals(0, files.totalFileCount()); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestUpdateSQL.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestUpdateSQL.java deleted file mode 100644 index 116432e247..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/sql/TestUpdateSQL.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.sql; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.extensions.EnableCatalogSelect; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.junit.Assert; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.util.stream.Stream; - -// TODO: @jinsilei -@EnableCatalogSelect -@EnableCatalogSelect.SelectCatalog(byTableFormat = true) -public class TestUpdateSQL extends MixedTableTestBase { - - public static Stream testUpdate() { - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", " where id = 3"), - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", " where day = 'c'"), - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", ""), - Arguments.of(TableFormat.MIXED_HIVE, "", " where id = 3"), - Arguments.of(TableFormat.MIXED_HIVE, "", ""), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", " where id = 3"), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", " where day = 'c'"), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", ""), - Arguments.of(TableFormat.MIXED_ICEBERG, "", " where id = 3"), - Arguments.of(TableFormat.MIXED_ICEBERG, "", "")); - } - - @DisplayName("Test `test update table`") - @ParameterizedTest - @MethodSource - public void testUpdate(TableFormat format, String primaryKeyDDL, String filter) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id int, data string, day string " - + primaryKeyDDL - + " ) using " - + provider(format) - + " PARTITIONED BY (day)"; - sql(sqlText); - sql( - "insert into " - + target().database - + "." - + target().table - + " values (1, 'a', 'a'), (2, 'b', 'b'), (3, 'c', 'c')"); - - sql("update " + target().database + "." + target().table + " set data = 'd'" + filter); - - Dataset sql = - sql("select id, data from " + target().database + "." + target().table + filter); - - if (filter.isEmpty()) { - Assertions.assertEquals(3, sql.collectAsList().size()); - Assertions.assertEquals("d", sql.collectAsList().get(0).get(1)); - } else { - Assertions.assertEquals(1, sql.collectAsList().size()); - Assertions.assertEquals("d", sql.collectAsList().get(0).get(1)); - } - } - - public static Stream testUpdatePartitionField() { - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", " where id = 3"), - Arguments.of(TableFormat.MIXED_HIVE, "", " where id = 3"), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", " where id = 3"), - Arguments.of(TableFormat.MIXED_ICEBERG, "", " where id = 3")); - } - - @DisplayName("Test `test update partition field`") - @ParameterizedTest - @MethodSource - public void testUpdatePartitionField(TableFormat format, String primaryKeyDDL, String filter) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id int, data string, day string " - + primaryKeyDDL - + " ) using " - + provider(format) - + " PARTITIONED BY (day)"; - sql(sqlText); - sql( - "insert into " - + target().database - + "." - + target().table - + " values (1, 'a', 'a'), (2, 'b', 'b'), (3, 'c', 'c')"); - - sql("update " + target().database + "." + target().table + " set day = 'd'" + filter); - - Dataset sql = - sql("select id, day from " + target().database + "." + target().table + filter); - - Assertions.assertEquals(1, sql.collectAsList().size()); - Assertions.assertEquals("d", sql.collectAsList().get(0).get(1)); - } - - public static Stream testUpdatePrimaryField() { - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, ", PRIMARY KEY(id)", " where data = 'c'"), - Arguments.of(TableFormat.MIXED_ICEBERG, ", PRIMARY KEY(id)", " where data = 'c'")); - } - - @DisplayName("Test `test update primary field`") - @ParameterizedTest - @MethodSource - public void testUpdatePrimaryField(TableFormat format, String primaryKeyDDL, String filter) { - String sqlText = - "CREATE TABLE " - + target() - + " ( \n" - + "id int, data string, day string " - + primaryKeyDDL - + " ) using " - + provider(format) - + " PARTITIONED BY (day)"; - sql(sqlText); - sql( - "insert into " - + target().database - + "." - + target().table - + " values (1, 'a', 'a'), (2, 'b', 'b'), (3, 'c', 'c')"); - - Assert.assertThrows( - UnsupportedOperationException.class, - () -> sql("update " + target().database + "." + target().table + " set id = 1" + filter)); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/ut/sql/parser/TestSqlExtendParser.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/ut/sql/parser/TestSqlExtendParser.java deleted file mode 100644 index b9bb0411b4..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/test/suites/ut/sql/parser/TestSqlExtendParser.java +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.test.suites.ut.sql.parser; - -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.sql.catalyst.parser.MixedFormatSqlExtensionsParser; -import org.apache.amoro.spark.test.utils.ScalaTestUtil; -import org.apache.spark.sql.catalyst.parser.AbstractSqlParser; -import org.apache.spark.sql.catalyst.parser.AstBuilder; -import org.apache.spark.sql.catalyst.parser.ParseException; -import org.apache.spark.sql.catalyst.plans.logical.CreateTableAsSelectStatement; -import org.apache.spark.sql.catalyst.plans.logical.CreateTableStatement; -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; -import org.apache.spark.sql.execution.SparkSqlParser; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.StructType; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; -import org.junit.jupiter.params.provider.ValueSource; -import scala.collection.Seq; - -import java.util.List; -import java.util.stream.Stream; - -public class TestSqlExtendParser { - - private final MixedFormatSqlExtensionsParser parser = - new MixedFormatSqlExtensionsParser( - new AbstractSqlParser() { - @Override - public AstBuilder astBuilder() { - return null; - } - }); - - private final SparkSqlParser sparkSqlParser = new SparkSqlParser(); - - @ParameterizedTest - @ValueSource( - strings = { - "CREATE TABLE t1 (id int, PRIMARY KEY(id)) USING arctic ROW FORMAT SERDE 'parquet'", - "CREATE TEMPORARY TABLE t1 PRIMARY KEY(id) USING arctic AS SELECT * from v1", - "CREATE TABLE t1 (id int , PRIMARY KEY(id)) USING arctic AS SELECT * FROM v1", - "CREATE TABLE t1 PRIMARY KEY(id) USING arctic PARTITIONED BY (pt string) AS SELECT * FROM v1", - "CREATE TABLE t1 PRIMARY KEY(id) USING arctic", - "CREATE TABLE t1 (id int, PRIMARY KEY(id)) USING arctic SKEWED BY (id) ", - "CREATE TABLE t1 (id int, pt string, PRIMARY KEY(id)) USING arctic " - + "CLUSTERED BY(id,pt) SORTED BY (pt DESC) INTO 8 BUCKETS", - "CREATE TEMPORARY TABLE IF NOT EXISTS t1 (id int , PRIMARY KEY(id)) USING arctic", - "CREATE TABLE t1 (id int, PRIMARY KEY(id)) USING arctic STORED BY 'a.b.c' ", - "CREATE TABLE t1 (id int, pt string, PRIMARY KEY(id)) USING arctic " - + "PARTITIONED BY (days(pt), dt string)", - }) - public void testOperationNotAllowed(String sqlText) { - ParseException e = - Assertions.assertThrows(ParseException.class, () -> parser.parsePlan(sqlText)); - System.out.println(e.getMessage()); - Assertions.assertTrue( - e.getMessage().contains("Operation not allowed:"), - "Not an 'Operation not allowed Exception'"); - } - - public static Arguments[] testCreateTableWithPrimaryKey() { - return new Arguments[] { - Arguments.arguments( - "CREATE TABLE mydb.t1 (id int, PRIMARY KEY(id)) ", - Lists.newArrayList("mydb", "t1"), - new StructType().add("id", DataTypes.IntegerType, false)) - }; - } - - @ParameterizedTest - @MethodSource - public void testCreateTableWithPrimaryKey( - String sqlText, List expectTableName, StructType expectSchema) { - LogicalPlan plan = parser.parsePlan(sqlText); - Assertions.assertTrue(plan instanceof CreateTableStatement, "Not a CreateTableStatement"); - CreateTableStatement create = (CreateTableStatement) plan; - - Seq expectNameSeq = ScalaTestUtil.seq(expectTableName); - Assertions.assertEquals(expectNameSeq, create.tableName()); - Assertions.assertEquals(expectSchema, create.tableSchema()); - } - - public static Stream testCreateTableAsSelect() { - String header = "CREATE TABLE mydb.t1 PRIMARY KEY(id) "; - List queries = - Lists.newArrayList( - "SELECT * FROM v1", - "sELEct * FroM a", - "select * from a union select * from b", - "select * from a union distinct select * from b", - "select * from a union all select * from b", - "select * from a except select * from b", - "select * from a except distinct select * from b", - "select * from a except all select * from b", - "select * from a minus select * from b", - "select * from a minus all select * from b", - "select * from a minus distinct select * from b", - "select * from a intersect select * from b", - "select * from a intersect distinct select * from b", - "select * from a intersect all select * from b", - "select 1", - "select a, b", - "select a, b from db.c", - "select a, b from db.c where x < 1", - "select a, b from db.c having x = 1", - "select a, b from db.c having x > 1", - "select a, b from db.c having x >= 1", - "select a, b from db.c having x <= 1", - "select distinct a, b from db.c", - "select all a, b from db.c", - "select a from 1k.2m", - "select a from db.c where x between 1 AND 10 ", - "select a from db.c where x not between 1 AND 10 ", - "select a from db.c where x in(1, 2, 3)", - "select a from db.c where x not in(1, 2, 3)", - "select a from db.c where x like 'a%'", - "select a from db.c where x is null ", - "select a from db.c where x is not null ", - "select a from db.c where x is true ", - "select a from db.c where x is false ", - "select a from db.c where x is unknown ", - "from a select b, c "); - - return queries.stream().map(s -> Arguments.arguments(header, s)); - } - - @ParameterizedTest - @MethodSource - public void testCreateTableAsSelect(String ctasHeader, String asSelectBody) { - String sqlText = ctasHeader + " AS " + asSelectBody; - LogicalPlan plan = parser.parsePlan(sqlText); - Assertions.assertTrue( - plan instanceof CreateTableAsSelectStatement, "Not a CreateTableAsSelectStatement"); - CreateTableAsSelectStatement create = (CreateTableAsSelectStatement) plan; - LogicalPlan ctasQuery = create.asSelect(); - LogicalPlan expectQuery = sparkSqlParser.parsePlan(asSelectBody); - - Assertions.assertEquals(expectQuery, ctasQuery); - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/writer/TestMixedWriter.java b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/writer/TestMixedWriter.java deleted file mode 100644 index fc5f46e5e6..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/java/org/apache/amoro/spark/writer/TestMixedWriter.java +++ /dev/null @@ -1,303 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.amoro.spark.writer; - -import org.apache.amoro.TableFormat; -import org.apache.amoro.hive.io.HiveDataTestHelpers; -import org.apache.amoro.hive.table.SupportHive; -import org.apache.amoro.properties.HiveTableProperties; -import org.apache.amoro.shade.guava32.com.google.common.collect.Iterators; -import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; -import org.apache.amoro.spark.io.TaskWriters; -import org.apache.amoro.spark.reader.SparkParquetReaders; -import org.apache.amoro.spark.test.MixedTableTestBase; -import org.apache.amoro.spark.test.utils.RecordGenerator; -import org.apache.amoro.spark.test.utils.TestTableUtil; -import org.apache.amoro.table.MixedTable; -import org.apache.amoro.table.PrimaryKeySpec; -import org.apache.amoro.table.TableProperties; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Files; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.AdaptHiveParquet; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.data.SparkOrcReader; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.connector.write.DataWriter; -import org.apache.spark.sql.connector.write.LogicalWriteInfoImpl; -import org.apache.spark.sql.connector.write.Write; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.apache.spark.unsafe.types.UTF8String; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -public class TestMixedWriter extends MixedTableTestBase { - - static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get()), - Types.NestedField.required(3, "pt", Types.StringType.get())); - - static final PrimaryKeySpec ID_PRIMARY_KEY_SPEC = - PrimaryKeySpec.builderFor(SCHEMA).addColumn("id").build(); - - static final PartitionSpec PT_SPEC = PartitionSpec.builderFor(SCHEMA).identity("pt").build(); - - public static Stream testWrite() { - return Stream.of( - Arguments.of( - MIXED_HIVE, WriteMode.APPEND, SCHEMA, ID_PRIMARY_KEY_SPEC, PT_SPEC, FileFormat.PARQUET), - Arguments.of( - MIXED_HIVE, WriteMode.APPEND, SCHEMA, NO_PRIMARY_KEY, PT_SPEC, FileFormat.PARQUET), - Arguments.of( - MIXED_HIVE, - WriteMode.APPEND, - SCHEMA, - ID_PRIMARY_KEY_SPEC, - UNPARTITIONED, - FileFormat.PARQUET), - Arguments.of( - MIXED_HIVE, - WriteMode.APPEND, - SCHEMA, - NO_PRIMARY_KEY, - UNPARTITIONED, - FileFormat.PARQUET), - Arguments.of( - MIXED_HIVE, WriteMode.APPEND, SCHEMA, ID_PRIMARY_KEY_SPEC, PT_SPEC, FileFormat.ORC), - Arguments.of(MIXED_HIVE, WriteMode.APPEND, SCHEMA, NO_PRIMARY_KEY, PT_SPEC, FileFormat.ORC), - Arguments.of( - MIXED_HIVE, - WriteMode.APPEND, - SCHEMA, - ID_PRIMARY_KEY_SPEC, - UNPARTITIONED, - FileFormat.ORC), - Arguments.of( - MIXED_HIVE, WriteMode.APPEND, SCHEMA, NO_PRIMARY_KEY, UNPARTITIONED, FileFormat.ORC), - Arguments.of( - MIXED_HIVE, - WriteMode.OVERWRITE_DYNAMIC, - SCHEMA, - ID_PRIMARY_KEY_SPEC, - PT_SPEC, - FileFormat.PARQUET), - Arguments.of( - MIXED_HIVE, - WriteMode.OVERWRITE_DYNAMIC, - SCHEMA, - NO_PRIMARY_KEY, - PT_SPEC, - FileFormat.PARQUET), - Arguments.of( - MIXED_HIVE, - WriteMode.OVERWRITE_DYNAMIC, - SCHEMA, - ID_PRIMARY_KEY_SPEC, - UNPARTITIONED, - FileFormat.PARQUET), - Arguments.of( - MIXED_HIVE, - WriteMode.OVERWRITE_DYNAMIC, - SCHEMA, - NO_PRIMARY_KEY, - UNPARTITIONED, - FileFormat.PARQUET), - Arguments.of( - MIXED_HIVE, - WriteMode.OVERWRITE_DYNAMIC, - SCHEMA, - ID_PRIMARY_KEY_SPEC, - PT_SPEC, - FileFormat.ORC), - Arguments.of( - MIXED_HIVE, - WriteMode.OVERWRITE_DYNAMIC, - SCHEMA, - NO_PRIMARY_KEY, - PT_SPEC, - FileFormat.ORC), - Arguments.of( - MIXED_HIVE, - WriteMode.OVERWRITE_DYNAMIC, - SCHEMA, - ID_PRIMARY_KEY_SPEC, - UNPARTITIONED, - FileFormat.ORC), - Arguments.of( - MIXED_HIVE, - WriteMode.OVERWRITE_DYNAMIC, - SCHEMA, - NO_PRIMARY_KEY, - UNPARTITIONED, - FileFormat.ORC)); - } - - @DisplayName("Test write mix_hive Table") - @ParameterizedTest - @MethodSource - public void testWrite( - TableFormat format, - WriteMode writeMode, - Schema schema, - PrimaryKeySpec keySpec, - PartitionSpec ptSpec, - FileFormat fileFormat) - throws IOException { - MixedTable table = - createTarget( - schema, - tableBuilder -> - tableBuilder - .withPrimaryKeySpec(keySpec) - .withProperty(TableProperties.CHANGE_FILE_FORMAT, fileFormat.name()) - .withProperty(TableProperties.BASE_FILE_FORMAT, fileFormat.name()) - .withProperty(TableProperties.DEFAULT_FILE_FORMAT, fileFormat.name()) - .withPartitionSpec(ptSpec)); - Map map = new HashMap<>(); - map.put(WriteMode.WRITE_MODE_KEY, writeMode.mode); - testWriteData(table, map); - } - - private void testWriteData(MixedTable table, Map map) throws IOException { - StructType structType = SparkSchemaUtil.convert(table.schema()); - LogicalWriteInfoImpl info = - new LogicalWriteInfoImpl("queryId", structType, new CaseInsensitiveStringMap(map)); - MixedFormatSparkWriteBuilder builder = new MixedFormatSparkWriteBuilder(table, info, catalog()); - Write write = builder.build(); - DataWriter writer = - write.toBatch().createBatchWriterFactory(null).createWriter(0, 0); - // create record - InternalRow record = geneRowData(); - List records = Collections.singletonList(record); - writer.write(record); - WriteTaskCommit commit = (WriteTaskCommit) writer.commit(); - DataFile[] files = commit.files(); - CloseableIterable concat = - CloseableIterable.concat( - Arrays.stream(files) - .map( - s -> { - switch (s.format()) { - case PARQUET: - return readParquet(table.schema(), s.path().toString()); - case ORC: - return readOrc(table.schema(), s.path().toString()); - default: - throw new UnsupportedOperationException( - "Cannot read unknown format: " + s.format()); - } - }) - .collect(Collectors.toList())); - Set result = new HashSet<>(); - Iterators.addAll(result, concat.iterator()); - Assertions.assertEquals(result, new HashSet<>(records)); - } - - private CloseableIterable readParquet(Schema schema, String path) { - AdaptHiveParquet.ReadBuilder builder = - AdaptHiveParquet.read(Files.localInput(path)) - .project(schema) - .createReaderFunc( - fileSchema -> SparkParquetReaders.buildReader(schema, fileSchema, new HashMap<>())) - .caseSensitive(false); - - return builder.build(); - } - - private CloseableIterable readOrc(Schema schema, String path) { - ORC.ReadBuilder builder = - ORC.read(Files.localInput(path)) - .project(schema) - .createReaderFunc(fileSchema -> new SparkOrcReader(schema, fileSchema, new HashMap<>())) - .caseSensitive(false); - - return builder.build(); - } - - private InternalRow geneRowData() { - return new GenericInternalRow( - new Object[] {1, UTF8String.fromString("aaa"), UTF8String.fromString("AAA")}); - } - - public static Stream testConsistentWrite() { - return Stream.of( - Arguments.of(TableFormat.MIXED_HIVE, true), Arguments.of(TableFormat.MIXED_HIVE, false)); - } - - @ParameterizedTest - @MethodSource - public void testConsistentWrite(TableFormat format, boolean enableConsistentWrite) { - MixedTable table = - createTarget( - SCHEMA, - builder -> - builder.withProperty( - HiveTableProperties.HIVE_CONSISTENT_WRITE_ENABLED, enableConsistentWrite + "")); - StructType dsSchema = SparkSchemaUtil.convert(SCHEMA); - List records = RecordGenerator.buildFor(SCHEMA).build().records(10); - try (TaskWriter writer = - TaskWriters.of(table) - .withOrderedWriter(false) - .withDataSourceSchema(dsSchema) - .newBaseWriter(true)) { - records.stream() - .map(r -> TestTableUtil.recordToInternalRow(SCHEMA, r)) - .forEach( - i -> { - try { - writer.write(i); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - WriteResult result = writer.complete(); - DataFile[] dataFiles = result.dataFiles(); - HiveDataTestHelpers.assertWriteConsistentFilesName( - (SupportHive) table, Lists.newArrayList(dataFiles)); - } catch (IOException e) { - throw new RuntimeException(e); - } - } -} diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/resources/log4j2.properties b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/resources/log4j2.properties deleted file mode 100644 index 018ac5fe9f..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-3.2/src/test/resources/log4j2.properties +++ /dev/null @@ -1,32 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -log4j.rootLogger=INFO -log4j.logger.org.apache.amoro=INFO -log4j.logger.org.apache.amoro.spark=DEBUG -log4j.logger.org.apache.thrift.server=WARN -log4j.logger.org.sparkproject.jetty.server=WARN -log4j.logger.org.apache.amoro.client=WARN - - -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.Threshold=DEBUG -log4j.appender.console.ImmediateFlush=true -log4j.appender.console.Target=System.out -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=[%-5p] %d{HH:mm:ss,SSS} [%t] %C{2}.%L: %m %x %n diff --git a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-runtime-3.2/pom.xml b/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-runtime-3.2/pom.xml deleted file mode 100644 index f243bd1d4b..0000000000 --- a/amoro-format-mixed/amoro-mixed-spark/v3.2/amoro-mixed-spark-runtime-3.2/pom.xml +++ /dev/null @@ -1,502 +0,0 @@ - - - - - - amoro-mixed-spark - org.apache.amoro - 0.8-SNAPSHOT - ../../pom.xml - - 4.0.0 - - amoro-format-mixed-spark-runtime-3.2 - jar - Amoro Project Mixed Format Spark 3.2 Runtime - https://amoro.apache.org - - - - org.apache.amoro - amoro-format-mixed-spark-3.2 - ${project.parent.version} - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-amoro - package - - shade - - - false - false - - - org.apache.amoro:amoro-format-mixed-spark-3-common - org.apache.amoro:amoro-format-mixed-spark-3.2 - org.apache.amoro:amoro-common - org.apache.amoro:amoro-format-iceberg - org.apache.amoro:amoro-mixed-hive - org.apache.amoro:amoro-shade-guava-32 - org.apache.amoro:amoro-shade-jackson-2 - org.apache.amoro:amoro-shade-thrift - org.apache.amoro:amoro-shade-zookeeper-3 - org.apache.iceberg:iceberg-core - org.apache.iceberg:iceberg-api - org.apache.iceberg:iceberg-spark-3.2_2.12 - org.apache.iceberg:iceberg-common - org.apache.iceberg:iceberg-data - org.apache.iceberg:iceberg-orc - org.apache.iceberg:iceberg-parquet - org.apache.iceberg:iceberg-aws - org.apache.iceberg:iceberg-aliyun - org.apache.parquet:parquet-column - org.apache.parquet:parquet-hadoop - org.apache.parquet:parquet-common - org.apache.parquet:parquet-format - org.apache.parquet:parquet-format-structures - org.apache.parquet:parquet-encoding - org.apache.parquet:parquet-jackson - org.apache.iceberg:iceberg-arrow - org.apache.iceberg:iceberg-hive-metastore - org.apache.iceberg:iceberg-spark - org.apache.iceberg:iceberg-spark-extensions-3.2_2.12 - - org.apache.iceberg:iceberg-bundled-guava - org.apache.orc:* - org.apache.thrift:libthrift - org.apache.avro:avro - org.apache.parquet:parquet-avro - org.apache.commons:commons-pool2 - org.apache.commons:commons-lang3 - org.apache.commons:commons-collections - com.github.ben-manes.caffeine:caffeine - org.apache.httpcomponents.client5:* - org.apache.httpcomponents.core5:* - - - com.google.code.findbugs:jsr305 - org.slf4j:* - org.apache.logging.log4j:* - - - - - * - - org/apache/iceberg/spark/extensions/** - - - - org.apache.iceberg:iceberg-spark-3.2_2.12 - - META-INF/services/org.apache.spark.sql.sources.DataSourceRegister - - - - - - - org.apache.spark.sql.amoro - org.apache.amoro.spark.sql - - - - - org.apache.orc - org.apache.amoro.shade.org.apache.orc - - - - org.apache.avro - org.apache.amoro.shade.org.apache.avro - - - - org.apache.parquet.avro - - org.apache.amoro.shade.org.apache.parquet.avro - - - - org.apache.parquet.column - - org.apache.amoro.shade.org.apache.parquet.column - - - - org.apache.parquet.hadoop - - org.apache.amoro.shade.org.apache.parquet.hadoop - - - - org.apache.parquet.common - - org.apache.amoro.shade.org.apache.parquet.common - - - - org.apache.parquet.format - - org.apache.amoro.shade.org.apache.parquet.format - - - - shaded.parquet - org.apache.amoro.shade.parquet - - - shaded.parquet.it - org.apache.amoro.shade.parquet.it - - - - org.apache.parquet.crypto - - org.apache.amoro.shade.org.apache.parquet.crypto - - - - org.apache.parquet - org.apache.amoro.shade.org.apache.parquet - - - - org.apache.commons.pool2 - - org.apache.amoro.shade.org.apache.commons.pool2 - - - - com.github.ben-manes - - org.apache.amoro.shade.com.github.ben-manes - - - - org.apache.commons.lang3 - - org.apache.amoro.shade.org.apache.commons.lang3 - - - - - - org.apache.iceberg - org.apache.amoro.shade.org.apache.iceberg - - - org.apache.iceberg.mr.hive.* - - - - org.apache.spark.sql.connector.iceberg - - org.apache.amoro.shade.org.apache.spark.sql.connector.iceberg - - - - org.apache.spark.sql.connector.iceberg.catalog.ProcedureCatalog - - - org.apache.spark.sql.connector.iceberg.catalog.Procedure - - - org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter - - - org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameterImpl - - - - - - - org.apache.spark.sql.execution.datasources.v2 - - org.apache.amoro.shade.org.apache.spark.sql.execution.datasources.v2 - - - - org.apache.spark.sql.execution.datasources.v2.AddPartitionFieldExec* - - - org.apache.spark.sql.execution.datasources.v2.CallExec* - - - org.apache.spark.sql.execution.datasources.v2.DropIdentifierFields* - - - org.apache.spark.sql.execution.datasources.v2.DropPartitionFieldExec* - - - org.apache.spark.sql.execution.datasources.v2.DynamicFileFilterExec* - - - org.apache.spark.sql.execution.datasources.v2.DynamicFileFilterWithCardinalityCheckExec* - - - org.apache.spark.sql.execution.datasources.v2.ExtendedBatchScanExec* - - - org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits* - - - org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Strategy* - - - org.apache.spark.sql.execution.datasources.v2.MixedFormatExtendedDataSourceV2Strategy* - - - org.apache.spark.sql.execution.datasources.v2.MergeIntoExec* - - - org.apache.spark.sql.execution.datasources.v2.ReplaceDataExec* - - - org.apache.spark.sql.execution.datasources.v2.ReplacePartitionFieldExec* - - - org.apache.spark.sql.execution.datasources.v2.SetIdentifierFieldsExec* - - - org.apache.spark.sql.execution.datasources.v2.SetWriteDistributionAndOrderingExec* - - - org.apache.spark.sql.execution.datasources.v2.OptimizeMetadataOnlyDeleteFromTable* - - - org.apache.spark.sql.execution.datasources.v2.RowLevelCommandScanRelationPushDown* - - - org.apache.spark.sql.execution.datasources.v2.ExtendedV2Writes* - - - org.apache.spark.sql.execution.datasources.v2.ReplaceRewrittenRowLevelCommand* - - - - - org.apache.spark.sql.execution.datasources - - org.apache.amoro.shade.org.apache.spark.sql.execution.datasources - - - - org.apache.spark.sql.execution.datasources.SparkExpressionConverter* - - - - - org.apache.spark.sql.catalyst.analysis - - org.apache.amoro.shade.org.apache.spark.sql.catalyst.analysis - - - - org.apache.spark.sql.catalyst.analysis.AlignRowLevelOperations* - - - org.apache.spark.sql.catalyst.analysis.AssignmentAlignmentSupport* - - - org.apache.spark.sql.catalyst.analysis.ProcedureArgumentCoercion* - - - org.apache.spark.sql.catalyst.analysis.ResolveProcedures* - - - org.apache.spark.sql.catalyst.analysis.RowLevelOperationsPredicateCheck* - - - org.apache.spark.sql.catalyst.analysis.ResolveMergeIntoTableReferences* - - - org.apache.spark.sql.catalyst.analysis.CheckMergeIntoTableConditions* - - - org.apache.spark.sql.catalyst.analysis.AlignRowLevelCommandAssignments* - - - org.apache.spark.sql.catalyst.analysis.RewriteDeleteFromTable* - - - org.apache.spark.sql.catalyst.analysis.RewriteUpdateTable* - - - org.apache.spark.sql.catalyst.analysis.RewriteMergeIntoTable* - - - org.apache.spark.sql.catalyst.analysis.MergeIntoIcebergTableResolutionCheck* - - - org.apache.spark.sql.catalyst.analysis.AlignedRowLevelIcebergCommandCheck* - - - - - org.apache.spark.sql.catalyst.expressions - - org.apache.amoro.shade.org.apache.spark.sql.catalyst.expressions - - - - org.apache.spark.sql.catalyst.expressions.AccumulateFiles* - - - org.apache.spark.sql.catalyst.expressions.Iceberg* - - - - - org.apache.spark.sql.catalyst.optimizer - - org.apache.amoro.shade.org.apache.spark.sql.catalyst.optimizer - - - - org.apache.spark.sql.catalyst.optimizer.ExtendedSimplifyConditionalsInPredicate* - - - org.apache.spark.sql.catalyst.optimizer.ExtendedReplaceNullWithFalseInPredicate* - - - - - org.apache.spark.sql.catalyst.parser.extensions - - - org.apache.amoro.shade.org.apache.spark.sql.catalyst.parser.extensions - - - - org.apache.spark.sql.catalyst.parser.extensions.Iceberg* - - - org.apache.spark.sql.catalyst.parser.extensions.UpperCaseCharStream* - - - org.apache.spark.sql.catalyst.parser.extensions.IcebergSparkSqlExtensionsParser* - - - - - org.apache.spark.sql.catalyst.plans.logical - - org.apache.amoro.shade.org.apache.spark.sql.catalyst.plans.logical - - - - org.apache.spark.sql.catalyst.plans.logical.AddPartitionField* - - org.apache.spark.sql.catalyst.plans.logical.Call* - - - org.apache.spark.sql.catalyst.plans.logical.DropPartitionField - - - org.apache.spark.sql.catalyst.plans.logical.DynamicFileFilter* - - - org.apache.spark.sql.catalyst.plans.logical.MergeInto - - - org.apache.spark.sql.catalyst.plans.logical.MergeInto$ - - - org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams* - - - org.apache.spark.sql.catalyst.plans.logical.NamedArgument* - - - org.apache.spark.sql.catalyst.plans.logical.PositionalArgument* - - - org.apache.spark.sql.catalyst.plans.logical.ReplaceData* - - - org.apache.spark.sql.catalyst.plans.logical.ReplacePartitionField* - - - org.apache.spark.sql.catalyst.plans.logical.SetIdentifierFields* - - - org.apache.spark.sql.catalyst.plans.logical.SetWriteDistributionAndOrdering* - - - org.apache.spark.sql.catalyst.plans.logical.SortOrderParserUtil* - - - - - org.apache.spark.sql.catalyst.utils - - org.apache.amoro.shade.org.apache.spark.sql.catalyst.utils - - - - org.apache.spark.sql.catalyst.utils.DistributionAndOrderingUtils* - - org.apache.spark.sql.catalyst.utils.PlanUtils* - - - org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper* - - - org.apache.spark.sql.catalyst.utils.SetAccumulator* - - - - - org.apache.hc - org.apache.amoro.shade.org.apache.hc - - - - org.apache.spark.sql.execution.dynamicpruning - - org.apache.amoro.shade.org.apache.spark.sql.execution.dynamicpruning - - - - org.apache.spark.sql.execution.dynamicpruning.RowLevelCommandDynamicPruning* - - - - - ${project.artifactId}-${project.version} - - - - - - - diff --git a/pom.xml b/pom.xml index 97a882b187..458481f385 100644 --- a/pom.xml +++ b/pom.xml @@ -101,7 +101,7 @@ 3.3.2 3.3.1 - 1.4.3 + 1.6.1 0.9.0 3.1.1 3.4.0 @@ -1360,8 +1360,8 @@ 14.0.1 2.3.9 2.10.2 - 3.2.4 - 3.2 + 3.3.3 + 3.3