DBZ-20 Initial import of Oracle connector based on XStream

This commit is contained in:
Gunnar Morling 2017-10-25 10:04:06 +02:00 committed by Jiri Pechanec
parent f8b3cea6d4
commit d7e196a18e
56 changed files with 11794 additions and 54 deletions

View File

@ -120,6 +120,32 @@ To run the integration tests of the PG connector using wal2json, enable the "wal
A few tests currently don't pass when using the wal2json plug-in.
Look for references to the types defined in `io.debezium.connector.postgresql.DecoderDifferences` to find these tests.
### Building the Oracle connector
In order to build the Debezium Oracle connector, the following prerequisites must be met:
* Oracle DB is installed and enabled for change data capturing (TODO: describe the details)
* The Instant Client is downloaded (e.g. [from here](http://www.oracle.com/technetwork/topics/linuxx86-64soft-092277.html) for Linux) and unpacked
* The _xstream.jar_ and _ojdbc8.jar_ from the Instant Client directory must be installed to the local Maven repository:
* mvn install:install-file \
-DgroupId=com.oracle.instantclient \
-DartifactId=ojdbc8 \
-Dversion=12.1.0.2 \
-Dpackaging=jar \
-Dfile=ojdbc8.jar`
* mvn install:install-file \
-DgroupId=com.oracle.instantclient \
-DartifactId=xstreams \
-Dversion=12.1.0.2 \
-Dpackaging=jar \
-Dfile=xstreams.jar
* A user "debezium" with password "dbz" exists in the pluggable database (PDB) "ORCLPDB1"
* A user "c##xstrmadmin" with password "xsa" exists in the container database (CDB) "ORCLCDB" and has XStream adminstrative privileges
Then the Oracle connector can be built like so:
$ mvn clean install -pl debezium-connector-oracle -am -Poracle -Dinstantclient.dir=/path/to/instant-client-dir
## Contributing
The Debezium community welcomes anyone that wants to help out in any way, whether that includes reporting problems, helping with documentation, or contributing code changes to fix bugs, add tests, or implement new features. See [this document](CONTRIBUTE.md) for details.

View File

@ -217,7 +217,7 @@ public class MongoDbConnectorConfig extends CommonConnectorConfig {
protected static Field.Set EXPOSED_FIELDS = ALL_FIELDS;
public MongoDbConnectorConfig(Configuration config) {
super(config);
super(config, LOGICAL_NAME);
}
protected static ConfigDef configDef() {

View File

@ -1053,7 +1053,7 @@ public static final Field MASK_COLUMN(int length) {
private final DdlParsingMode ddlParsingMode;
public MySqlConnectorConfig(Configuration config) {
super(config);
super(config, SERVER_NAME);
this.config = config;

View File

@ -0,0 +1,64 @@
package io.debezium.connector.mysql;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.connect.connector.Task;
import org.apache.kafka.connect.source.SourceConnector;
import org.apache.kafka.connect.source.SourceRecord;
import org.apache.kafka.connect.source.SourceTask;
public class TestConnector extends SourceConnector {
@Override
public String version() {
return null;
}
@Override
public void start(Map<String, String> props) {
}
@Override
public Class<? extends Task> taskClass() {
return TestTask.class;
}
@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
return Collections.singletonList(Collections.singletonMap("foo", "bar"));
}
@Override
public void stop() {
}
@Override
public ConfigDef config() {
return new ConfigDef();
}
public static class TestTask extends SourceTask {
@Override
public String version() {
return null;
}
@Override
public void start(Map<String, String> props) {
}
@Override
public List<SourceRecord> poll() throws InterruptedException {
throw new RuntimeException();
}
@Override
public void stop() {
System.out.println("stop() called");
}
}
}

View File

@ -5,12 +5,12 @@
*/
package io.debezium.connector.mysql;
import java.util.Map;
import io.debezium.config.Configuration;
import io.debezium.jdbc.JdbcConfiguration;
import io.debezium.jdbc.JdbcConnection;
import java.util.Map;
/**
* A utility for integration test cases to connect the MySQL server running in the Docker container created by this module's
* build.
@ -82,15 +82,4 @@ protected static void addDefaults(Configuration.Builder builder) {
public MySQLConnection(Configuration config) {
super(config, FACTORY, null, MySQLConnection::addDefaults);
}
/**
* Create a new instance with the given configuration and connection factory, and specify the operations that should be
* run against each newly-established connection.
*
* @param config the configuration; may not be null
* @param initialOperations the initial operations that should be run on each new connection; may be null
*/
public MySQLConnection(Configuration config, Operations initialOperations) {
super(config, FACTORY, initialOperations, MySQLConnection::addDefaults);
}
}

View File

@ -0,0 +1,198 @@
<?xml version="1.0"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<groupId>io.debezium</groupId>
<artifactId>debezium-parent</artifactId>
<version>0.8.0-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>debezium-connector-oracle</artifactId>
<name>Debezium Connector for Oracle</name>
<packaging>jar</packaging>
<properties>
<antlr4.listener>true</antlr4.listener>
</properties>
<dependencies>
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>connect-api</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.oracle.instantclient</groupId>
<artifactId>ojdbc8</artifactId>
<version>12.1.0.2</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.oracle.instantclient</groupId>
<artifactId>xstreams</artifactId>
<version>12.1.0.2</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.antlr</groupId>
<artifactId>antlr4-runtime</artifactId>
<version>4.7</version>
</dependency>
<!-- Testing -->
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-core</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-embedded</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-embedded</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easytesting</groupId>
<artifactId>fest-assert</artifactId>
<scope>test</scope>
</dependency>
<!-- Required by VerifyRecord -->
<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-connect-avro-converter</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<resources>
<!-- Apply the properties set in the POM to the resource files -->
<resource>
<filtering>true</filtering>
<directory>src/main/resources</directory>
<includes>
<include>*</include>
<include>**/*</include>
</includes>
</resource>
</resources>
<testResources>
<testResource>
<directory>src/test/resources</directory>
<filtering>true</filtering>
<includes>
<include>*</include>
<include>**/*</include>
</includes>
</testResource>
</testResources>
<plugins>
<plugin>
<groupId>org.antlr</groupId>
<artifactId>antlr4-maven-plugin</artifactId>
<version>4.7.1</version>
<executions>
<execution>
<id>antlr</id>
<goals>
<goal>antlr4</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-failsafe-plugin</artifactId>
<executions>
<execution>
<id>integration-test</id>
<goals>
<goal>integration-test</goal>
</goals>
</execution>
<execution>
<id>verify</id>
<goals>
<goal>verify</goal>
</goals>
</execution>
</executions>
<configuration>
<skipTests>${skipITs}</skipTests>
<enableAssertions>true</enableAssertions>
<forkCount>1</forkCount>
<argLine>-Djava.library.path=${instantclient.dir}</argLine>
</configuration>
</plugin>
</plugins>
</build>
<!--
Define several useful profiles
-->
<profiles>
<profile>
<id>assembly</id>
<activation>
<activeByDefault>false</activeByDefault>
</activation>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-assembly-plugin</artifactId>
<dependencies>
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-assembly-descriptors</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
<executions>
<execution>
<id>default</id>
<phase>package</phase>
<goals>
<goal>single</goal>
</goals>
<configuration>
<finalName>${project.artifactId}-${project.version}</finalName>
<attach>true</attach> <!-- we want attach & deploy these to Maven -->
<descriptorRefs>
<descriptorRef>connector-distribution</descriptorRef>
</descriptorRefs>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View File

@ -0,0 +1,2387 @@
/**
* Oracle(c) PL/SQL 11g Parser
*
* Copyright (c) 2009-2011 Alexandre Porcelli <alexandre.porcelli@gmail.com>
* Copyright (c) 2015-2017 Ivan Kochurkin (KvanTTT, kvanttt@gmail.com, Positive Technologies).
* Copyright (c) 2017 Mark Adams <madams51703@gmail.com>
*
* 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.
*/
lexer grammar PlSqlLexer;
ABORT: 'ABORT';
ABS: 'ABS';
ACCESS: 'ACCESS';
ACCESSED: 'ACCESSED';
ACCOUNT: 'ACCOUNT';
ACL: 'ACL';
ACOS: 'ACOS';
ACTION: 'ACTION';
ACTIONS: 'ACTIONS';
ACTIVATE: 'ACTIVATE';
ACTIVE: 'ACTIVE';
ACTIVE_COMPONENT: 'ACTIVE_COMPONENT';
ACTIVE_DATA: 'ACTIVE_DATA';
ACTIVE_FUNCTION: 'ACTIVE_FUNCTION';
ACTIVE_TAG: 'ACTIVE_TAG';
ACTIVITY: 'ACTIVITY';
ADAPTIVE_PLAN: 'ADAPTIVE_PLAN';
ADD: 'ADD';
ADD_COLUMN: 'ADD_COLUMN';
ADD_GROUP: 'ADD_GROUP';
ADD_MONTHS: 'ADD_MONTHS';
ADJ_DATE: 'ADJ_DATE';
ADMIN: 'ADMIN';
ADMINISTER: 'ADMINISTER';
ADMINISTRATOR: 'ADMINISTRATOR';
ADVANCED: 'ADVANCED';
ADVISE: 'ADVISE';
ADVISOR: 'ADVISOR';
AFD_DISKSTRING: 'AFD_DISKSTRING';
AFTER: 'AFTER';
AGENT: 'AGENT';
AGGREGATE: 'AGGREGATE';
A_LETTER: 'A';
ALIAS: 'ALIAS';
ALL: 'ALL';
ALLOCATE: 'ALLOCATE';
ALLOW: 'ALLOW';
ALL_ROWS: 'ALL_ROWS';
ALTER: 'ALTER';
ALWAYS: 'ALWAYS';
ANALYZE: 'ANALYZE';
ANCILLARY: 'ANCILLARY';
AND: 'AND';
AND_EQUAL: 'AND_EQUAL';
ANOMALY: 'ANOMALY';
ANSI_REARCH: 'ANSI_REARCH';
ANTIJOIN: 'ANTIJOIN';
ANY: 'ANY';
ANYSCHEMA: 'ANYSCHEMA';
APPEND: 'APPEND';
APPENDCHILDXML: 'APPENDCHILDXML';
APPEND_VALUES: 'APPEND_VALUES';
APPLICATION: 'APPLICATION';
APPLY: 'APPLY';
APPROX_COUNT_DISTINCT: 'APPROX_COUNT_DISTINCT';
ARCHIVAL: 'ARCHIVAL';
ARCHIVE: 'ARCHIVE';
ARCHIVED: 'ARCHIVED';
ARCHIVELOG: 'ARCHIVELOG';
ARRAY: 'ARRAY';
AS: 'AS';
ASC: 'ASC';
ASCII: 'ASCII';
ASCIISTR: 'ASCIISTR';
ASIN: 'ASIN';
ASIS: 'ASIS';
ASSEMBLY: 'ASSEMBLY';
ASSIGN: 'ASSIGN';
ASSOCIATE: 'ASSOCIATE';
ASYNC: 'ASYNC';
ASYNCHRONOUS: 'ASYNCHRONOUS';
ATAN2: 'ATAN2';
ATAN: 'ATAN';
AT: 'AT';
ATTRIBUTE: 'ATTRIBUTE';
ATTRIBUTES: 'ATTRIBUTES';
AUDIT: 'AUDIT';
AUTHENTICATED: 'AUTHENTICATED';
AUTHENTICATION: 'AUTHENTICATION';
AUTHID: 'AUTHID';
AUTHORIZATION: 'AUTHORIZATION';
AUTOALLOCATE: 'AUTOALLOCATE';
AUTO: 'AUTO';
AUTOEXTEND: 'AUTOEXTEND';
AUTO_LOGIN: 'AUTO_LOGIN';
AUTOMATIC: 'AUTOMATIC';
AUTONOMOUS_TRANSACTION: 'AUTONOMOUS_TRANSACTION';
AUTO_REOPTIMIZE: 'AUTO_REOPTIMIZE';
AVAILABILITY: 'AVAILABILITY';
AVRO: 'AVRO';
BACKGROUND: 'BACKGROUND';
BACKUP: 'BACKUP';
BASIC: 'BASIC';
BASICFILE: 'BASICFILE';
BATCH: 'BATCH';
BATCHSIZE: 'BATCHSIZE';
BATCH_TABLE_ACCESS_BY_ROWID: 'BATCH_TABLE_ACCESS_BY_ROWID';
BECOME: 'BECOME';
BEFORE: 'BEFORE';
BEGIN: 'BEGIN';
BEGINNING: 'BEGINNING';
BEGIN_OUTLINE_DATA: 'BEGIN_OUTLINE_DATA';
BEHALF: 'BEHALF';
BEQUEATH: 'BEQUEATH';
BETWEEN: 'BETWEEN';
BFILE: 'BFILE';
BFILENAME: 'BFILENAME';
BIGFILE: 'BIGFILE';
BINARY: 'BINARY';
BINARY_DOUBLE: 'BINARY_DOUBLE';
BINARY_DOUBLE_INFINITY: 'BINARY_DOUBLE_INFINITY';
BINARY_DOUBLE_NAN: 'BINARY_DOUBLE_NAN';
BINARY_FLOAT: 'BINARY_FLOAT';
BINARY_FLOAT_INFINITY: 'BINARY_FLOAT_INFINITY';
BINARY_FLOAT_NAN: 'BINARY_FLOAT_NAN';
BINARY_INTEGER: 'BINARY_INTEGER';
BIND_AWARE: 'BIND_AWARE';
BINDING: 'BINDING';
BIN_TO_NUM: 'BIN_TO_NUM';
BITAND: 'BITAND';
BITMAP_AND: 'BITMAP_AND';
BITMAP: 'BITMAP';
BITMAPS: 'BITMAPS';
BITMAP_TREE: 'BITMAP_TREE';
BITS: 'BITS';
BLOB: 'BLOB';
BLOCK: 'BLOCK';
BLOCK_RANGE: 'BLOCK_RANGE';
BLOCKS: 'BLOCKS';
BLOCKSIZE: 'BLOCKSIZE';
BODY: 'BODY';
BOOLEAN: 'BOOLEAN';
BOTH: 'BOTH';
BOUND: 'BOUND';
BRANCH: 'BRANCH';
BREADTH: 'BREADTH';
BROADCAST: 'BROADCAST';
BSON: 'BSON';
BUFFER: 'BUFFER';
BUFFER_CACHE: 'BUFFER_CACHE';
BUFFER_POOL: 'BUFFER_POOL';
BUILD: 'BUILD';
BULK: 'BULK';
BY: 'BY';
BYPASS_RECURSIVE_CHECK: 'BYPASS_RECURSIVE_CHECK';
BYPASS_UJVC: 'BYPASS_UJVC';
BYTE: 'BYTE';
CACHE: 'CACHE';
CACHE_CB: 'CACHE_CB';
CACHE_INSTANCES: 'CACHE_INSTANCES';
CACHE_TEMP_TABLE: 'CACHE_TEMP_TABLE';
CACHING: 'CACHING';
CALCULATED: 'CALCULATED';
CALLBACK: 'CALLBACK';
CALL: 'CALL';
CANCEL: 'CANCEL';
CANONICAL: 'CANONICAL';
CAPACITY: 'CAPACITY';
CARDINALITY: 'CARDINALITY';
CASCADE: 'CASCADE';
CASE: 'CASE';
CAST: 'CAST';
CATEGORY: 'CATEGORY';
CDBDEFAULT: 'CDB$DEFAULT';
CEIL: 'CEIL';
CELL_FLASH_CACHE: 'CELL_FLASH_CACHE';
CERTIFICATE: 'CERTIFICATE';
CFILE: 'CFILE';
CHAINED: 'CHAINED';
CHANGE: 'CHANGE';
CHANGE_DUPKEY_ERROR_INDEX: 'CHANGE_DUPKEY_ERROR_INDEX';
CHARACTER: 'CHARACTER';
CHAR: 'CHAR';
CHAR_CS: 'CHAR_CS';
CHARTOROWID: 'CHARTOROWID';
CHECK_ACL_REWRITE: 'CHECK_ACL_REWRITE';
CHECK: 'CHECK';
CHECKPOINT: 'CHECKPOINT';
CHILD: 'CHILD';
CHOOSE: 'CHOOSE';
CHR: 'CHR';
CHUNK: 'CHUNK';
CLASS: 'CLASS';
CLASSIFIER: 'CLASSIFIER';
CLEANUP: 'CLEANUP';
CLEAR: 'CLEAR';
C_LETTER: 'C';
CLIENT: 'CLIENT';
CLOB: 'CLOB';
CLONE: 'CLONE';
CLOSE_CACHED_OPEN_CURSORS: 'CLOSE_CACHED_OPEN_CURSORS';
CLOSE: 'CLOSE';
CLUSTER_BY_ROWID: 'CLUSTER_BY_ROWID';
CLUSTER: 'CLUSTER';
CLUSTER_DETAILS: 'CLUSTER_DETAILS';
CLUSTER_DISTANCE: 'CLUSTER_DISTANCE';
CLUSTER_ID: 'CLUSTER_ID';
CLUSTERING: 'CLUSTERING';
CLUSTERING_FACTOR: 'CLUSTERING_FACTOR';
CLUSTER_PROBABILITY: 'CLUSTER_PROBABILITY';
CLUSTER_SET: 'CLUSTER_SET';
COALESCE: 'COALESCE';
COALESCE_SQ: 'COALESCE_SQ';
COARSE: 'COARSE';
CO_AUTH_IND: 'CO_AUTH_IND';
COLD: 'COLD';
COLLECT: 'COLLECT';
COLUMNAR: 'COLUMNAR';
COLUMN_AUTH_INDICATOR: 'COLUMN_AUTH_INDICATOR';
COLUMN: 'COLUMN';
COLUMNS: 'COLUMNS';
COLUMN_STATS: 'COLUMN_STATS';
COLUMN_VALUE: 'COLUMN_VALUE';
COMMENT: 'COMMENT';
COMMIT: 'COMMIT';
COMMITTED: 'COMMITTED';
COMMON_DATA: 'COMMON_DATA';
COMPACT: 'COMPACT';
COMPATIBILITY: 'COMPATIBILITY';
COMPILE: 'COMPILE';
COMPLETE: 'COMPLETE';
COMPLIANCE: 'COMPLIANCE';
COMPONENT: 'COMPONENT';
COMPONENTS: 'COMPONENTS';
COMPOSE: 'COMPOSE';
COMPOSITE: 'COMPOSITE';
COMPOSITE_LIMIT: 'COMPOSITE_LIMIT';
COMPOUND: 'COMPOUND';
COMPRESS: 'COMPRESS';
COMPUTE: 'COMPUTE';
CONCAT: 'CONCAT';
CON_DBID_TO_ID: 'CON_DBID_TO_ID';
CONDITIONAL: 'CONDITIONAL';
CONDITION: 'CONDITION';
CONFIRM: 'CONFIRM';
CONFORMING: 'CONFORMING';
CON_GUID_TO_ID: 'CON_GUID_TO_ID';
CON_ID: 'CON_ID';
CON_NAME_TO_ID: 'CON_NAME_TO_ID';
CONNECT_BY_CB_WHR_ONLY: 'CONNECT_BY_CB_WHR_ONLY';
CONNECT_BY_COMBINE_SW: 'CONNECT_BY_COMBINE_SW';
CONNECT_BY_COST_BASED: 'CONNECT_BY_COST_BASED';
CONNECT_BY_ELIM_DUPS: 'CONNECT_BY_ELIM_DUPS';
CONNECT_BY_FILTERING: 'CONNECT_BY_FILTERING';
CONNECT_BY_ISCYCLE: 'CONNECT_BY_ISCYCLE';
CONNECT_BY_ISLEAF: 'CONNECT_BY_ISLEAF';
CONNECT_BY_ROOT: 'CONNECT_BY_ROOT';
CONNECT: 'CONNECT';
CONNECT_TIME: 'CONNECT_TIME';
CONSIDER: 'CONSIDER';
CONSISTENT: 'CONSISTENT';
CONSTANT: 'CONSTANT';
CONST: 'CONST';
CONSTRAINT: 'CONSTRAINT';
CONSTRAINTS: 'CONSTRAINTS';
CONSTRUCTOR: 'CONSTRUCTOR';
CONTAINER: 'CONTAINER';
CONTAINER_DATA: 'CONTAINER_DATA';
CONTAINERS: 'CONTAINERS';
CONTENT: 'CONTENT';
CONTENTS: 'CONTENTS';
CONTEXT: 'CONTEXT';
CONTINUE: 'CONTINUE';
CONTROLFILE: 'CONTROLFILE';
CON_UID_TO_ID: 'CON_UID_TO_ID';
CONVERT: 'CONVERT';
COOKIE: 'COOKIE';
COPY: 'COPY';
CORR_K: 'CORR_K';
CORR_S: 'CORR_S';
CORRUPTION: 'CORRUPTION';
CORRUPT_XID_ALL: 'CORRUPT_XID_ALL';
CORRUPT_XID: 'CORRUPT_XID';
COS: 'COS';
COSH: 'COSH';
COST: 'COST';
COST_XML_QUERY_REWRITE: 'COST_XML_QUERY_REWRITE';
COUNT: 'COUNT';
COVAR_POP: 'COVAR_POP';
COVAR_SAMP: 'COVAR_SAMP';
CPU_COSTING: 'CPU_COSTING';
CPU_PER_CALL: 'CPU_PER_CALL';
CPU_PER_SESSION: 'CPU_PER_SESSION';
CRASH: 'CRASH';
CREATE: 'CREATE';
CREATE_FILE_DEST: 'CREATE_FILE_DEST';
CREATE_STORED_OUTLINES: 'CREATE_STORED_OUTLINES';
CREATION: 'CREATION';
CREDENTIAL: 'CREDENTIAL';
CRITICAL: 'CRITICAL';
CROSS: 'CROSS';
CROSSEDITION: 'CROSSEDITION';
CSCONVERT: 'CSCONVERT';
CUBE_AJ: 'CUBE_AJ';
CUBE: 'CUBE';
CUBE_GB: 'CUBE_GB';
CUBE_SJ: 'CUBE_SJ';
CUME_DISTM: 'CUME_DISTM';
CURRENT: 'CURRENT';
CURRENT_DATE: 'CURRENT_DATE';
CURRENT_SCHEMA: 'CURRENT_SCHEMA';
CURRENT_TIME: 'CURRENT_TIME';
CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP';
CURRENT_USER: 'CURRENT_USER';
CURRENTV: 'CURRENTV';
CURSOR: 'CURSOR';
CURSOR_SHARING_EXACT: 'CURSOR_SHARING_EXACT';
CURSOR_SPECIFIC_SEGMENT: 'CURSOR_SPECIFIC_SEGMENT';
CUSTOMDATUM: 'CUSTOMDATUM';
CV: 'CV';
CYCLE: 'CYCLE';
DANGLING: 'DANGLING';
DATABASE: 'DATABASE';
DATA: 'DATA';
DATAFILE: 'DATAFILE';
DATAFILES: 'DATAFILES';
DATAMOVEMENT: 'DATAMOVEMENT';
DATAOBJNO: 'DATAOBJNO';
DATAOBJ_TO_MAT_PARTITION: 'DATAOBJ_TO_MAT_PARTITION';
DATAOBJ_TO_PARTITION: 'DATAOBJ_TO_PARTITION';
DATAPUMP: 'DATAPUMP';
DATA_SECURITY_REWRITE_LIMIT: 'DATA_SECURITY_REWRITE_LIMIT';
DATE: 'DATE';
DATE_MODE: 'DATE_MODE';
DAY: 'DAY';
DAYS: 'DAYS';
DBA: 'DBA';
DBA_RECYCLEBIN: 'DBA_RECYCLEBIN';
DBMS_STATS: 'DBMS_STATS';
DB_ROLE_CHANGE: 'DB_ROLE_CHANGE';
DBTIMEZONE: 'DBTIMEZONE';
DB_UNIQUE_NAME: 'DB_UNIQUE_NAME';
DB_VERSION: 'DB_VERSION';
DDL: 'DDL';
DEALLOCATE: 'DEALLOCATE';
DEBUG: 'DEBUG';
DEBUGGER: 'DEBUGGER';
DEC: 'DEC';
DECIMAL: 'DECIMAL';
DECLARE: 'DECLARE';
DECOMPOSE: 'DECOMPOSE';
DECORRELATE: 'DECORRELATE';
DECR: 'DECR';
DECREMENT: 'DECREMENT';
DECRYPT: 'DECRYPT';
DEDUPLICATE: 'DEDUPLICATE';
DEFAULT: 'DEFAULT';
DEFAULTS: 'DEFAULTS';
DEFERRABLE: 'DEFERRABLE';
DEFERRED: 'DEFERRED';
DEFINED: 'DEFINED';
DEFINE: 'DEFINE';
DEFINER: 'DEFINER';
DEGREE: 'DEGREE';
DELAY: 'DELAY';
DELEGATE: 'DELEGATE';
DELETE_ALL: 'DELETE_ALL';
DELETE: 'DELETE';
DELETEXML: 'DELETEXML';
DEMAND: 'DEMAND';
DENSE_RANKM: 'DENSE_RANKM';
DEPENDENT: 'DEPENDENT';
DEPTH: 'DEPTH';
DEQUEUE: 'DEQUEUE';
DEREF: 'DEREF';
DEREF_NO_REWRITE: 'DEREF_NO_REWRITE';
DESC: 'DESC';
DESTROY: 'DESTROY';
DETACHED: 'DETACHED';
DETERMINES: 'DETERMINES';
DETERMINISTIC: 'DETERMINISTIC';
DICTIONARY: 'DICTIONARY';
DIMENSION: 'DIMENSION';
DIMENSIONS: 'DIMENSIONS';
DIRECT_LOAD: 'DIRECT_LOAD';
DIRECTORY: 'DIRECTORY';
DIRECT_PATH: 'DIRECT_PATH';
DISABLE_ALL: 'DISABLE_ALL';
DISABLE: 'DISABLE';
DISABLE_PARALLEL_DML: 'DISABLE_PARALLEL_DML';
DISABLE_PRESET: 'DISABLE_PRESET';
DISABLE_RPKE: 'DISABLE_RPKE';
DISALLOW: 'DISALLOW';
DISASSOCIATE: 'DISASSOCIATE';
DISCARD: 'DISCARD';
DISCONNECT: 'DISCONNECT';
DISK: 'DISK';
DISKGROUP: 'DISKGROUP';
DISKS: 'DISKS';
DISMOUNT: 'DISMOUNT';
DISTINCT: 'DISTINCT';
DISTINGUISHED: 'DISTINGUISHED';
DISTRIBUTED: 'DISTRIBUTED';
DISTRIBUTE: 'DISTRIBUTE';
DML: 'DML';
DML_UPDATE: 'DML_UPDATE';
DOCFIDELITY: 'DOCFIDELITY';
DOCUMENT: 'DOCUMENT';
DOMAIN_INDEX_FILTER: 'DOMAIN_INDEX_FILTER';
DOMAIN_INDEX_NO_SORT: 'DOMAIN_INDEX_NO_SORT';
DOMAIN_INDEX_SORT: 'DOMAIN_INDEX_SORT';
DOUBLE: 'DOUBLE';
DOWNGRADE: 'DOWNGRADE';
DRIVING_SITE: 'DRIVING_SITE';
DROP_COLUMN: 'DROP_COLUMN';
DROP: 'DROP';
DROP_GROUP: 'DROP_GROUP';
DSINTERVAL_UNCONSTRAINED: 'DSINTERVAL_UNCONSTRAINED';
DST_UPGRADE_INSERT_CONV: 'DST_UPGRADE_INSERT_CONV';
DUMP: 'DUMP';
DUPLICATE: 'DUPLICATE';
DV: 'DV';
DYNAMIC: 'DYNAMIC';
DYNAMIC_SAMPLING: 'DYNAMIC_SAMPLING';
DYNAMIC_SAMPLING_EST_CDN: 'DYNAMIC_SAMPLING_EST_CDN';
EACH: 'EACH';
EDITIONABLE: 'EDITIONABLE';
EDITION: 'EDITION';
EDITIONING: 'EDITIONING';
EDITIONS: 'EDITIONS';
ELEMENT: 'ELEMENT';
ELIM_GROUPBY: 'ELIM_GROUPBY';
ELIMINATE_JOIN: 'ELIMINATE_JOIN';
ELIMINATE_OBY: 'ELIMINATE_OBY';
ELIMINATE_OUTER_JOIN: 'ELIMINATE_OUTER_JOIN';
ELSE: 'ELSE';
ELSIF: 'ELSIF';
EM: 'EM';
EMPTY_BLOB: 'EMPTY_BLOB';
EMPTY_CLOB: 'EMPTY_CLOB';
EMPTY: 'EMPTY';
ENABLE_ALL: 'ENABLE_ALL';
ENABLE: 'ENABLE';
ENABLE_PARALLEL_DML: 'ENABLE_PARALLEL_DML';
ENABLE_PRESET: 'ENABLE_PRESET';
ENCODING: 'ENCODING';
ENCRYPT: 'ENCRYPT';
ENCRYPTION: 'ENCRYPTION';
END: 'END';
END_OUTLINE_DATA: 'END_OUTLINE_DATA';
ENFORCED: 'ENFORCED';
ENFORCE: 'ENFORCE';
ENQUEUE: 'ENQUEUE';
ENTERPRISE: 'ENTERPRISE';
ENTITYESCAPING: 'ENTITYESCAPING';
ENTRY: 'ENTRY';
EQUIPART: 'EQUIPART';
ERR: 'ERR';
ERROR_ARGUMENT: 'ERROR_ARGUMENT';
ERROR: 'ERROR';
ERROR_ON_OVERLAP_TIME: 'ERROR_ON_OVERLAP_TIME';
ERRORS: 'ERRORS';
ESCAPE: 'ESCAPE';
ESTIMATE: 'ESTIMATE';
EVAL: 'EVAL';
EVALNAME: 'EVALNAME';
EVALUATE: 'EVALUATE';
EVALUATION: 'EVALUATION';
EVENTS: 'EVENTS';
EVERY: 'EVERY';
EXCEPT: 'EXCEPT';
EXCEPTION: 'EXCEPTION';
EXCEPTION_INIT: 'EXCEPTION_INIT';
EXCEPTIONS: 'EXCEPTIONS';
EXCHANGE: 'EXCHANGE';
EXCLUDE: 'EXCLUDE';
EXCLUDING: 'EXCLUDING';
EXCLUSIVE: 'EXCLUSIVE';
EXECUTE: 'EXECUTE';
EXEMPT: 'EXEMPT';
EXISTING: 'EXISTING';
EXISTS: 'EXISTS';
EXISTSNODE: 'EXISTSNODE';
EXIT: 'EXIT';
EXPAND_GSET_TO_UNION: 'EXPAND_GSET_TO_UNION';
EXPAND_TABLE: 'EXPAND_TABLE';
EXP: 'EXP';
EXPIRE: 'EXPIRE';
EXPLAIN: 'EXPLAIN';
EXPLOSION: 'EXPLOSION';
EXPORT: 'EXPORT';
EXPR_CORR_CHECK: 'EXPR_CORR_CHECK';
EXPRESS: 'EXPRESS';
EXTENDS: 'EXTENDS';
EXTENT: 'EXTENT';
EXTENTS: 'EXTENTS';
EXTERNAL: 'EXTERNAL';
EXTERNALLY: 'EXTERNALLY';
EXTRACTCLOBXML: 'EXTRACTCLOBXML';
EXTRACT: 'EXTRACT';
EXTRACTVALUE: 'EXTRACTVALUE';
EXTRA: 'EXTRA';
FACILITY: 'FACILITY';
FACT: 'FACT';
FACTOR: 'FACTOR';
FACTORIZE_JOIN: 'FACTORIZE_JOIN';
FAILED: 'FAILED';
FAILED_LOGIN_ATTEMPTS: 'FAILED_LOGIN_ATTEMPTS';
FAILGROUP: 'FAILGROUP';
FAILOVER: 'FAILOVER';
FAILURE: 'FAILURE';
FALSE: 'FALSE';
FAMILY: 'FAMILY';
FAR: 'FAR';
FAST: 'FAST';
FASTSTART: 'FASTSTART';
FBTSCAN: 'FBTSCAN';
FEATURE_DETAILS: 'FEATURE_DETAILS';
FEATURE_ID: 'FEATURE_ID';
FEATURE_SET: 'FEATURE_SET';
FEATURE_VALUE: 'FEATURE_VALUE';
FETCH: 'FETCH';
FILE: 'FILE';
FILE_NAME_CONVERT: 'FILE_NAME_CONVERT';
FILESYSTEM_LIKE_LOGGING: 'FILESYSTEM_LIKE_LOGGING';
FILTER: 'FILTER';
FINAL: 'FINAL';
FINE: 'FINE';
FINISH: 'FINISH';
FIRST: 'FIRST';
FIRSTM: 'FIRSTM';
FIRST_ROWS: 'FIRST_ROWS';
FIRST_VALUE: 'FIRST_VALUE';
FIXED_VIEW_DATA: 'FIXED_VIEW_DATA';
FLAGGER: 'FLAGGER';
FLASHBACK: 'FLASHBACK';
FLASH_CACHE: 'FLASH_CACHE';
FLOAT: 'FLOAT';
FLOB: 'FLOB';
FLOOR: 'FLOOR';
FLUSH: 'FLUSH';
FOLDER: 'FOLDER';
FOLLOWING: 'FOLLOWING';
FOLLOWS: 'FOLLOWS';
FORALL: 'FORALL';
FORCE: 'FORCE';
FORCE_XML_QUERY_REWRITE: 'FORCE_XML_QUERY_REWRITE';
FOREIGN: 'FOREIGN';
FOREVER: 'FOREVER';
FOR: 'FOR';
FORMAT: 'FORMAT';
FORWARD: 'FORWARD';
FRAGMENT_NUMBER: 'FRAGMENT_NUMBER';
FREELIST: 'FREELIST';
FREELISTS: 'FREELISTS';
FREEPOOLS: 'FREEPOOLS';
FRESH: 'FRESH';
FROM: 'FROM';
FROM_TZ: 'FROM_TZ';
FULL: 'FULL';
FULL_OUTER_JOIN_TO_OUTER: 'FULL_OUTER_JOIN_TO_OUTER';
FUNCTION: 'FUNCTION';
FUNCTIONS: 'FUNCTIONS';
GATHER_OPTIMIZER_STATISTICS: 'GATHER_OPTIMIZER_STATISTICS';
GATHER_PLAN_STATISTICS: 'GATHER_PLAN_STATISTICS';
GBY_CONC_ROLLUP: 'GBY_CONC_ROLLUP';
GBY_PUSHDOWN: 'GBY_PUSHDOWN';
GENERATED: 'GENERATED';
GET: 'GET';
GLOBAL: 'GLOBAL';
GLOBALLY: 'GLOBALLY';
GLOBAL_NAME: 'GLOBAL_NAME';
GLOBAL_TOPIC_ENABLED: 'GLOBAL_TOPIC_ENABLED';
GOTO: 'GOTO';
GRANT: 'GRANT';
GROUP_BY: 'GROUP_BY';
GROUP: 'GROUP';
GROUP_ID: 'GROUP_ID';
GROUPING: 'GROUPING';
GROUPING_ID: 'GROUPING_ID';
GROUPS: 'GROUPS';
GUARANTEED: 'GUARANTEED';
GUARANTEE: 'GUARANTEE';
GUARD: 'GUARD';
HASH_AJ: 'HASH_AJ';
HASH: 'HASH';
HASHKEYS: 'HASHKEYS';
HASH_SJ: 'HASH_SJ';
HAVING: 'HAVING';
HEADER: 'HEADER';
HEAP: 'HEAP';
HELP: 'HELP';
HEXTORAW: 'HEXTORAW';
HEXTOREF: 'HEXTOREF';
HIDDEN_KEYWORD: 'HIDDEN';
HIDE: 'HIDE';
HIERARCHY: 'HIERARCHY';
HIGH: 'HIGH';
HINTSET_BEGIN: 'HINTSET_BEGIN';
HINTSET_END: 'HINTSET_END';
HOT: 'HOT';
HOUR: 'HOUR';
HWM_BROKERED: 'HWM_BROKERED';
HYBRID: 'HYBRID';
IDENTIFIED: 'IDENTIFIED';
IDENTIFIER: 'IDENTIFIER';
IDENTITY: 'IDENTITY';
IDGENERATORS: 'IDGENERATORS';
ID: 'ID';
IDLE_TIME: 'IDLE_TIME';
IF: 'IF';
IGNORE: 'IGNORE';
IGNORE_OPTIM_EMBEDDED_HINTS: 'IGNORE_OPTIM_EMBEDDED_HINTS';
IGNORE_ROW_ON_DUPKEY_INDEX: 'IGNORE_ROW_ON_DUPKEY_INDEX';
IGNORE_WHERE_CLAUSE: 'IGNORE_WHERE_CLAUSE';
ILM: 'ILM';
IMMEDIATE: 'IMMEDIATE';
IMPACT: 'IMPACT';
IMPORT: 'IMPORT';
INACTIVE: 'INACTIVE';
INCLUDE: 'INCLUDE';
INCLUDE_VERSION: 'INCLUDE_VERSION';
INCLUDING: 'INCLUDING';
INCREMENTAL: 'INCREMENTAL';
INCREMENT: 'INCREMENT';
INCR: 'INCR';
INDENT: 'INDENT';
INDEX_ASC: 'INDEX_ASC';
INDEX_COMBINE: 'INDEX_COMBINE';
INDEX_DESC: 'INDEX_DESC';
INDEXED: 'INDEXED';
INDEXES: 'INDEXES';
INDEX_FFS: 'INDEX_FFS';
INDEX_FILTER: 'INDEX_FILTER';
INDEX: 'INDEX';
INDEXING: 'INDEXING';
INDEX_JOIN: 'INDEX_JOIN';
INDEX_ROWS: 'INDEX_ROWS';
INDEX_RRS: 'INDEX_RRS';
INDEX_RS_ASC: 'INDEX_RS_ASC';
INDEX_RS_DESC: 'INDEX_RS_DESC';
INDEX_RS: 'INDEX_RS';
INDEX_SCAN: 'INDEX_SCAN';
INDEX_SKIP_SCAN: 'INDEX_SKIP_SCAN';
INDEX_SS_ASC: 'INDEX_SS_ASC';
INDEX_SS_DESC: 'INDEX_SS_DESC';
INDEX_SS: 'INDEX_SS';
INDEX_STATS: 'INDEX_STATS';
INDEXTYPE: 'INDEXTYPE';
INDEXTYPES: 'INDEXTYPES';
INDICATOR: 'INDICATOR';
INDICES: 'INDICES';
INFINITE: 'INFINITE';
INFORMATIONAL: 'INFORMATIONAL';
INHERIT: 'INHERIT';
IN: 'IN';
INITCAP: 'INITCAP';
INITIAL: 'INITIAL';
INITIALIZED: 'INITIALIZED';
INITIALLY: 'INITIALLY';
INITRANS: 'INITRANS';
INLINE: 'INLINE';
INLINE_XMLTYPE_NT: 'INLINE_XMLTYPE_NT';
INMEMORY: 'INMEMORY';
IN_MEMORY_METADATA: 'IN_MEMORY_METADATA';
INMEMORY_PRUNING: 'INMEMORY_PRUNING';
INNER: 'INNER';
INOUT: 'INOUT';
INPLACE: 'INPLACE';
INSERTCHILDXMLAFTER: 'INSERTCHILDXMLAFTER';
INSERTCHILDXMLBEFORE: 'INSERTCHILDXMLBEFORE';
INSERTCHILDXML: 'INSERTCHILDXML';
INSERT: 'INSERT';
INSERTXMLAFTER: 'INSERTXMLAFTER';
INSERTXMLBEFORE: 'INSERTXMLBEFORE';
INSTANCE: 'INSTANCE';
INSTANCES: 'INSTANCES';
INSTANTIABLE: 'INSTANTIABLE';
INSTANTLY: 'INSTANTLY';
INSTEAD: 'INSTEAD';
INSTR2: 'INSTR2';
INSTR4: 'INSTR4';
INSTRB: 'INSTRB';
INSTRC: 'INSTRC';
INSTR: 'INSTR';
INTEGER: 'INTEGER';
INTERLEAVED: 'INTERLEAVED';
INTERMEDIATE: 'INTERMEDIATE';
INTERNAL_CONVERT: 'INTERNAL_CONVERT';
INTERNAL_USE: 'INTERNAL_USE';
INTERPRETED: 'INTERPRETED';
INTERSECT: 'INTERSECT';
INTERVAL: 'INTERVAL';
INT: 'INT';
INTO: 'INTO';
INVALIDATE: 'INVALIDATE';
INVISIBLE: 'INVISIBLE';
IN_XQUERY: 'IN_XQUERY';
IS: 'IS';
ISOLATION: 'ISOLATION';
ISOLATION_LEVEL: 'ISOLATION_LEVEL';
ITERATE: 'ITERATE';
ITERATION_NUMBER: 'ITERATION_NUMBER';
JAVA: 'JAVA';
JOB: 'JOB';
JOIN: 'JOIN';
JSON_ARRAYAGG: 'JSON_ARRAYAGG';
JSON_ARRAY: 'JSON_ARRAY';
JSON_EQUAL: 'JSON_EQUAL';
JSON_EXISTS2: 'JSON_EXISTS2';
JSON_EXISTS: 'JSON_EXISTS';
JSONGET: 'JSONGET';
JSON: 'JSON';
JSON_OBJECTAGG: 'JSON_OBJECTAGG';
JSON_OBJECT: 'JSON_OBJECT';
JSONPARSE: 'JSONPARSE';
JSON_QUERY: 'JSON_QUERY';
JSON_SERIALIZE: 'JSON_SERIALIZE';
JSON_TABLE: 'JSON_TABLE';
JSON_TEXTCONTAINS2: 'JSON_TEXTCONTAINS2';
JSON_TEXTCONTAINS: 'JSON_TEXTCONTAINS';
JSON_VALUE: 'JSON_VALUE';
KEEP_DUPLICATES: 'KEEP_DUPLICATES';
KEEP: 'KEEP';
KERBEROS: 'KERBEROS';
KEY: 'KEY';
KEY_LENGTH: 'KEY_LENGTH';
KEYSIZE: 'KEYSIZE';
KEYS: 'KEYS';
KEYSTORE: 'KEYSTORE';
KILL: 'KILL';
LABEL: 'LABEL';
LANGUAGE: 'LANGUAGE';
LAST_DAY: 'LAST_DAY';
LAST: 'LAST';
LAST_VALUE: 'LAST_VALUE';
LATERAL: 'LATERAL';
LAX: 'LAX';
LAYER: 'LAYER';
LDAP_REGISTRATION_ENABLED: 'LDAP_REGISTRATION_ENABLED';
LDAP_REGISTRATION: 'LDAP_REGISTRATION';
LDAP_REG_SYNC_INTERVAL: 'LDAP_REG_SYNC_INTERVAL';
LEADING: 'LEADING';
LEFT: 'LEFT';
LENGTH2: 'LENGTH2';
LENGTH4: 'LENGTH4';
LENGTHB: 'LENGTHB';
LENGTHC: 'LENGTHC';
LENGTH: 'LENGTH';
LESS: 'LESS';
LEVEL: 'LEVEL';
LEVELS: 'LEVELS';
LIBRARY: 'LIBRARY';
LIFECYCLE: 'LIFECYCLE';
LIFE: 'LIFE';
LIFETIME: 'LIFETIME';
LIKE2: 'LIKE2';
LIKE4: 'LIKE4';
LIKEC: 'LIKEC';
LIKE_EXPAND: 'LIKE_EXPAND';
LIKE: 'LIKE';
LIMIT: 'LIMIT';
LINEAR: 'LINEAR';
LINK: 'LINK';
LIST: 'LIST';
LN: 'LN';
LNNVL: 'LNNVL';
LOAD: 'LOAD';
LOB: 'LOB';
LOBNVL: 'LOBNVL';
LOBS: 'LOBS';
LOCAL_INDEXES: 'LOCAL_INDEXES';
LOCAL: 'LOCAL';
LOCALTIME: 'LOCALTIME';
LOCALTIMESTAMP: 'LOCALTIMESTAMP';
LOCATION: 'LOCATION';
LOCATOR: 'LOCATOR';
LOCKED: 'LOCKED';
LOCKING: 'LOCKING';
LOCK: 'LOCK';
LOGFILE: 'LOGFILE';
LOGFILES: 'LOGFILES';
LOGGING: 'LOGGING';
LOGICAL: 'LOGICAL';
LOGICAL_READS_PER_CALL: 'LOGICAL_READS_PER_CALL';
LOGICAL_READS_PER_SESSION: 'LOGICAL_READS_PER_SESSION';
LOG: 'LOG';
LOGMINING: 'LOGMINING';
LOGOFF: 'LOGOFF';
LOGON: 'LOGON';
LOG_READ_ONLY_VIOLATIONS: 'LOG_READ_ONLY_VIOLATIONS';
LONG: 'LONG';
LOOP: 'LOOP';
LOWER: 'LOWER';
LOW: 'LOW';
LPAD: 'LPAD';
LTRIM: 'LTRIM';
MAIN: 'MAIN';
MAKE_REF: 'MAKE_REF';
MANAGED: 'MANAGED';
MANAGE: 'MANAGE';
MANAGEMENT: 'MANAGEMENT';
MANAGER: 'MANAGER';
MANUAL: 'MANUAL';
MAP: 'MAP';
MAPPING: 'MAPPING';
MASTER: 'MASTER';
MATCHED: 'MATCHED';
MATCHES: 'MATCHES';
MATCH: 'MATCH';
MATCH_NUMBER: 'MATCH_NUMBER';
MATCH_RECOGNIZE: 'MATCH_RECOGNIZE';
MATERIALIZED: 'MATERIALIZED';
MATERIALIZE: 'MATERIALIZE';
MAXARCHLOGS: 'MAXARCHLOGS';
MAXDATAFILES: 'MAXDATAFILES';
MAXEXTENTS: 'MAXEXTENTS';
MAXIMIZE: 'MAXIMIZE';
MAXINSTANCES: 'MAXINSTANCES';
MAXLOGFILES: 'MAXLOGFILES';
MAXLOGHISTORY: 'MAXLOGHISTORY';
MAXLOGMEMBERS: 'MAXLOGMEMBERS';
MAX_SHARED_TEMP_SIZE: 'MAX_SHARED_TEMP_SIZE';
MAXSIZE: 'MAXSIZE';
MAXTRANS: 'MAXTRANS';
MAXVALUE: 'MAXVALUE';
MEASURE: 'MEASURE';
MEASURES: 'MEASURES';
MEDIUM: 'MEDIUM';
MEMBER: 'MEMBER';
MEMCOMPRESS: 'MEMCOMPRESS';
MEMORY: 'MEMORY';
MERGEACTIONS: 'MERGE$ACTIONS';
MERGE_AJ: 'MERGE_AJ';
MERGE_CONST_ON: 'MERGE_CONST_ON';
MERGE: 'MERGE';
MERGE_SJ: 'MERGE_SJ';
METADATA: 'METADATA';
METHOD: 'METHOD';
MIGRATE: 'MIGRATE';
MIGRATION: 'MIGRATION';
MINEXTENTS: 'MINEXTENTS';
MINIMIZE: 'MINIMIZE';
MINIMUM: 'MINIMUM';
MINING: 'MINING';
MINUS: 'MINUS';
MINUS_NULL: 'MINUS_NULL';
MINUTE: 'MINUTE';
MINVALUE: 'MINVALUE';
MIRRORCOLD: 'MIRRORCOLD';
MIRRORHOT: 'MIRRORHOT';
MIRROR: 'MIRROR';
MLSLABEL: 'MLSLABEL';
MODEL_COMPILE_SUBQUERY: 'MODEL_COMPILE_SUBQUERY';
MODEL_DONTVERIFY_UNIQUENESS: 'MODEL_DONTVERIFY_UNIQUENESS';
MODEL_DYNAMIC_SUBQUERY: 'MODEL_DYNAMIC_SUBQUERY';
MODEL_MIN_ANALYSIS: 'MODEL_MIN_ANALYSIS';
MODEL: 'MODEL';
MODEL_NB: 'MODEL_NB';
MODEL_NO_ANALYSIS: 'MODEL_NO_ANALYSIS';
MODEL_PBY: 'MODEL_PBY';
MODEL_PUSH_REF: 'MODEL_PUSH_REF';
MODEL_SV: 'MODEL_SV';
MODE: 'MODE';
MODIFICATION: 'MODIFICATION';
MODIFY_COLUMN_TYPE: 'MODIFY_COLUMN_TYPE';
MODIFY: 'MODIFY';
MOD: 'MOD';
MODULE: 'MODULE';
MONITORING: 'MONITORING';
MONITOR: 'MONITOR';
MONTH: 'MONTH';
MONTHS_BETWEEN: 'MONTHS_BETWEEN';
MONTHS: 'MONTHS';
MOUNT: 'MOUNT';
MOUNTPATH: 'MOUNTPATH';
MOVEMENT: 'MOVEMENT';
MOVE: 'MOVE';
MULTIDIMENSIONAL: 'MULTIDIMENSIONAL';
MULTISET: 'MULTISET';
MV_MERGE: 'MV_MERGE';
NAMED: 'NAMED';
NAME: 'NAME';
NAMESPACE: 'NAMESPACE';
NAN: 'NAN';
NANVL: 'NANVL';
NATIONAL: 'NATIONAL';
NATIVE_FULL_OUTER_JOIN: 'NATIVE_FULL_OUTER_JOIN';
NATIVE: 'NATIVE';
NATURAL: 'NATURAL';
NATURALN: 'NATURALN';
NAV: 'NAV';
NCHAR_CS: 'NCHAR_CS';
NCHAR: 'NCHAR';
NCHR: 'NCHR';
NCLOB: 'NCLOB';
NEEDED: 'NEEDED';
NEG: 'NEG';
NESTED: 'NESTED';
NESTED_TABLE_FAST_INSERT: 'NESTED_TABLE_FAST_INSERT';
NESTED_TABLE_GET_REFS: 'NESTED_TABLE_GET_REFS';
NESTED_TABLE_ID: 'NESTED_TABLE_ID';
NESTED_TABLE_SET_REFS: 'NESTED_TABLE_SET_REFS';
NESTED_TABLE_SET_SETID: 'NESTED_TABLE_SET_SETID';
NETWORK: 'NETWORK';
NEVER: 'NEVER';
NEW: 'NEW';
NEW_TIME: 'NEW_TIME';
NEXT_DAY: 'NEXT_DAY';
NEXT: 'NEXT';
NL_AJ: 'NL_AJ';
NLJ_BATCHING: 'NLJ_BATCHING';
NLJ_INDEX_FILTER: 'NLJ_INDEX_FILTER';
NLJ_INDEX_SCAN: 'NLJ_INDEX_SCAN';
NLJ_PREFETCH: 'NLJ_PREFETCH';
NLS_CALENDAR: 'NLS_CALENDAR';
NLS_CHARACTERSET: 'NLS_CHARACTERSET';
NLS_CHARSET_DECL_LEN: 'NLS_CHARSET_DECL_LEN';
NLS_CHARSET_ID: 'NLS_CHARSET_ID';
NLS_CHARSET_NAME: 'NLS_CHARSET_NAME';
NLS_COMP: 'NLS_COMP';
NLS_CURRENCY: 'NLS_CURRENCY';
NLS_DATE_FORMAT: 'NLS_DATE_FORMAT';
NLS_DATE_LANGUAGE: 'NLS_DATE_LANGUAGE';
NLS_INITCAP: 'NLS_INITCAP';
NLS_ISO_CURRENCY: 'NLS_ISO_CURRENCY';
NL_SJ: 'NL_SJ';
NLS_LANG: 'NLS_LANG';
NLS_LANGUAGE: 'NLS_LANGUAGE';
NLS_LENGTH_SEMANTICS: 'NLS_LENGTH_SEMANTICS';
NLS_LOWER: 'NLS_LOWER';
NLS_NCHAR_CONV_EXCP: 'NLS_NCHAR_CONV_EXCP';
NLS_NUMERIC_CHARACTERS: 'NLS_NUMERIC_CHARACTERS';
NLS_SORT: 'NLS_SORT';
NLSSORT: 'NLSSORT';
NLS_SPECIAL_CHARS: 'NLS_SPECIAL_CHARS';
NLS_TERRITORY: 'NLS_TERRITORY';
NLS_UPPER: 'NLS_UPPER';
NO_ACCESS: 'NO_ACCESS';
NO_ADAPTIVE_PLAN: 'NO_ADAPTIVE_PLAN';
NO_ANSI_REARCH: 'NO_ANSI_REARCH';
NOAPPEND: 'NOAPPEND';
NOARCHIVELOG: 'NOARCHIVELOG';
NOAUDIT: 'NOAUDIT';
NO_AUTO_REOPTIMIZE: 'NO_AUTO_REOPTIMIZE';
NO_BASETABLE_MULTIMV_REWRITE: 'NO_BASETABLE_MULTIMV_REWRITE';
NO_BATCH_TABLE_ACCESS_BY_ROWID: 'NO_BATCH_TABLE_ACCESS_BY_ROWID';
NO_BIND_AWARE: 'NO_BIND_AWARE';
NO_BUFFER: 'NO_BUFFER';
NOCACHE: 'NOCACHE';
NO_CARTESIAN: 'NO_CARTESIAN';
NO_CHECK_ACL_REWRITE: 'NO_CHECK_ACL_REWRITE';
NO_CLUSTER_BY_ROWID: 'NO_CLUSTER_BY_ROWID';
NO_CLUSTERING: 'NO_CLUSTERING';
NO_COALESCE_SQ: 'NO_COALESCE_SQ';
NO_COMMON_DATA: 'NO_COMMON_DATA';
NOCOMPRESS: 'NOCOMPRESS';
NO_CONNECT_BY_CB_WHR_ONLY: 'NO_CONNECT_BY_CB_WHR_ONLY';
NO_CONNECT_BY_COMBINE_SW: 'NO_CONNECT_BY_COMBINE_SW';
NO_CONNECT_BY_COST_BASED: 'NO_CONNECT_BY_COST_BASED';
NO_CONNECT_BY_ELIM_DUPS: 'NO_CONNECT_BY_ELIM_DUPS';
NO_CONNECT_BY_FILTERING: 'NO_CONNECT_BY_FILTERING';
NOCOPY: 'NOCOPY';
NO_COST_XML_QUERY_REWRITE: 'NO_COST_XML_QUERY_REWRITE';
NO_CPU_COSTING: 'NO_CPU_COSTING';
NOCPU_COSTING: 'NOCPU_COSTING';
NOCYCLE: 'NOCYCLE';
NO_DATA_SECURITY_REWRITE: 'NO_DATA_SECURITY_REWRITE';
NO_DECORRELATE: 'NO_DECORRELATE';
NODELAY: 'NODELAY';
NO_DOMAIN_INDEX_FILTER: 'NO_DOMAIN_INDEX_FILTER';
NO_DST_UPGRADE_INSERT_CONV: 'NO_DST_UPGRADE_INSERT_CONV';
NO_ELIM_GROUPBY: 'NO_ELIM_GROUPBY';
NO_ELIMINATE_JOIN: 'NO_ELIMINATE_JOIN';
NO_ELIMINATE_OBY: 'NO_ELIMINATE_OBY';
NO_ELIMINATE_OUTER_JOIN: 'NO_ELIMINATE_OUTER_JOIN';
NOENTITYESCAPING: 'NOENTITYESCAPING';
NO_EXPAND_GSET_TO_UNION: 'NO_EXPAND_GSET_TO_UNION';
NO_EXPAND: 'NO_EXPAND';
NO_EXPAND_TABLE: 'NO_EXPAND_TABLE';
NO_FACT: 'NO_FACT';
NO_FACTORIZE_JOIN: 'NO_FACTORIZE_JOIN';
NO_FILTERING: 'NO_FILTERING';
NOFORCE: 'NOFORCE';
NO_FULL_OUTER_JOIN_TO_OUTER: 'NO_FULL_OUTER_JOIN_TO_OUTER';
NO_GATHER_OPTIMIZER_STATISTICS: 'NO_GATHER_OPTIMIZER_STATISTICS';
NO_GBY_PUSHDOWN: 'NO_GBY_PUSHDOWN';
NOGUARANTEE: 'NOGUARANTEE';
NO_INDEX_FFS: 'NO_INDEX_FFS';
NO_INDEX: 'NO_INDEX';
NO_INDEX_SS: 'NO_INDEX_SS';
NO_INMEMORY: 'NO_INMEMORY';
NO_INMEMORY_PRUNING: 'NO_INMEMORY_PRUNING';
NOKEEP: 'NOKEEP';
NO_LOAD: 'NO_LOAD';
NOLOCAL: 'NOLOCAL';
NOLOGGING: 'NOLOGGING';
NOMAPPING: 'NOMAPPING';
NOMAXVALUE: 'NOMAXVALUE';
NO_MERGE: 'NO_MERGE';
NOMINIMIZE: 'NOMINIMIZE';
NOMINVALUE: 'NOMINVALUE';
NO_MODEL_PUSH_REF: 'NO_MODEL_PUSH_REF';
NO_MONITORING: 'NO_MONITORING';
NOMONITORING: 'NOMONITORING';
NO_MONITOR: 'NO_MONITOR';
NO_MULTIMV_REWRITE: 'NO_MULTIMV_REWRITE';
NO_NATIVE_FULL_OUTER_JOIN: 'NO_NATIVE_FULL_OUTER_JOIN';
NONBLOCKING: 'NONBLOCKING';
NONEDITIONABLE: 'NONEDITIONABLE';
NONE: 'NONE';
NO_NLJ_BATCHING: 'NO_NLJ_BATCHING';
NO_NLJ_PREFETCH: 'NO_NLJ_PREFETCH';
NO: 'NO';
NONSCHEMA: 'NONSCHEMA';
NO_OBJECT_LINK: 'NO_OBJECT_LINK';
NOORDER: 'NOORDER';
NO_ORDER_ROLLUPS: 'NO_ORDER_ROLLUPS';
NO_OUTER_JOIN_TO_ANTI: 'NO_OUTER_JOIN_TO_ANTI';
NO_OUTER_JOIN_TO_INNER: 'NO_OUTER_JOIN_TO_INNER';
NOOVERRIDE: 'NOOVERRIDE';
NO_PARALLEL_INDEX: 'NO_PARALLEL_INDEX';
NOPARALLEL_INDEX: 'NOPARALLEL_INDEX';
NO_PARALLEL: 'NO_PARALLEL';
NOPARALLEL: 'NOPARALLEL';
NO_PARTIAL_COMMIT: 'NO_PARTIAL_COMMIT';
NO_PARTIAL_JOIN: 'NO_PARTIAL_JOIN';
NO_PARTIAL_ROLLUP_PUSHDOWN: 'NO_PARTIAL_ROLLUP_PUSHDOWN';
NOPARTITION: 'NOPARTITION';
NO_PLACE_DISTINCT: 'NO_PLACE_DISTINCT';
NO_PLACE_GROUP_BY: 'NO_PLACE_GROUP_BY';
NO_PQ_CONCURRENT_UNION: 'NO_PQ_CONCURRENT_UNION';
NO_PQ_MAP: 'NO_PQ_MAP';
NO_PQ_REPLICATE: 'NO_PQ_REPLICATE';
NO_PQ_SKEW: 'NO_PQ_SKEW';
NO_PRUNE_GSETS: 'NO_PRUNE_GSETS';
NO_PULL_PRED: 'NO_PULL_PRED';
NO_PUSH_PRED: 'NO_PUSH_PRED';
NO_PUSH_SUBQ: 'NO_PUSH_SUBQ';
NO_PX_FAULT_TOLERANCE: 'NO_PX_FAULT_TOLERANCE';
NO_PX_JOIN_FILTER: 'NO_PX_JOIN_FILTER';
NO_QKN_BUFF: 'NO_QKN_BUFF';
NO_QUERY_TRANSFORMATION: 'NO_QUERY_TRANSFORMATION';
NO_REF_CASCADE: 'NO_REF_CASCADE';
NORELOCATE: 'NORELOCATE';
NORELY: 'NORELY';
NOREPAIR: 'NOREPAIR';
NOREPLAY: 'NOREPLAY';
NORESETLOGS: 'NORESETLOGS';
NO_RESULT_CACHE: 'NO_RESULT_CACHE';
NOREVERSE: 'NOREVERSE';
NO_REWRITE: 'NO_REWRITE';
NOREWRITE: 'NOREWRITE';
NORMAL: 'NORMAL';
NO_ROOT_SW_FOR_LOCAL: 'NO_ROOT_SW_FOR_LOCAL';
NOROWDEPENDENCIES: 'NOROWDEPENDENCIES';
NOSCHEMACHECK: 'NOSCHEMACHECK';
NOSEGMENT: 'NOSEGMENT';
NO_SEMIJOIN: 'NO_SEMIJOIN';
NO_SEMI_TO_INNER: 'NO_SEMI_TO_INNER';
NO_SET_TO_JOIN: 'NO_SET_TO_JOIN';
NOSORT: 'NOSORT';
NO_SQL_TRANSLATION: 'NO_SQL_TRANSLATION';
NO_SQL_TUNE: 'NO_SQL_TUNE';
NO_STAR_TRANSFORMATION: 'NO_STAR_TRANSFORMATION';
NO_STATEMENT_QUEUING: 'NO_STATEMENT_QUEUING';
NO_STATS_GSETS: 'NO_STATS_GSETS';
NOSTRICT: 'NOSTRICT';
NO_SUBQUERY_PRUNING: 'NO_SUBQUERY_PRUNING';
NO_SUBSTRB_PAD: 'NO_SUBSTRB_PAD';
NO_SWAP_JOIN_INPUTS: 'NO_SWAP_JOIN_INPUTS';
NOSWITCH: 'NOSWITCH';
NO_TABLE_LOOKUP_BY_NL: 'NO_TABLE_LOOKUP_BY_NL';
NO_TEMP_TABLE: 'NO_TEMP_TABLE';
NOTHING: 'NOTHING';
NOTIFICATION: 'NOTIFICATION';
NOT: 'NOT';
NO_TRANSFORM_DISTINCT_AGG: 'NO_TRANSFORM_DISTINCT_AGG';
NO_UNNEST: 'NO_UNNEST';
NO_USE_CUBE: 'NO_USE_CUBE';
NO_USE_HASH_AGGREGATION: 'NO_USE_HASH_AGGREGATION';
NO_USE_HASH_GBY_FOR_PUSHDOWN: 'NO_USE_HASH_GBY_FOR_PUSHDOWN';
NO_USE_HASH: 'NO_USE_HASH';
NO_USE_INVISIBLE_INDEXES: 'NO_USE_INVISIBLE_INDEXES';
NO_USE_MERGE: 'NO_USE_MERGE';
NO_USE_NL: 'NO_USE_NL';
NO_USE_VECTOR_AGGREGATION: 'NO_USE_VECTOR_AGGREGATION';
NOVALIDATE: 'NOVALIDATE';
NO_VECTOR_TRANSFORM_DIMS: 'NO_VECTOR_TRANSFORM_DIMS';
NO_VECTOR_TRANSFORM_FACT: 'NO_VECTOR_TRANSFORM_FACT';
NO_VECTOR_TRANSFORM: 'NO_VECTOR_TRANSFORM';
NOWAIT: 'NOWAIT';
NO_XDB_FASTPATH_INSERT: 'NO_XDB_FASTPATH_INSERT';
NO_XML_DML_REWRITE: 'NO_XML_DML_REWRITE';
NO_XMLINDEX_REWRITE_IN_SELECT: 'NO_XMLINDEX_REWRITE_IN_SELECT';
NO_XMLINDEX_REWRITE: 'NO_XMLINDEX_REWRITE';
NO_XML_QUERY_REWRITE: 'NO_XML_QUERY_REWRITE';
NO_ZONEMAP: 'NO_ZONEMAP';
NTH_VALUE: 'NTH_VALUE';
NULLIF: 'NULLIF';
NULL: 'NULL';
NULLS: 'NULLS';
NUMBER: 'NUMBER';
NUMERIC: 'NUMERIC';
NUM_INDEX_KEYS: 'NUM_INDEX_KEYS';
NUMTODSINTERVAL: 'NUMTODSINTERVAL';
NUMTOYMINTERVAL: 'NUMTOYMINTERVAL';
NVARCHAR2: 'NVARCHAR2';
NVL2: 'NVL2';
OBJECT2XML: 'OBJECT2XML';
OBJECT: 'OBJECT';
OBJ_ID: 'OBJ_ID';
OBJNO: 'OBJNO';
OBJNO_REUSE: 'OBJNO_REUSE';
OCCURENCES: 'OCCURENCES';
OFFLINE: 'OFFLINE';
OFF: 'OFF';
OFFSET: 'OFFSET';
OF: 'OF';
OIDINDEX: 'OIDINDEX';
OID: 'OID';
OLAP: 'OLAP';
OLD: 'OLD';
OLD_PUSH_PRED: 'OLD_PUSH_PRED';
OLS: 'OLS';
OLTP: 'OLTP';
OMIT: 'OMIT';
ONE: 'ONE';
ONLINE: 'ONLINE';
ONLY: 'ONLY';
ON: 'ON';
OPAQUE: 'OPAQUE';
OPAQUE_TRANSFORM: 'OPAQUE_TRANSFORM';
OPAQUE_XCANONICAL: 'OPAQUE_XCANONICAL';
OPCODE: 'OPCODE';
OPEN: 'OPEN';
OPERATIONS: 'OPERATIONS';
OPERATOR: 'OPERATOR';
OPT_ESTIMATE: 'OPT_ESTIMATE';
OPTIMAL: 'OPTIMAL';
OPTIMIZE: 'OPTIMIZE';
OPTIMIZER_FEATURES_ENABLE: 'OPTIMIZER_FEATURES_ENABLE';
OPTIMIZER_GOAL: 'OPTIMIZER_GOAL';
OPTION: 'OPTION';
OPT_PARAM: 'OPT_PARAM';
ORA_BRANCH: 'ORA_BRANCH';
ORA_CHECK_ACL: 'ORA_CHECK_ACL';
ORA_CHECK_PRIVILEGE: 'ORA_CHECK_PRIVILEGE';
ORA_CLUSTERING: 'ORA_CLUSTERING';
ORADATA: 'ORADATA';
ORADEBUG: 'ORADEBUG';
ORA_DST_AFFECTED: 'ORA_DST_AFFECTED';
ORA_DST_CONVERT: 'ORA_DST_CONVERT';
ORA_DST_ERROR: 'ORA_DST_ERROR';
ORA_GET_ACLIDS: 'ORA_GET_ACLIDS';
ORA_GET_PRIVILEGES: 'ORA_GET_PRIVILEGES';
ORA_HASH: 'ORA_HASH';
ORA_INVOKING_USERID: 'ORA_INVOKING_USERID';
ORA_INVOKING_USER: 'ORA_INVOKING_USER';
ORA_INVOKING_XS_USER_GUID: 'ORA_INVOKING_XS_USER_GUID';
ORA_INVOKING_XS_USER: 'ORA_INVOKING_XS_USER';
ORA_RAWCOMPARE: 'ORA_RAWCOMPARE';
ORA_RAWCONCAT: 'ORA_RAWCONCAT';
ORA_ROWSCN: 'ORA_ROWSCN';
ORA_ROWSCN_RAW: 'ORA_ROWSCN_RAW';
ORA_ROWVERSION: 'ORA_ROWVERSION';
ORA_TABVERSION: 'ORA_TABVERSION';
ORA_WRITE_TIME: 'ORA_WRITE_TIME';
ORDERED: 'ORDERED';
ORDERED_PREDICATES: 'ORDERED_PREDICATES';
ORDER: 'ORDER';
ORDINALITY: 'ORDINALITY';
OR_EXPAND: 'OR_EXPAND';
ORGANIZATION: 'ORGANIZATION';
OR: 'OR';
OR_PREDICATES: 'OR_PREDICATES';
OSERROR: 'OSERROR';
OTHER: 'OTHER';
OUTER_JOIN_TO_ANTI: 'OUTER_JOIN_TO_ANTI';
OUTER_JOIN_TO_INNER: 'OUTER_JOIN_TO_INNER';
OUTER: 'OUTER';
OUTLINE_LEAF: 'OUTLINE_LEAF';
OUTLINE: 'OUTLINE';
OUT_OF_LINE: 'OUT_OF_LINE';
OUT: 'OUT';
OVERFLOW_NOMOVE: 'OVERFLOW_NOMOVE';
OVERFLOW: 'OVERFLOW';
OVERLAPS: 'OVERLAPS';
OVER: 'OVER';
OVERRIDING: 'OVERRIDING';
OWNER: 'OWNER';
OWNERSHIP: 'OWNERSHIP';
OWN: 'OWN';
PACKAGE: 'PACKAGE';
PACKAGES: 'PACKAGES';
PARALLEL_ENABLE: 'PARALLEL_ENABLE';
PARALLEL_INDEX: 'PARALLEL_INDEX';
PARALLEL: 'PARALLEL';
PARAMETERS: 'PARAMETERS';
PARAM: 'PARAM';
PARENT: 'PARENT';
PARITY: 'PARITY';
PARTIAL_JOIN: 'PARTIAL_JOIN';
PARTIALLY: 'PARTIALLY';
PARTIAL: 'PARTIAL';
PARTIAL_ROLLUP_PUSHDOWN: 'PARTIAL_ROLLUP_PUSHDOWN';
PARTITION_HASH: 'PARTITION_HASH';
PARTITION_LIST: 'PARTITION_LIST';
PARTITION: 'PARTITION';
PARTITION_RANGE: 'PARTITION_RANGE';
PARTITIONS: 'PARTITIONS';
PARTNUMINST: 'PART$NUM$INST';
PASSING: 'PASSING';
PASSWORD_GRACE_TIME: 'PASSWORD_GRACE_TIME';
PASSWORD_LIFE_TIME: 'PASSWORD_LIFE_TIME';
PASSWORD_LOCK_TIME: 'PASSWORD_LOCK_TIME';
PASSWORD: 'PASSWORD';
PASSWORD_REUSE_MAX: 'PASSWORD_REUSE_MAX';
PASSWORD_REUSE_TIME: 'PASSWORD_REUSE_TIME';
PASSWORD_VERIFY_FUNCTION: 'PASSWORD_VERIFY_FUNCTION';
PAST: 'PAST';
PATCH: 'PATCH';
PATH: 'PATH';
PATH_PREFIX: 'PATH_PREFIX';
PATHS: 'PATHS';
PATTERN: 'PATTERN';
PBL_HS_BEGIN: 'PBL_HS_BEGIN';
PBL_HS_END: 'PBL_HS_END';
PCTFREE: 'PCTFREE';
PCTINCREASE: 'PCTINCREASE';
PCTTHRESHOLD: 'PCTTHRESHOLD';
PCTUSED: 'PCTUSED';
PCTVERSION: 'PCTVERSION';
PENDING: 'PENDING';
PERCENT_FOUND: '%FOUND';
PERCENT_ISOPEN: '%ISOPEN';
PERCENT_NOTFOUND: '%NOTFOUND';
PERCENT_KEYWORD: 'PERCENT';
PERCENT_RANKM: 'PERCENT_RANKM';
PERCENT_ROWCOUNT: '%ROWCOUNT';
PERCENT_ROWTYPE: '%ROWTYPE';
PERCENT_TYPE: '%TYPE';
PERFORMANCE: 'PERFORMANCE';
PERIOD_KEYWORD: 'PERIOD';
PERMANENT: 'PERMANENT';
PERMISSION: 'PERMISSION';
PERMUTE: 'PERMUTE';
PER: 'PER';
PFILE: 'PFILE';
PHYSICAL: 'PHYSICAL';
PIKEY: 'PIKEY';
PIPELINED: 'PIPELINED';
PIPE: 'PIPE';
PIV_GB: 'PIV_GB';
PIVOT: 'PIVOT';
PIV_SSF: 'PIV_SSF';
PLACE_DISTINCT: 'PLACE_DISTINCT';
PLACE_GROUP_BY: 'PLACE_GROUP_BY';
PLAN: 'PLAN';
PLSCOPE_SETTINGS: 'PLSCOPE_SETTINGS';
PLS_INTEGER: 'PLS_INTEGER';
PLSQL_CCFLAGS: 'PLSQL_CCFLAGS';
PLSQL_CODE_TYPE: 'PLSQL_CODE_TYPE';
PLSQL_DEBUG: 'PLSQL_DEBUG';
PLSQL_OPTIMIZE_LEVEL: 'PLSQL_OPTIMIZE_LEVEL';
PLSQL_WARNINGS: 'PLSQL_WARNINGS';
PLUGGABLE: 'PLUGGABLE';
POINT: 'POINT';
POLICY: 'POLICY';
POOL_16K: 'POOL_16K';
POOL_2K: 'POOL_2K';
POOL_32K: 'POOL_32K';
POOL_4K: 'POOL_4K';
POOL_8K: 'POOL_8K';
POSITIVEN: 'POSITIVEN';
POSITIVE: 'POSITIVE';
POST_TRANSACTION: 'POST_TRANSACTION';
POWERMULTISET_BY_CARDINALITY: 'POWERMULTISET_BY_CARDINALITY';
POWERMULTISET: 'POWERMULTISET';
POWER: 'POWER';
PQ_CONCURRENT_UNION: 'PQ_CONCURRENT_UNION';
PQ_DISTRIBUTE: 'PQ_DISTRIBUTE';
PQ_DISTRIBUTE_WINDOW: 'PQ_DISTRIBUTE_WINDOW';
PQ_FILTER: 'PQ_FILTER';
PQ_MAP: 'PQ_MAP';
PQ_NOMAP: 'PQ_NOMAP';
PQ_REPLICATE: 'PQ_REPLICATE';
PQ_SKEW: 'PQ_SKEW';
PRAGMA: 'PRAGMA';
PREBUILT: 'PREBUILT';
PRECEDES: 'PRECEDES';
PRECEDING: 'PRECEDING';
PRECISION: 'PRECISION';
PRECOMPUTE_SUBQUERY: 'PRECOMPUTE_SUBQUERY';
PREDICATE_REORDERS: 'PREDICATE_REORDERS';
PRELOAD: 'PRELOAD';
PREPARE: 'PREPARE';
PRESENTNNV: 'PRESENTNNV';
PRESENT: 'PRESENT';
PRESENTV: 'PRESENTV';
PRESERVE_OID: 'PRESERVE_OID';
PRESERVE: 'PRESERVE';
PRETTY: 'PRETTY';
PREVIOUS: 'PREVIOUS';
PREV: 'PREV';
PRIMARY: 'PRIMARY';
PRINTBLOBTOCLOB: 'PRINTBLOBTOCLOB';
PRIORITY: 'PRIORITY';
PRIOR: 'PRIOR';
PRIVATE: 'PRIVATE';
PRIVATE_SGA: 'PRIVATE_SGA';
PRIVILEGED: 'PRIVILEGED';
PRIVILEGE: 'PRIVILEGE';
PRIVILEGES: 'PRIVILEGES';
PROCEDURAL: 'PROCEDURAL';
PROCEDURE: 'PROCEDURE';
PROCESS: 'PROCESS';
PROFILE: 'PROFILE';
PROGRAM: 'PROGRAM';
PROJECT: 'PROJECT';
PROPAGATE: 'PROPAGATE';
PROTECTED: 'PROTECTED';
PROTECTION: 'PROTECTION';
PROXY: 'PROXY';
PRUNING: 'PRUNING';
PUBLIC: 'PUBLIC';
PULL_PRED: 'PULL_PRED';
PURGE: 'PURGE';
PUSH_PRED: 'PUSH_PRED';
PUSH_SUBQ: 'PUSH_SUBQ';
PX_FAULT_TOLERANCE: 'PX_FAULT_TOLERANCE';
PX_GRANULE: 'PX_GRANULE';
PX_JOIN_FILTER: 'PX_JOIN_FILTER';
QB_NAME: 'QB_NAME';
QUERY_BLOCK: 'QUERY_BLOCK';
QUERY: 'QUERY';
QUEUE_CURR: 'QUEUE_CURR';
QUEUE: 'QUEUE';
QUEUE_ROWP: 'QUEUE_ROWP';
QUIESCE: 'QUIESCE';
QUORUM: 'QUORUM';
QUOTA: 'QUOTA';
RAISE: 'RAISE';
RANDOM_LOCAL: 'RANDOM_LOCAL';
RANDOM: 'RANDOM';
RANGE: 'RANGE';
RANKM: 'RANKM';
RAPIDLY: 'RAPIDLY';
RAW: 'RAW';
RAWTOHEX: 'RAWTOHEX';
RAWTONHEX: 'RAWTONHEX';
RBA: 'RBA';
RBO_OUTLINE: 'RBO_OUTLINE';
RDBA: 'RDBA';
READ: 'READ';
READS: 'READS';
REALM: 'REALM';
REAL: 'REAL';
REBALANCE: 'REBALANCE';
REBUILD: 'REBUILD';
RECORD: 'RECORD';
RECORDS_PER_BLOCK: 'RECORDS_PER_BLOCK';
RECOVERABLE: 'RECOVERABLE';
RECOVER: 'RECOVER';
RECOVERY: 'RECOVERY';
RECYCLEBIN: 'RECYCLEBIN';
RECYCLE: 'RECYCLE';
REDACTION: 'REDACTION';
REDEFINE: 'REDEFINE';
REDO: 'REDO';
REDUCED: 'REDUCED';
REDUNDANCY: 'REDUNDANCY';
REF_CASCADE_CURSOR: 'REF_CASCADE_CURSOR';
REFERENCED: 'REFERENCED';
REFERENCE: 'REFERENCE';
REFERENCES: 'REFERENCES';
REFERENCING: 'REFERENCING';
REF: 'REF';
REFRESH: 'REFRESH';
REFTOHEX: 'REFTOHEX';
REGEXP_COUNT: 'REGEXP_COUNT';
REGEXP_INSTR: 'REGEXP_INSTR';
REGEXP_LIKE: 'REGEXP_LIKE';
REGEXP_REPLACE: 'REGEXP_REPLACE';
REGEXP_SUBSTR: 'REGEXP_SUBSTR';
REGISTER: 'REGISTER';
REGR_AVGX: 'REGR_AVGX';
REGR_AVGY: 'REGR_AVGY';
REGR_COUNT: 'REGR_COUNT';
REGR_INTERCEPT: 'REGR_INTERCEPT';
REGR_R2: 'REGR_R2';
REGR_SLOPE: 'REGR_SLOPE';
REGR_SXX: 'REGR_SXX';
REGR_SXY: 'REGR_SXY';
REGR_SYY: 'REGR_SYY';
REGULAR: 'REGULAR';
REJECT: 'REJECT';
REKEY: 'REKEY';
RELATIONAL: 'RELATIONAL';
RELIES_ON: 'RELIES_ON';
RELOCATE: 'RELOCATE';
RELY: 'RELY';
REMAINDER: 'REMAINDER';
REMOTE_MAPPED: 'REMOTE_MAPPED';
REMOVE: 'REMOVE';
RENAME: 'RENAME';
REPAIR: 'REPAIR';
REPEAT: 'REPEAT';
REPLACE: 'REPLACE';
REPLICATION: 'REPLICATION';
REQUIRED: 'REQUIRED';
RESETLOGS: 'RESETLOGS';
RESET: 'RESET';
RESIZE: 'RESIZE';
RESOLVE: 'RESOLVE';
RESOLVER: 'RESOLVER';
RESOURCE: 'RESOURCE';
RESPECT: 'RESPECT';
RESTART: 'RESTART';
RESTORE_AS_INTERVALS: 'RESTORE_AS_INTERVALS';
RESTORE: 'RESTORE';
RESTRICT_ALL_REF_CONS: 'RESTRICT_ALL_REF_CONS';
RESTRICTED: 'RESTRICTED';
RESTRICT_REFERENCES: 'RESTRICT_REFERENCES';
RESTRICT: 'RESTRICT';
RESULT_CACHE: 'RESULT_CACHE';
RESULT: 'RESULT';
RESUMABLE: 'RESUMABLE';
RESUME: 'RESUME';
RETENTION: 'RETENTION';
RETRY_ON_ROW_CHANGE: 'RETRY_ON_ROW_CHANGE';
RETURNING: 'RETURNING';
RETURN: 'RETURN';
REUSE: 'REUSE';
REVERSE: 'REVERSE';
REVOKE: 'REVOKE';
REWRITE_OR_ERROR: 'REWRITE_OR_ERROR';
REWRITE: 'REWRITE';
RIGHT: 'RIGHT';
ROLE: 'ROLE';
ROLESET: 'ROLESET';
ROLES: 'ROLES';
ROLLBACK: 'ROLLBACK';
ROLLING: 'ROLLING';
ROLLUP: 'ROLLUP';
ROWDEPENDENCIES: 'ROWDEPENDENCIES';
ROWID_MAPPING_TABLE: 'ROWID_MAPPING_TABLE';
ROWID: 'ROWID';
ROWIDTOCHAR: 'ROWIDTOCHAR';
ROWIDTONCHAR: 'ROWIDTONCHAR';
ROW_LENGTH: 'ROW_LENGTH';
ROWNUM: 'ROWNUM';
ROW: 'ROW';
ROWS: 'ROWS';
RPAD: 'RPAD';
RTRIM: 'RTRIM';
RULE: 'RULE';
RULES: 'RULES';
RUNNING: 'RUNNING';
SALT: 'SALT';
SAMPLE: 'SAMPLE';
SAVE_AS_INTERVALS: 'SAVE_AS_INTERVALS';
SAVEPOINT: 'SAVEPOINT';
SAVE: 'SAVE';
SB4: 'SB4';
SCALE_ROWS: 'SCALE_ROWS';
SCALE: 'SCALE';
SCAN_INSTANCES: 'SCAN_INSTANCES';
SCAN: 'SCAN';
SCHEDULER: 'SCHEDULER';
SCHEMACHECK: 'SCHEMACHECK';
SCHEMA: 'SCHEMA';
SCN_ASCENDING: 'SCN_ASCENDING';
SCN: 'SCN';
SCOPE: 'SCOPE';
SCRUB: 'SCRUB';
SD_ALL: 'SD_ALL';
SD_INHIBIT: 'SD_INHIBIT';
SDO_GEOM_MBR: 'SDO_GEOM_MBR';
SD_SHOW: 'SD_SHOW';
SEARCH: 'SEARCH';
SECOND: 'SECOND';
SECRET: 'SECRET';
SECUREFILE_DBA: 'SECUREFILE_DBA';
SECUREFILE: 'SECUREFILE';
SECURITY: 'SECURITY';
SEED: 'SEED';
SEG_BLOCK: 'SEG_BLOCK';
SEG_FILE: 'SEG_FILE';
SEGMENT: 'SEGMENT';
SELECTIVITY: 'SELECTIVITY';
SELECT: 'SELECT';
SELF: 'SELF';
SEMIJOIN_DRIVER: 'SEMIJOIN_DRIVER';
SEMIJOIN: 'SEMIJOIN';
SEMI_TO_INNER: 'SEMI_TO_INNER';
SEQUENCED: 'SEQUENCED';
SEQUENCE: 'SEQUENCE';
SEQUENTIAL: 'SEQUENTIAL';
SERIALIZABLE: 'SERIALIZABLE';
SERIALLY_REUSABLE: 'SERIALLY_REUSABLE';
SERIAL: 'SERIAL';
SERVERERROR: 'SERVERERROR';
SERVICE_NAME_CONVERT: 'SERVICE_NAME_CONVERT';
SERVICES: 'SERVICES';
SESSION_CACHED_CURSORS: 'SESSION_CACHED_CURSORS';
SESSION: 'SESSION';
SESSIONS_PER_USER: 'SESSIONS_PER_USER';
SESSIONTIMEZONE: 'SESSIONTIMEZONE';
SESSIONTZNAME: 'SESSIONTZNAME';
SET: 'SET';
SETS: 'SETS';
SETTINGS: 'SETTINGS';
SET_TO_JOIN: 'SET_TO_JOIN';
SEVERE: 'SEVERE';
SHARED_POOL: 'SHARED_POOL';
SHARED: 'SHARED';
SHARE: 'SHARE';
SHARING: 'SHARING';
SHELFLIFE: 'SHELFLIFE';
SHOW: 'SHOW';
SHRINK: 'SHRINK';
SHUTDOWN: 'SHUTDOWN';
SIBLINGS: 'SIBLINGS';
SID: 'SID';
SIGNAL_COMPONENT: 'SIGNAL_COMPONENT';
SIGNAL_FUNCTION: 'SIGNAL_FUNCTION';
SIGN: 'SIGN';
SIGNTYPE: 'SIGNTYPE';
SIMPLE_INTEGER: 'SIMPLE_INTEGER';
SIMPLE: 'SIMPLE';
SINGLE: 'SINGLE';
SINGLETASK: 'SINGLETASK';
SINH: 'SINH';
SIN: 'SIN';
SIZE: 'SIZE';
SKIP_EXT_OPTIMIZER: 'SKIP_EXT_OPTIMIZER';
SKIP_ : 'SKIP';
SKIP_UNQ_UNUSABLE_IDX: 'SKIP_UNQ_UNUSABLE_IDX';
SKIP_UNUSABLE_INDEXES: 'SKIP_UNUSABLE_INDEXES';
SMALLFILE: 'SMALLFILE';
SMALLINT: 'SMALLINT';
SNAPSHOT: 'SNAPSHOT';
SOME: 'SOME';
SORT: 'SORT';
SOUNDEX: 'SOUNDEX';
SOURCE_FILE_DIRECTORY: 'SOURCE_FILE_DIRECTORY';
SOURCE_FILE_NAME_CONVERT: 'SOURCE_FILE_NAME_CONVERT';
SOURCE: 'SOURCE';
SPACE_KEYWORD: 'SPACE';
SPECIFICATION: 'SPECIFICATION';
SPFILE: 'SPFILE';
SPLIT: 'SPLIT';
SPREADSHEET: 'SPREADSHEET';
SQLDATA: 'SQLDATA';
SQLERROR: 'SQLERROR';
SQLLDR: 'SQLLDR';
SQL: 'SQL';
SQL_TRACE: 'SQL_TRACE';
SQL_TRANSLATION_PROFILE: 'SQL_TRANSLATION_PROFILE';
SQRT: 'SQRT';
STALE: 'STALE';
STANDALONE: 'STANDALONE';
STANDARD_HASH: 'STANDARD_HASH';
STANDBY_MAX_DATA_DELAY: 'STANDBY_MAX_DATA_DELAY';
STANDBYS: 'STANDBYS';
STANDBY: 'STANDBY';
STAR: 'STAR';
STAR_TRANSFORMATION: 'STAR_TRANSFORMATION';
START: 'START';
STARTUP: 'STARTUP';
STATEMENT_ID: 'STATEMENT_ID';
STATEMENT_QUEUING: 'STATEMENT_QUEUING';
STATEMENTS: 'STATEMENTS';
STATEMENT: 'STATEMENT';
STATE: 'STATE';
STATIC: 'STATIC';
STATISTICS: 'STATISTICS';
STATS_BINOMIAL_TEST: 'STATS_BINOMIAL_TEST';
STATS_CROSSTAB: 'STATS_CROSSTAB';
STATS_F_TEST: 'STATS_F_TEST';
STATS_KS_TEST: 'STATS_KS_TEST';
STATS_MODE: 'STATS_MODE';
STATS_MW_TEST: 'STATS_MW_TEST';
STATS_ONE_WAY_ANOVA: 'STATS_ONE_WAY_ANOVA';
STATS_T_TEST_INDEP: 'STATS_T_TEST_INDEP';
STATS_T_TEST_INDEPU: 'STATS_T_TEST_INDEPU';
STATS_T_TEST_ONE: 'STATS_T_TEST_ONE';
STATS_T_TEST_PAIRED: 'STATS_T_TEST_PAIRED';
STATS_WSR_TEST: 'STATS_WSR_TEST';
STDDEV_POP: 'STDDEV_POP';
STDDEV_SAMP: 'STDDEV_SAMP';
STOP: 'STOP';
STORAGE: 'STORAGE';
STORE: 'STORE';
STREAMS: 'STREAMS';
STREAM: 'STREAM';
STRICT: 'STRICT';
STRING: 'STRING';
STRIPE_COLUMNS: 'STRIPE_COLUMNS';
STRIPE_WIDTH: 'STRIPE_WIDTH';
STRIP: 'STRIP';
STRUCTURE: 'STRUCTURE';
SUBMULTISET: 'SUBMULTISET';
SUBPARTITION_REL: 'SUBPARTITION_REL';
SUBPARTITIONS: 'SUBPARTITIONS';
SUBPARTITION: 'SUBPARTITION';
SUBQUERIES: 'SUBQUERIES';
SUBQUERY_PRUNING: 'SUBQUERY_PRUNING';
SUBSCRIBE: 'SUBSCRIBE';
SUBSET: 'SUBSET';
SUBSTITUTABLE: 'SUBSTITUTABLE';
SUBSTR2: 'SUBSTR2';
SUBSTR4: 'SUBSTR4';
SUBSTRB: 'SUBSTRB';
SUBSTRC: 'SUBSTRC';
SUBTYPE: 'SUBTYPE';
SUCCESSFUL: 'SUCCESSFUL';
SUCCESS: 'SUCCESS';
SUMMARY: 'SUMMARY';
SUPPLEMENTAL: 'SUPPLEMENTAL';
SUSPEND: 'SUSPEND';
SWAP_JOIN_INPUTS: 'SWAP_JOIN_INPUTS';
SWITCHOVER: 'SWITCHOVER';
SWITCH: 'SWITCH';
SYNCHRONOUS: 'SYNCHRONOUS';
SYNC: 'SYNC';
SYNONYM: 'SYNONYM';
SYSASM: 'SYSASM';
SYS_AUDIT: 'SYS_AUDIT';
SYSAUX: 'SYSAUX';
SYSBACKUP: 'SYSBACKUP';
SYS_CHECKACL: 'SYS_CHECKACL';
SYS_CHECK_PRIVILEGE: 'SYS_CHECK_PRIVILEGE';
SYS_CONNECT_BY_PATH: 'SYS_CONNECT_BY_PATH';
SYS_CONTEXT: 'SYS_CONTEXT';
SYSDATE: 'SYSDATE';
SYSDBA: 'SYSDBA';
SYS_DBURIGEN: 'SYS_DBURIGEN';
SYSDG: 'SYSDG';
SYS_DL_CURSOR: 'SYS_DL_CURSOR';
SYS_DM_RXFORM_CHR: 'SYS_DM_RXFORM_CHR';
SYS_DM_RXFORM_NUM: 'SYS_DM_RXFORM_NUM';
SYS_DOM_COMPARE: 'SYS_DOM_COMPARE';
SYS_DST_PRIM2SEC: 'SYS_DST_PRIM2SEC';
SYS_DST_SEC2PRIM: 'SYS_DST_SEC2PRIM';
SYS_ET_BFILE_TO_RAW: 'SYS_ET_BFILE_TO_RAW';
SYS_ET_BLOB_TO_IMAGE: 'SYS_ET_BLOB_TO_IMAGE';
SYS_ET_IMAGE_TO_BLOB: 'SYS_ET_IMAGE_TO_BLOB';
SYS_ET_RAW_TO_BFILE: 'SYS_ET_RAW_TO_BFILE';
SYS_EXTPDTXT: 'SYS_EXTPDTXT';
SYS_EXTRACT_UTC: 'SYS_EXTRACT_UTC';
SYS_FBT_INSDEL: 'SYS_FBT_INSDEL';
SYS_FILTER_ACLS: 'SYS_FILTER_ACLS';
SYS_FNMATCHES: 'SYS_FNMATCHES';
SYS_FNREPLACE: 'SYS_FNREPLACE';
SYS_GET_ACLIDS: 'SYS_GET_ACLIDS';
SYS_GET_COL_ACLIDS: 'SYS_GET_COL_ACLIDS';
SYS_GET_PRIVILEGES: 'SYS_GET_PRIVILEGES';
SYS_GETTOKENID: 'SYS_GETTOKENID';
SYS_GETXTIVAL: 'SYS_GETXTIVAL';
SYS_GUID: 'SYS_GUID';
SYSGUID: 'SYSGUID';
SYSKM: 'SYSKM';
SYS_MAKE_XMLNODEID: 'SYS_MAKE_XMLNODEID';
SYS_MAKEXML: 'SYS_MAKEXML';
SYS_MKXMLATTR: 'SYS_MKXMLATTR';
SYS_MKXTI: 'SYS_MKXTI';
SYSOBJ: 'SYSOBJ';
SYS_OP_ADT2BIN: 'SYS_OP_ADT2BIN';
SYS_OP_ADTCONS: 'SYS_OP_ADTCONS';
SYS_OP_ALSCRVAL: 'SYS_OP_ALSCRVAL';
SYS_OP_ATG: 'SYS_OP_ATG';
SYS_OP_BIN2ADT: 'SYS_OP_BIN2ADT';
SYS_OP_BITVEC: 'SYS_OP_BITVEC';
SYS_OP_BL2R: 'SYS_OP_BL2R';
SYS_OP_BLOOM_FILTER_LIST: 'SYS_OP_BLOOM_FILTER_LIST';
SYS_OP_BLOOM_FILTER: 'SYS_OP_BLOOM_FILTER';
SYS_OP_C2C: 'SYS_OP_C2C';
SYS_OP_CAST: 'SYS_OP_CAST';
SYS_OP_CEG: 'SYS_OP_CEG';
SYS_OP_CL2C: 'SYS_OP_CL2C';
SYS_OP_COMBINED_HASH: 'SYS_OP_COMBINED_HASH';
SYS_OP_COMP: 'SYS_OP_COMP';
SYS_OP_CONVERT: 'SYS_OP_CONVERT';
SYS_OP_COUNTCHG: 'SYS_OP_COUNTCHG';
SYS_OP_CSCONV: 'SYS_OP_CSCONV';
SYS_OP_CSCONVTEST: 'SYS_OP_CSCONVTEST';
SYS_OP_CSR: 'SYS_OP_CSR';
SYS_OP_CSX_PATCH: 'SYS_OP_CSX_PATCH';
SYS_OP_CYCLED_SEQ: 'SYS_OP_CYCLED_SEQ';
SYS_OP_DECOMP: 'SYS_OP_DECOMP';
SYS_OP_DESCEND: 'SYS_OP_DESCEND';
SYS_OP_DISTINCT: 'SYS_OP_DISTINCT';
SYS_OP_DRA: 'SYS_OP_DRA';
SYS_OP_DUMP: 'SYS_OP_DUMP';
SYS_OP_DV_CHECK: 'SYS_OP_DV_CHECK';
SYS_OP_ENFORCE_NOT_NULL: 'SYS_OP_ENFORCE_NOT_NULL$';
SYSOPER: 'SYSOPER';
SYS_OP_EXTRACT: 'SYS_OP_EXTRACT';
SYS_OP_GROUPING: 'SYS_OP_GROUPING';
SYS_OP_GUID: 'SYS_OP_GUID';
SYS_OP_HASH: 'SYS_OP_HASH';
SYS_OP_IIX: 'SYS_OP_IIX';
SYS_OP_ITR: 'SYS_OP_ITR';
SYS_OP_KEY_VECTOR_CREATE: 'SYS_OP_KEY_VECTOR_CREATE';
SYS_OP_KEY_VECTOR_FILTER_LIST: 'SYS_OP_KEY_VECTOR_FILTER_LIST';
SYS_OP_KEY_VECTOR_FILTER: 'SYS_OP_KEY_VECTOR_FILTER';
SYS_OP_KEY_VECTOR_SUCCEEDED: 'SYS_OP_KEY_VECTOR_SUCCEEDED';
SYS_OP_KEY_VECTOR_USE: 'SYS_OP_KEY_VECTOR_USE';
SYS_OP_LBID: 'SYS_OP_LBID';
SYS_OP_LOBLOC2BLOB: 'SYS_OP_LOBLOC2BLOB';
SYS_OP_LOBLOC2CLOB: 'SYS_OP_LOBLOC2CLOB';
SYS_OP_LOBLOC2ID: 'SYS_OP_LOBLOC2ID';
SYS_OP_LOBLOC2NCLOB: 'SYS_OP_LOBLOC2NCLOB';
SYS_OP_LOBLOC2TYP: 'SYS_OP_LOBLOC2TYP';
SYS_OP_LSVI: 'SYS_OP_LSVI';
SYS_OP_LVL: 'SYS_OP_LVL';
SYS_OP_MAKEOID: 'SYS_OP_MAKEOID';
SYS_OP_MAP_NONNULL: 'SYS_OP_MAP_NONNULL';
SYS_OP_MSR: 'SYS_OP_MSR';
SYS_OP_NICOMBINE: 'SYS_OP_NICOMBINE';
SYS_OP_NIEXTRACT: 'SYS_OP_NIEXTRACT';
SYS_OP_NII: 'SYS_OP_NII';
SYS_OP_NIX: 'SYS_OP_NIX';
SYS_OP_NOEXPAND: 'SYS_OP_NOEXPAND';
SYS_OP_NTCIMG: 'SYS_OP_NTCIMG$';
SYS_OP_NUMTORAW: 'SYS_OP_NUMTORAW';
SYS_OP_OIDVALUE: 'SYS_OP_OIDVALUE';
SYS_OP_OPNSIZE: 'SYS_OP_OPNSIZE';
SYS_OP_PAR_1: 'SYS_OP_PAR_1';
SYS_OP_PARGID_1: 'SYS_OP_PARGID_1';
SYS_OP_PARGID: 'SYS_OP_PARGID';
SYS_OP_PAR: 'SYS_OP_PAR';
SYS_OP_PART_ID: 'SYS_OP_PART_ID';
SYS_OP_PIVOT: 'SYS_OP_PIVOT';
SYS_OP_R2O: 'SYS_OP_R2O';
SYS_OP_RAWTONUM: 'SYS_OP_RAWTONUM';
SYS_OP_RDTM: 'SYS_OP_RDTM';
SYS_OP_REF: 'SYS_OP_REF';
SYS_OP_RMTD: 'SYS_OP_RMTD';
SYS_OP_ROWIDTOOBJ: 'SYS_OP_ROWIDTOOBJ';
SYS_OP_RPB: 'SYS_OP_RPB';
SYS_OPTLOBPRBSC: 'SYS_OPTLOBPRBSC';
SYS_OP_TOSETID: 'SYS_OP_TOSETID';
SYS_OP_TPR: 'SYS_OP_TPR';
SYS_OP_TRTB: 'SYS_OP_TRTB';
SYS_OPTXICMP: 'SYS_OPTXICMP';
SYS_OPTXQCASTASNQ: 'SYS_OPTXQCASTASNQ';
SYS_OP_UNDESCEND: 'SYS_OP_UNDESCEND';
SYS_OP_VECAND: 'SYS_OP_VECAND';
SYS_OP_VECBIT: 'SYS_OP_VECBIT';
SYS_OP_VECOR: 'SYS_OP_VECOR';
SYS_OP_VECXOR: 'SYS_OP_VECXOR';
SYS_OP_VERSION: 'SYS_OP_VERSION';
SYS_OP_VREF: 'SYS_OP_VREF';
SYS_OP_VVD: 'SYS_OP_VVD';
SYS_OP_XMLCONS_FOR_CSX: 'SYS_OP_XMLCONS_FOR_CSX';
SYS_OP_XPTHATG: 'SYS_OP_XPTHATG';
SYS_OP_XPTHIDX: 'SYS_OP_XPTHIDX';
SYS_OP_XPTHOP: 'SYS_OP_XPTHOP';
SYS_OP_XTXT2SQLT: 'SYS_OP_XTXT2SQLT';
SYS_OP_ZONE_ID: 'SYS_OP_ZONE_ID';
SYS_ORDERKEY_DEPTH: 'SYS_ORDERKEY_DEPTH';
SYS_ORDERKEY_MAXCHILD: 'SYS_ORDERKEY_MAXCHILD';
SYS_ORDERKEY_PARENT: 'SYS_ORDERKEY_PARENT';
SYS_PARALLEL_TXN: 'SYS_PARALLEL_TXN';
SYS_PATHID_IS_ATTR: 'SYS_PATHID_IS_ATTR';
SYS_PATHID_IS_NMSPC: 'SYS_PATHID_IS_NMSPC';
SYS_PATHID_LASTNAME: 'SYS_PATHID_LASTNAME';
SYS_PATHID_LASTNMSPC: 'SYS_PATHID_LASTNMSPC';
SYS_PATH_REVERSE: 'SYS_PATH_REVERSE';
SYS_PXQEXTRACT: 'SYS_PXQEXTRACT';
SYS_RAW_TO_XSID: 'SYS_RAW_TO_XSID';
SYS_RID_ORDER: 'SYS_RID_ORDER';
SYS_ROW_DELTA: 'SYS_ROW_DELTA';
SYS_SC_2_XMLT: 'SYS_SC_2_XMLT';
SYS_SYNRCIREDO: 'SYS_SYNRCIREDO';
SYSTEM_DEFINED: 'SYSTEM_DEFINED';
SYSTEM: 'SYSTEM';
SYSTIMESTAMP: 'SYSTIMESTAMP';
SYS_TYPEID: 'SYS_TYPEID';
SYS_UMAKEXML: 'SYS_UMAKEXML';
SYS_XMLANALYZE: 'SYS_XMLANALYZE';
SYS_XMLCONTAINS: 'SYS_XMLCONTAINS';
SYS_XMLCONV: 'SYS_XMLCONV';
SYS_XMLEXNSURI: 'SYS_XMLEXNSURI';
SYS_XMLGEN: 'SYS_XMLGEN';
SYS_XMLI_LOC_ISNODE: 'SYS_XMLI_LOC_ISNODE';
SYS_XMLI_LOC_ISTEXT: 'SYS_XMLI_LOC_ISTEXT';
SYS_XMLINSTR: 'SYS_XMLINSTR';
SYS_XMLLOCATOR_GETSVAL: 'SYS_XMLLOCATOR_GETSVAL';
SYS_XMLNODEID_GETCID: 'SYS_XMLNODEID_GETCID';
SYS_XMLNODEID_GETLOCATOR: 'SYS_XMLNODEID_GETLOCATOR';
SYS_XMLNODEID_GETOKEY: 'SYS_XMLNODEID_GETOKEY';
SYS_XMLNODEID_GETPATHID: 'SYS_XMLNODEID_GETPATHID';
SYS_XMLNODEID_GETPTRID: 'SYS_XMLNODEID_GETPTRID';
SYS_XMLNODEID_GETRID: 'SYS_XMLNODEID_GETRID';
SYS_XMLNODEID_GETSVAL: 'SYS_XMLNODEID_GETSVAL';
SYS_XMLNODEID_GETTID: 'SYS_XMLNODEID_GETTID';
SYS_XMLNODEID: 'SYS_XMLNODEID';
SYS_XMLT_2_SC: 'SYS_XMLT_2_SC';
SYS_XMLTRANSLATE: 'SYS_XMLTRANSLATE';
SYS_XMLTYPE2SQL: 'SYS_XMLTYPE2SQL';
SYS_XQ_ASQLCNV: 'SYS_XQ_ASQLCNV';
SYS_XQ_ATOMCNVCHK: 'SYS_XQ_ATOMCNVCHK';
SYS_XQBASEURI: 'SYS_XQBASEURI';
SYS_XQCASTABLEERRH: 'SYS_XQCASTABLEERRH';
SYS_XQCODEP2STR: 'SYS_XQCODEP2STR';
SYS_XQCODEPEQ: 'SYS_XQCODEPEQ';
SYS_XQCON2SEQ: 'SYS_XQCON2SEQ';
SYS_XQCONCAT: 'SYS_XQCONCAT';
SYS_XQDELETE: 'SYS_XQDELETE';
SYS_XQDFLTCOLATION: 'SYS_XQDFLTCOLATION';
SYS_XQDOC: 'SYS_XQDOC';
SYS_XQDOCURI: 'SYS_XQDOCURI';
SYS_XQDURDIV: 'SYS_XQDURDIV';
SYS_XQED4URI: 'SYS_XQED4URI';
SYS_XQENDSWITH: 'SYS_XQENDSWITH';
SYS_XQERRH: 'SYS_XQERRH';
SYS_XQERR: 'SYS_XQERR';
SYS_XQESHTMLURI: 'SYS_XQESHTMLURI';
SYS_XQEXLOBVAL: 'SYS_XQEXLOBVAL';
SYS_XQEXSTWRP: 'SYS_XQEXSTWRP';
SYS_XQEXTRACT: 'SYS_XQEXTRACT';
SYS_XQEXTRREF: 'SYS_XQEXTRREF';
SYS_XQEXVAL: 'SYS_XQEXVAL';
SYS_XQFB2STR: 'SYS_XQFB2STR';
SYS_XQFNBOOL: 'SYS_XQFNBOOL';
SYS_XQFNCMP: 'SYS_XQFNCMP';
SYS_XQFNDATIM: 'SYS_XQFNDATIM';
SYS_XQFNLNAME: 'SYS_XQFNLNAME';
SYS_XQFNNM: 'SYS_XQFNNM';
SYS_XQFNNSURI: 'SYS_XQFNNSURI';
SYS_XQFNPREDTRUTH: 'SYS_XQFNPREDTRUTH';
SYS_XQFNQNM: 'SYS_XQFNQNM';
SYS_XQFNROOT: 'SYS_XQFNROOT';
SYS_XQFORMATNUM: 'SYS_XQFORMATNUM';
SYS_XQFTCONTAIN: 'SYS_XQFTCONTAIN';
SYS_XQFUNCR: 'SYS_XQFUNCR';
SYS_XQGETCONTENT: 'SYS_XQGETCONTENT';
SYS_XQINDXOF: 'SYS_XQINDXOF';
SYS_XQINSERT: 'SYS_XQINSERT';
SYS_XQINSPFX: 'SYS_XQINSPFX';
SYS_XQIRI2URI: 'SYS_XQIRI2URI';
SYS_XQLANG: 'SYS_XQLANG';
SYS_XQLLNMFRMQNM: 'SYS_XQLLNMFRMQNM';
SYS_XQMKNODEREF: 'SYS_XQMKNODEREF';
SYS_XQNILLED: 'SYS_XQNILLED';
SYS_XQNODENAME: 'SYS_XQNODENAME';
SYS_XQNORMSPACE: 'SYS_XQNORMSPACE';
SYS_XQNORMUCODE: 'SYS_XQNORMUCODE';
SYS_XQ_NRNG: 'SYS_XQ_NRNG';
SYS_XQNSP4PFX: 'SYS_XQNSP4PFX';
SYS_XQNSPFRMQNM: 'SYS_XQNSPFRMQNM';
SYS_XQPFXFRMQNM: 'SYS_XQPFXFRMQNM';
SYS_XQ_PKSQL2XML: 'SYS_XQ_PKSQL2XML';
SYS_XQPOLYABS: 'SYS_XQPOLYABS';
SYS_XQPOLYADD: 'SYS_XQPOLYADD';
SYS_XQPOLYCEL: 'SYS_XQPOLYCEL';
SYS_XQPOLYCSTBL: 'SYS_XQPOLYCSTBL';
SYS_XQPOLYCST: 'SYS_XQPOLYCST';
SYS_XQPOLYDIV: 'SYS_XQPOLYDIV';
SYS_XQPOLYFLR: 'SYS_XQPOLYFLR';
SYS_XQPOLYMOD: 'SYS_XQPOLYMOD';
SYS_XQPOLYMUL: 'SYS_XQPOLYMUL';
SYS_XQPOLYRND: 'SYS_XQPOLYRND';
SYS_XQPOLYSQRT: 'SYS_XQPOLYSQRT';
SYS_XQPOLYSUB: 'SYS_XQPOLYSUB';
SYS_XQPOLYUMUS: 'SYS_XQPOLYUMUS';
SYS_XQPOLYUPLS: 'SYS_XQPOLYUPLS';
SYS_XQPOLYVEQ: 'SYS_XQPOLYVEQ';
SYS_XQPOLYVGE: 'SYS_XQPOLYVGE';
SYS_XQPOLYVGT: 'SYS_XQPOLYVGT';
SYS_XQPOLYVLE: 'SYS_XQPOLYVLE';
SYS_XQPOLYVLT: 'SYS_XQPOLYVLT';
SYS_XQPOLYVNE: 'SYS_XQPOLYVNE';
SYS_XQREF2VAL: 'SYS_XQREF2VAL';
SYS_XQRENAME: 'SYS_XQRENAME';
SYS_XQREPLACE: 'SYS_XQREPLACE';
SYS_XQRESVURI: 'SYS_XQRESVURI';
SYS_XQRNDHALF2EVN: 'SYS_XQRNDHALF2EVN';
SYS_XQRSLVQNM: 'SYS_XQRSLVQNM';
SYS_XQRYENVPGET: 'SYS_XQRYENVPGET';
SYS_XQRYVARGET: 'SYS_XQRYVARGET';
SYS_XQRYWRP: 'SYS_XQRYWRP';
SYS_XQSEQ2CON4XC: 'SYS_XQSEQ2CON4XC';
SYS_XQSEQ2CON: 'SYS_XQSEQ2CON';
SYS_XQSEQDEEPEQ: 'SYS_XQSEQDEEPEQ';
SYS_XQSEQINSB: 'SYS_XQSEQINSB';
SYS_XQSEQRM: 'SYS_XQSEQRM';
SYS_XQSEQRVS: 'SYS_XQSEQRVS';
SYS_XQSEQSUB: 'SYS_XQSEQSUB';
SYS_XQSEQTYPMATCH: 'SYS_XQSEQTYPMATCH';
SYS_XQSTARTSWITH: 'SYS_XQSTARTSWITH';
SYS_XQSTATBURI: 'SYS_XQSTATBURI';
SYS_XQSTR2CODEP: 'SYS_XQSTR2CODEP';
SYS_XQSTRJOIN: 'SYS_XQSTRJOIN';
SYS_XQSUBSTRAFT: 'SYS_XQSUBSTRAFT';
SYS_XQSUBSTRBEF: 'SYS_XQSUBSTRBEF';
SYS_XQTOKENIZE: 'SYS_XQTOKENIZE';
SYS_XQTREATAS: 'SYS_XQTREATAS';
SYS_XQ_UPKXML2SQL: 'SYS_XQ_UPKXML2SQL';
SYS_XQXFORM: 'SYS_XQXFORM';
SYS_XSID_TO_RAW: 'SYS_XSID_TO_RAW';
SYS_ZMAP_FILTER: 'SYS_ZMAP_FILTER';
SYS_ZMAP_REFRESH: 'SYS_ZMAP_REFRESH';
TABLE_LOOKUP_BY_NL: 'TABLE_LOOKUP_BY_NL';
TABLESPACE_NO: 'TABLESPACE_NO';
TABLESPACE: 'TABLESPACE';
TABLES: 'TABLES';
TABLE_STATS: 'TABLE_STATS';
TABLE: 'TABLE';
TABNO: 'TABNO';
TAG: 'TAG';
TANH: 'TANH';
TAN: 'TAN';
TBLORIDXPARTNUM: 'TBL$OR$IDX$PART$NUM';
TEMPFILE: 'TEMPFILE';
TEMPLATE: 'TEMPLATE';
TEMPORARY: 'TEMPORARY';
TEMP_TABLE: 'TEMP_TABLE';
TEST: 'TEST';
TEXT: 'TEXT';
THAN: 'THAN';
THEN: 'THEN';
THE: 'THE';
THREAD: 'THREAD';
THROUGH: 'THROUGH';
TIER: 'TIER';
TIES: 'TIES';
TIMEOUT: 'TIMEOUT';
TIMESTAMP_LTZ_UNCONSTRAINED: 'TIMESTAMP_LTZ_UNCONSTRAINED';
TIMESTAMP: 'TIMESTAMP';
TIMESTAMP_TZ_UNCONSTRAINED: 'TIMESTAMP_TZ_UNCONSTRAINED';
TIMESTAMP_UNCONSTRAINED: 'TIMESTAMP_UNCONSTRAINED';
TIMES: 'TIMES';
TIME: 'TIME';
TIMEZONE: 'TIMEZONE';
TIMEZONE_ABBR: 'TIMEZONE_ABBR';
TIMEZONE_HOUR: 'TIMEZONE_HOUR';
TIMEZONE_MINUTE: 'TIMEZONE_MINUTE';
TIMEZONE_OFFSET: 'TIMEZONE_OFFSET';
TIMEZONE_REGION: 'TIMEZONE_REGION';
TIME_ZONE: 'TIME_ZONE';
TIV_GB: 'TIV_GB';
TIV_SSF: 'TIV_SSF';
TO_ACLID: 'TO_ACLID';
TO_BINARY_DOUBLE: 'TO_BINARY_DOUBLE';
TO_BINARY_FLOAT: 'TO_BINARY_FLOAT';
TO_BLOB: 'TO_BLOB';
TO_CLOB: 'TO_CLOB';
TO_DSINTERVAL: 'TO_DSINTERVAL';
TO_LOB: 'TO_LOB';
TO_MULTI_BYTE: 'TO_MULTI_BYTE';
TO_NCHAR: 'TO_NCHAR';
TO_NCLOB: 'TO_NCLOB';
TO_NUMBER: 'TO_NUMBER';
TOPLEVEL: 'TOPLEVEL';
TO_SINGLE_BYTE: 'TO_SINGLE_BYTE';
TO_TIMESTAMP: 'TO_TIMESTAMP';
TO_TIMESTAMP_TZ: 'TO_TIMESTAMP_TZ';
TO_TIME: 'TO_TIME';
TO_TIME_TZ: 'TO_TIME_TZ';
TO: 'TO';
TO_YMINTERVAL: 'TO_YMINTERVAL';
TRACE: 'TRACE';
TRACING: 'TRACING';
TRACKING: 'TRACKING';
TRAILING: 'TRAILING';
TRANSACTION: 'TRANSACTION';
TRANSFORM_DISTINCT_AGG: 'TRANSFORM_DISTINCT_AGG';
TRANSITIONAL: 'TRANSITIONAL';
TRANSITION: 'TRANSITION';
TRANSLATE: 'TRANSLATE';
TRANSLATION: 'TRANSLATION';
TREAT: 'TREAT';
TRIGGERS: 'TRIGGERS';
TRIGGER: 'TRIGGER';
TRUE: 'TRUE';
TRUNCATE: 'TRUNCATE';
TRUNC: 'TRUNC';
TRUSTED: 'TRUSTED';
TRUST: 'TRUST';
TUNING: 'TUNING';
TX: 'TX';
TYPES: 'TYPES';
TYPE: 'TYPE';
TZ_OFFSET: 'TZ_OFFSET';
UB2: 'UB2';
UBA: 'UBA';
UCS2: 'UCS2';
UID: 'UID';
UNARCHIVED: 'UNARCHIVED';
UNBOUNDED: 'UNBOUNDED';
UNBOUND: 'UNBOUND';
UNCONDITIONAL: 'UNCONDITIONAL';
UNDER: 'UNDER';
UNDO: 'UNDO';
UNDROP: 'UNDROP';
UNIFORM: 'UNIFORM';
UNION: 'UNION';
UNIQUE: 'UNIQUE';
UNISTR: 'UNISTR';
UNLIMITED: 'UNLIMITED';
UNLOAD: 'UNLOAD';
UNLOCK: 'UNLOCK';
UNMATCHED: 'UNMATCHED';
UNNEST_INNERJ_DISTINCT_VIEW: 'UNNEST_INNERJ_DISTINCT_VIEW';
UNNEST_NOSEMIJ_NODISTINCTVIEW: 'UNNEST_NOSEMIJ_NODISTINCTVIEW';
UNNEST_SEMIJ_VIEW: 'UNNEST_SEMIJ_VIEW';
UNNEST: 'UNNEST';
UNPACKED: 'UNPACKED';
UNPIVOT: 'UNPIVOT';
UNPLUG: 'UNPLUG';
UNPROTECTED: 'UNPROTECTED';
UNQUIESCE: 'UNQUIESCE';
UNRECOVERABLE: 'UNRECOVERABLE';
UNRESTRICTED: 'UNRESTRICTED';
UNSUBSCRIBE: 'UNSUBSCRIBE';
UNTIL: 'UNTIL';
UNUSABLE: 'UNUSABLE';
UNUSED: 'UNUSED';
UPDATABLE: 'UPDATABLE';
UPDATED: 'UPDATED';
UPDATE: 'UPDATE';
UPDATEXML: 'UPDATEXML';
UPD_INDEXES: 'UPD_INDEXES';
UPD_JOININDEX: 'UPD_JOININDEX';
UPGRADE: 'UPGRADE';
UPPER: 'UPPER';
UPSERT: 'UPSERT';
UROWID: 'UROWID';
USABLE: 'USABLE';
USAGE: 'USAGE';
USE_ANTI: 'USE_ANTI';
USE_CONCAT: 'USE_CONCAT';
USE_CUBE: 'USE_CUBE';
USE_HASH_AGGREGATION: 'USE_HASH_AGGREGATION';
USE_HASH_GBY_FOR_PUSHDOWN: 'USE_HASH_GBY_FOR_PUSHDOWN';
USE_HASH: 'USE_HASH';
USE_HIDDEN_PARTITIONS: 'USE_HIDDEN_PARTITIONS';
USE_INVISIBLE_INDEXES: 'USE_INVISIBLE_INDEXES';
USE_MERGE_CARTESIAN: 'USE_MERGE_CARTESIAN';
USE_MERGE: 'USE_MERGE';
USE_NL: 'USE_NL';
USE_NL_WITH_INDEX: 'USE_NL_WITH_INDEX';
USE_PRIVATE_OUTLINES: 'USE_PRIVATE_OUTLINES';
USER_DATA: 'USER_DATA';
USER_DEFINED: 'USER_DEFINED';
USERENV: 'USERENV';
USERGROUP: 'USERGROUP';
USER_RECYCLEBIN: 'USER_RECYCLEBIN';
USERS: 'USERS';
USER_TABLESPACES: 'USER_TABLESPACES';
USER: 'USER';
USE_SEMI: 'USE_SEMI';
USE_STORED_OUTLINES: 'USE_STORED_OUTLINES';
USE_TTT_FOR_GSETS: 'USE_TTT_FOR_GSETS';
USE: 'USE';
USE_VECTOR_AGGREGATION: 'USE_VECTOR_AGGREGATION';
USE_WEAK_NAME_RESL: 'USE_WEAK_NAME_RESL';
USING_NO_EXPAND: 'USING_NO_EXPAND';
USING: 'USING';
UTF16BE: 'UTF16BE';
UTF16LE: 'UTF16LE';
UTF32: 'UTF32';
UTF8: 'UTF8';
V1: 'V1';
V2: 'V2';
VALIDATE: 'VALIDATE';
VALIDATION: 'VALIDATION';
VALID_TIME_END: 'VALID_TIME_END';
VALUES: 'VALUES';
VALUE: 'VALUE';
VARCHAR2: 'VARCHAR2';
VARCHAR: 'VARCHAR';
VARIABLE: 'VARIABLE';
VAR_POP: 'VAR_POP';
VARRAYS: 'VARRAYS';
VARRAY: 'VARRAY';
VAR_SAMP: 'VAR_SAMP';
VARYING: 'VARYING';
VECTOR_READ_TRACE: 'VECTOR_READ_TRACE';
VECTOR_READ: 'VECTOR_READ';
VECTOR_TRANSFORM_DIMS: 'VECTOR_TRANSFORM_DIMS';
VECTOR_TRANSFORM_FACT: 'VECTOR_TRANSFORM_FACT';
VECTOR_TRANSFORM: 'VECTOR_TRANSFORM';
VERIFIER: 'VERIFIER';
VERIFY: 'VERIFY';
VERSIONING: 'VERSIONING';
VERSIONS_ENDSCN: 'VERSIONS_ENDSCN';
VERSIONS_ENDTIME: 'VERSIONS_ENDTIME';
VERSIONS_OPERATION: 'VERSIONS_OPERATION';
VERSIONS_STARTSCN: 'VERSIONS_STARTSCN';
VERSIONS_STARTTIME: 'VERSIONS_STARTTIME';
VERSIONS: 'VERSIONS';
VERSIONS_XID: 'VERSIONS_XID';
VERSION: 'VERSION';
VIEW: 'VIEW';
VIOLATION: 'VIOLATION';
VIRTUAL: 'VIRTUAL';
VISIBILITY: 'VISIBILITY';
VISIBLE: 'VISIBLE';
VOLUME: 'VOLUME';
VSIZE: 'VSIZE';
WAIT: 'WAIT';
WALLET: 'WALLET';
WARNING: 'WARNING';
WEEKS: 'WEEKS';
WEEK: 'WEEK';
WELLFORMED: 'WELLFORMED';
WHENEVER: 'WHENEVER';
WHEN: 'WHEN';
WHERE: 'WHERE';
WHILE: 'WHILE';
WHITESPACE: 'WHITESPACE';
WIDTH_BUCKET: 'WIDTH_BUCKET';
WITHIN: 'WITHIN';
WITHOUT: 'WITHOUT';
WITH_PLSQL: 'WITH_PLSQL';
WITH: 'WITH';
WORK: 'WORK';
WRAPPED: 'WRAPPED';
WRAPPER: 'WRAPPER';
WRITE: 'WRITE';
XDB_FASTPATH_INSERT: 'XDB_FASTPATH_INSERT';
XDB: 'XDB';
X_DYN_PRUNE: 'X_DYN_PRUNE';
XID: 'XID';
XML2OBJECT: 'XML2OBJECT';
XMLAGG: 'XMLAGG';
XMLATTRIBUTES: 'XMLATTRIBUTES';
XMLCAST: 'XMLCAST';
XMLCDATA: 'XMLCDATA';
XMLCOLATTVAL: 'XMLCOLATTVAL';
XMLCOMMENT: 'XMLCOMMENT';
XMLCONCAT: 'XMLCONCAT';
XMLDIFF: 'XMLDIFF';
XML_DML_RWT_STMT: 'XML_DML_RWT_STMT';
XMLELEMENT: 'XMLELEMENT';
XMLEXISTS2: 'XMLEXISTS2';
XMLEXISTS: 'XMLEXISTS';
XMLFOREST: 'XMLFOREST';
XMLINDEX: 'XMLINDEX';
XMLINDEX_REWRITE_IN_SELECT: 'XMLINDEX_REWRITE_IN_SELECT';
XMLINDEX_REWRITE: 'XMLINDEX_REWRITE';
XMLINDEX_SEL_IDX_TBL: 'XMLINDEX_SEL_IDX_TBL';
XMLISNODE: 'XMLISNODE';
XMLISVALID: 'XMLISVALID';
XMLNAMESPACES: 'XMLNAMESPACES';
XMLPARSE: 'XMLPARSE';
XMLPATCH: 'XMLPATCH';
XMLPI: 'XMLPI';
XMLQUERYVAL: 'XMLQUERYVAL';
XMLQUERY: 'XMLQUERY';
XMLROOT: 'XMLROOT';
XMLSCHEMA: 'XMLSCHEMA';
XMLSERIALIZE: 'XMLSERIALIZE';
XMLTABLE: 'XMLTABLE';
XMLTRANSFORMBLOB: 'XMLTRANSFORMBLOB';
XMLTRANSFORM: 'XMLTRANSFORM';
XMLTYPE: 'XMLTYPE';
XML: 'XML';
XPATHTABLE: 'XPATHTABLE';
XS_SYS_CONTEXT: 'XS_SYS_CONTEXT';
XS: 'XS';
YEARS: 'YEARS';
YEAR: 'YEAR';
YES: 'YES';
YMINTERVAL_UNCONSTRAINED: 'YMINTERVAL_UNCONSTRAINED';
ZONEMAP: 'ZONEMAP';
ZONE: 'ZONE';
PREDICTION: 'PREDICTION';
PREDICTION_BOUNDS: 'PREDICTION_BOUNDS';
PREDICTION_COST: 'PREDICTION_COST';
PREDICTION_DETAILS: 'PREDICTION_DETAILS';
PREDICTION_PROBABILITY: 'PREDICTION_PROBABILITY';
PREDICTION_SET: 'PREDICTION_SET';
CUME_DIST: 'CUME_DIST';
DENSE_RANK: 'DENSE_RANK';
LISTAGG: 'LISTAGG';
PERCENT_RANK: 'PERCENT_RANK';
PERCENTILE_CONT: 'PERCENTILE_CONT';
PERCENTILE_DISC: 'PERCENTILE_DISC';
RANK: 'RANK';
AVG: 'AVG';
CORR: 'CORR';
COVAR_: 'COVAR_';
DECODE: 'DECODE';
LAG: 'LAG';
LEAD: 'LEAD';
MAX: 'MAX';
MEDIAN: 'MEDIAN';
MIN: 'MIN';
NTILE: 'NTILE';
NVL: 'NVL';
RATIO_TO_REPORT: 'RATIO_TO_REPORT';
REGR_: 'REGR_';
ROUND: 'ROUND';
ROW_NUMBER: 'ROW_NUMBER';
SUBSTR: 'SUBSTR';
TO_CHAR: 'TO_CHAR';
TRIM: 'TRIM';
SUM: 'SUM';
STDDEV: 'STDDEV';
VAR_: 'VAR_';
VARIANCE: 'VARIANCE';
LEAST: 'LEAST';
GREATEST: 'GREATEST';
TO_DATE: 'TO_DATE';
// Rule #358 <NATIONAL_CHAR_STRING_LIT> - subtoken typecast in <REGULAR_ID>, it also incorporates <character_representation>
// Lowercase 'n' is a usual addition to the standard
NATIONAL_CHAR_STRING_LIT: 'N' '\'' (~('\'' | '\r' | '\n' ) | '\'' '\'' | NEWLINE)* '\'';
// Rule #040 <BIT_STRING_LIT> - subtoken typecast in <REGULAR_ID>
// Lowercase 'b' is a usual addition to the standard
BIT_STRING_LIT: 'B' ('\'' [01]* '\'')+;
// Rule #284 <HEX_STRING_LIT> - subtoken typecast in <REGULAR_ID>
// Lowercase 'x' is a usual addition to the standard
HEX_STRING_LIT: 'X' ('\'' [A-F0-9]* '\'')+;
DOUBLE_PERIOD: '..';
PERIOD: '.';
//{ Rule #238 <EXACT_NUM_LIT>
// This rule is a bit tricky - it resolves the ambiguity with <PERIOD>
// It also incorporates <mantisa> and <exponent> for the <APPROXIMATE_NUM_LIT>
// Rule #501 <signed_integer> was incorporated directly in the token <APPROXIMATE_NUM_LIT>
// See also the rule #617 <unsigned_num_lit>
/*
: (
UNSIGNED_INTEGER
( '.' UNSIGNED_INTEGER
| {$type = UNSIGNED_INTEGER;}
) ( E ('+' | '-')? UNSIGNED_INTEGER {$type = APPROXIMATE_NUM_LIT;} )?
| '.' UNSIGNED_INTEGER ( E ('+' | '-')? UNSIGNED_INTEGER {$type = APPROXIMATE_NUM_LIT;} )?
)
(D | F)?
;*/
UNSIGNED_INTEGER: [0-9]+;
APPROXIMATE_NUM_LIT: FLOAT_FRAGMENT ('E' ('+'|'-')? (FLOAT_FRAGMENT | [0-9]+))? ('D' | 'F')?;
// Rule #--- <CHAR_STRING> is a base for Rule #065 <char_string_lit> , it incorporates <character_representation>
// and a superfluous subtoken typecasting of the "QUOTE"
CHAR_STRING: '\'' (~('\'' | '\r' | '\n') | '\'' '\'' | NEWLINE)* '\'';
// Perl-style quoted string, see Oracle SQL reference, chapter String Literals
CHAR_STRING_PERL : 'Q' ( QS_ANGLE | QS_BRACE | QS_BRACK | QS_PAREN) -> type(CHAR_STRING);
fragment QUOTE : '\'' ;
fragment QS_ANGLE : QUOTE '<' .*? '>' QUOTE ;
fragment QS_BRACE : QUOTE '{' .*? '}' QUOTE ;
fragment QS_BRACK : QUOTE '[' .*? ']' QUOTE ;
fragment QS_PAREN : QUOTE '(' .*? ')' QUOTE ;
fragment QS_OTHER_CH: ~('<' | '{' | '[' | '(' | ' ' | '\t' | '\n' | '\r');
DELIMITED_ID: '"' (~('"' | '\r' | '\n') | '"' '"')+ '"' ;
// SQL_SPECIAL_CHAR was split into single rules
PERCENT: '%';
AMPERSAND: '&';
LEFT_PAREN: '(';
RIGHT_PAREN: ')';
DOUBLE_ASTERISK: '**';
ASTERISK: '*';
PLUS_SIGN: '+';
MINUS_SIGN: '-';
COMMA: ',';
SOLIDUS: '/';
AT_SIGN: '@';
ASSIGN_OP: ':=';
// See OCI reference for more information about this
BINDVAR
: ':' SIMPLE_LETTER (SIMPLE_LETTER | [0-9] | '_')*
| ':' DELIMITED_ID // not used in SQL but spotted in v$sqltext when using cursor_sharing
| ':' UNSIGNED_INTEGER
| QUESTION_MARK // not in SQL, not in Oracle, not in OCI, use this for JDBC
;
NOT_EQUAL_OP: '!='
| '<>'
| '^='
| '~='
;
CARRET_OPERATOR_PART: '^';
TILDE_OPERATOR_PART: '~';
EXCLAMATION_OPERATOR_PART: '!';
GREATER_THAN_OP: '>';
LESS_THAN_OP: '<';
COLON: ':';
SEMICOLON: ';';
fragment
QUESTION_MARK: '?';
// protected UNDERSCORE : '_' SEPARATOR ; // subtoken typecast within <INTRODUCER>
BAR: '|';
EQUALS_OP: '=';
// Rule #532 <SQL_EMBDD_LANGUAGE_CHAR> was split into single rules:
LEFT_BRACKET: '[';
RIGHT_BRACKET: ']';
//{ Rule #319 <INTRODUCER>
INTRODUCER
: '_' //(SEPARATOR {$type = UNDERSCORE;})?
;
//{ Rule #479 <SEPARATOR>
// It was originally a protected rule set to be filtered out but the <COMMENT> and <'-'> clashed.
/*SEPARATOR
: '-' -> type('-')
| COMMENT -> channel(HIDDEN)
| (SPACE | NEWLINE)+ -> channel(HIDDEN)
;*/
//}
SPACES: [ \t\r\n]+ -> skip;
// Rule #504 <SIMPLE_LETTER> - simple_latin _letter was generalised into SIMPLE_LETTER
// Unicode is yet to be implemented - see NSF0
fragment
SIMPLE_LETTER
: [A-Z]
;
fragment
FLOAT_FRAGMENT
: UNSIGNED_INTEGER* '.'? UNSIGNED_INTEGER+
;
// Rule #097 <COMMENT>
SINGLE_LINE_COMMENT: '--' ~('\r' | '\n')* (NEWLINE | EOF) -> channel(HIDDEN);
MULTI_LINE_COMMENT: '/*' .*? '*/' -> channel(HIDDEN);
// SQL*Plus prompt
// TODO should be grammar rule, but tricky to implement
PROMPT
: 'prompt' SPACE ( ~('\r' | '\n') )* (NEWLINE|EOF)
;
START_CMD
// TODO When using full word START there is a conflict with START WITH in sequences and CONNECT BY queries
// 'start' SPACE ( ~( '\r' | '\n') )* (NEWLINE|EOF)
: 'sta' SPACE ( ~('\r' | '\n') )* (NEWLINE|EOF)
// TODO Single @ conflicts with a database link name, like employees@remote
// | '@' ( ~('\r' | '\n') )* (NEWLINE|EOF)
| '@@' ( ~('\r' | '\n') )* (NEWLINE|EOF)
;
fragment
NEWLINE: '\r'? '\n';
fragment
SPACE: [ \t];
//{ Rule #442 <REGULAR_ID> additionally encapsulates a few STRING_LITs.
// Within testLiterals all reserved and non-reserved words are being resolved
REGULAR_ID: SIMPLE_LETTER (SIMPLE_LETTER | '$' | '_' | '#' | [0-9])*;
ZV: '@!' -> channel(HIDDEN);

View File

@ -0,0 +1,6661 @@
/**
* Oracle(c) PL/SQL 11g Parser
*
* Copyright (c) 2009-2011 Alexandre Porcelli <alexandre.porcelli@gmail.com>
* Copyright (c) 2015-2017 Ivan Kochurkin (KvanTTT, kvanttt@gmail.com, Positive Technologies).
* Copyright (c) 2017 Mark Adams <madams51703@gmail.com>
*
* 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.
*/
parser grammar PlSqlParser;
options { tokenVocab=PlSqlLexer; }
@members {boolean version12=true;}
sql_script
: ((unit_statement | sql_plus_command) SEMICOLON?)* EOF
;
unit_statement
: transaction_control_statements
| alter_cluster
| alter_function
| alter_package
| alter_procedure
| alter_sequence
| alter_trigger
| alter_type
| alter_table
| alter_tablespace
| alter_index
| alter_library
| alter_materialized_view
| alter_materialized_view_log
| alter_user
| alter_view
| analyze
| associate_statistics
| audit_traditional
| unified_auditing
| create_function_body
| create_procedure_body
| create_package
| create_package_body
| create_index
| create_table
| create_tablespace
| create_cluster
| create_context
| create_view //TODO
| create_directory
| create_materialized_view
| create_materialized_view_log
| create_user
| create_sequence
| create_trigger
| create_type
| create_synonym
| drop_function
| drop_package
| drop_procedure
| drop_sequence
| drop_trigger
| drop_type
| data_manipulation_language_statements
| drop_table
| drop_index
| comment_on_column
| comment_on_table
| anonymous_block
| grant_statement
;
// DDL -> SQL Statements for Stored PL/SQL Units
// Function DDLs
drop_function
: DROP FUNCTION function_name ';'
;
alter_function
: ALTER FUNCTION function_name COMPILE DEBUG? compiler_parameters_clause* (REUSE SETTINGS)? ';'
;
create_function_body
: CREATE (OR REPLACE)? FUNCTION function_name ('(' (','? parameter)+ ')')?
RETURN type_spec (invoker_rights_clause | parallel_enable_clause | result_cache_clause | DETERMINISTIC)*
((PIPELINED? (IS | AS) (DECLARE? seq_of_declare_specs? body | call_spec)) | (PIPELINED | AGGREGATE) USING implementation_type_name) ';'
;
// Creation Function - Specific Clauses
parallel_enable_clause
: PARALLEL_ENABLE partition_by_clause?
;
partition_by_clause
: '(' PARTITION expression BY (ANY | (HASH | RANGE | LIST) paren_column_list) streaming_clause? ')'
;
result_cache_clause
: RESULT_CACHE relies_on_part?
;
relies_on_part
: RELIES_ON '(' tableview_name (',' tableview_name)* ')'
;
streaming_clause
: (ORDER | CLUSTER) expression BY paren_column_list
;
// Package DDLs
drop_package
: DROP PACKAGE BODY? (schema_object_name '.')? package_name ';'
;
alter_package
: ALTER PACKAGE package_name COMPILE DEBUG? (PACKAGE | BODY | SPECIFICATION)? compiler_parameters_clause* (REUSE SETTINGS)? ';'
;
create_package
: CREATE (OR REPLACE)? PACKAGE (schema_object_name '.')? package_name invoker_rights_clause? (IS | AS) package_obj_spec* END package_name? ';'
;
create_package_body
: CREATE (OR REPLACE)? PACKAGE BODY (schema_object_name '.')? package_name (IS | AS) package_obj_body* (BEGIN seq_of_statements)? END package_name? ';'
;
// Create Package Specific Clauses
package_obj_spec
: pragma_declaration
| variable_declaration
| subtype_declaration
| cursor_declaration
| exception_declaration
| type_declaration
| procedure_spec
| function_spec
;
procedure_spec
: PROCEDURE identifier ('(' parameter ( ',' parameter )* ')')? ';'
;
function_spec
: FUNCTION identifier ('(' parameter ( ',' parameter)* ')')?
RETURN type_spec (DETERMINISTIC)? (RESULT_CACHE)? ';'
;
package_obj_body
: variable_declaration
| subtype_declaration
| cursor_declaration
| exception_declaration
| type_declaration
| procedure_body
| function_body
| procedure_spec
| function_spec
;
// Procedure DDLs
drop_procedure
: DROP PROCEDURE procedure_name ';'
;
alter_procedure
: ALTER PROCEDURE procedure_name COMPILE DEBUG? compiler_parameters_clause* (REUSE SETTINGS)? ';'
;
function_body
: FUNCTION identifier ('(' parameter (',' parameter)* ')')?
RETURN type_spec (invoker_rights_clause | parallel_enable_clause | result_cache_clause | DETERMINISTIC)*
((PIPELINED? (IS | AS) (DECLARE? seq_of_declare_specs? body | call_spec)) | (PIPELINED | AGGREGATE) USING implementation_type_name) ';'
;
procedure_body
: PROCEDURE identifier ('(' parameter (',' parameter)* ')')? (IS | AS)
(DECLARE? seq_of_declare_specs? body | call_spec | EXTERNAL) ';'
;
create_procedure_body
: CREATE (OR REPLACE)? PROCEDURE procedure_name ('(' parameter (',' parameter)* ')')?
invoker_rights_clause? (IS | AS)
(DECLARE? seq_of_declare_specs? body | call_spec | EXTERNAL) ';'
;
// Trigger DDLs
drop_trigger
: DROP TRIGGER trigger_name ';'
;
alter_trigger
: ALTER TRIGGER alter_trigger_name=trigger_name
((ENABLE | DISABLE) | RENAME TO rename_trigger_name=trigger_name | COMPILE DEBUG? compiler_parameters_clause* (REUSE SETTINGS)?) ';'
;
create_trigger
: CREATE ( OR REPLACE )? TRIGGER trigger_name
(simple_dml_trigger | compound_dml_trigger | non_dml_trigger)
trigger_follows_clause? (ENABLE | DISABLE)? trigger_when_clause? trigger_body ';'
;
trigger_follows_clause
: FOLLOWS trigger_name (',' trigger_name)*
;
trigger_when_clause
: WHEN '(' condition ')'
;
// Create Trigger Specific Clauses
simple_dml_trigger
: (BEFORE | AFTER | INSTEAD OF) dml_event_clause referencing_clause? for_each_row?
;
for_each_row
: FOR EACH ROW
;
compound_dml_trigger
: FOR dml_event_clause referencing_clause?
;
non_dml_trigger
: (BEFORE | AFTER) non_dml_event (OR non_dml_event)* ON (DATABASE | (schema_name '.')? SCHEMA)
;
trigger_body
: COMPOUND TRIGGER
| CALL identifier
| trigger_block
;
routine_clause
: routine_name function_argument?
;
compound_trigger_block
: COMPOUND TRIGGER seq_of_declare_specs? timing_point_section+ END trigger_name
;
timing_point_section
: bk=BEFORE STATEMENT IS trigger_block BEFORE STATEMENT ';'
| bk=BEFORE EACH ROW IS trigger_block BEFORE EACH ROW ';'
| ak=AFTER STATEMENT IS trigger_block AFTER STATEMENT ';'
| ak=AFTER EACH ROW IS trigger_block AFTER EACH ROW ';'
;
non_dml_event
: ALTER
| ANALYZE
| ASSOCIATE STATISTICS
| AUDIT
| COMMENT
| CREATE
| DISASSOCIATE STATISTICS
| DROP
| GRANT
| NOAUDIT
| RENAME
| REVOKE
| TRUNCATE
| DDL
| STARTUP
| SHUTDOWN
| DB_ROLE_CHANGE
| LOGON
| LOGOFF
| SERVERERROR
| SUSPEND
| DATABASE
| SCHEMA
| FOLLOWS
;
dml_event_clause
: dml_event_element (OR dml_event_element)* ON dml_event_nested_clause? tableview_name
;
dml_event_element
: (DELETE | INSERT | UPDATE) (OF column_list)?
;
dml_event_nested_clause
: NESTED TABLE tableview_name OF
;
referencing_clause
: REFERENCING referencing_element+
;
referencing_element
: (NEW | OLD | PARENT) column_alias
;
// DDLs
drop_type
: DROP TYPE BODY? type_name (FORCE | VALIDATE)? ';'
;
alter_type
: ALTER TYPE type_name
(compile_type_clause
| replace_type_clause
//TODO | {input.LT(2).getText().equalsIgnoreCase("attribute")}? alter_attribute_definition
| alter_method_spec
| alter_collection_clauses
| modifier_clause
) dependent_handling_clause? ';'
;
// Alter Type Specific Clauses
compile_type_clause
: COMPILE DEBUG? (SPECIFICATION | BODY)? compiler_parameters_clause* (REUSE SETTINGS)?
;
replace_type_clause
: REPLACE invoker_rights_clause? AS OBJECT '(' object_member_spec (',' object_member_spec)* ')'
;
alter_method_spec
: alter_method_element (',' alter_method_element)*
;
alter_method_element
: (ADD | DROP) (map_order_function_spec | subprogram_spec)
;
alter_attribute_definition
: (ADD | MODIFY | DROP) ATTRIBUTE (attribute_definition | '(' attribute_definition (',' attribute_definition)* ')')
;
attribute_definition
: attribute_name type_spec?
;
alter_collection_clauses
: MODIFY (LIMIT expression | ELEMENT TYPE type_spec)
;
dependent_handling_clause
: INVALIDATE
| CASCADE (CONVERT TO SUBSTITUTABLE | NOT? INCLUDING TABLE DATA)? dependent_exceptions_part?
;
dependent_exceptions_part
: FORCE? EXCEPTIONS INTO tableview_name
;
create_type
: CREATE (OR REPLACE)? TYPE (type_definition | type_body) ';'
;
// Create Type Specific Clauses
type_definition
: type_name (OID CHAR_STRING)? object_type_def?
;
object_type_def
: invoker_rights_clause? (object_as_part | object_under_part) sqlj_object_type?
('(' object_member_spec (',' object_member_spec)* ')')? modifier_clause*
;
object_as_part
: (IS | AS) (OBJECT | varray_type_def | nested_table_type_def)
;
object_under_part
: UNDER type_spec
;
nested_table_type_def
: TABLE OF type_spec (NOT NULL)?
;
sqlj_object_type
: EXTERNAL NAME expression LANGUAGE JAVA USING (SQLDATA | CUSTOMDATUM | ORADATA)
;
type_body
: BODY type_name (IS | AS) (type_body_elements)+ END
;
type_body_elements
: map_order_func_declaration
| subprog_decl_in_type
;
map_order_func_declaration
: (MAP | ORDER) MEMBER func_decl_in_type
;
subprog_decl_in_type
: (MEMBER | STATIC) (proc_decl_in_type | func_decl_in_type | constructor_declaration)
;
proc_decl_in_type
: PROCEDURE procedure_name '(' type_elements_parameter (',' type_elements_parameter)* ')'
(IS | AS) (call_spec | DECLARE? seq_of_declare_specs? body ';')
;
func_decl_in_type
: FUNCTION function_name ('(' type_elements_parameter (',' type_elements_parameter)* ')')?
RETURN type_spec (IS | AS) (call_spec | DECLARE? seq_of_declare_specs? body ';')
;
constructor_declaration
: FINAL? INSTANTIABLE? CONSTRUCTOR FUNCTION type_spec
('(' (SELF IN OUT type_spec ',') type_elements_parameter (',' type_elements_parameter)* ')')?
RETURN SELF AS RESULT (IS | AS) (call_spec | DECLARE? seq_of_declare_specs? body ';')
;
// Common Type Clauses
modifier_clause
: NOT? (INSTANTIABLE | FINAL | OVERRIDING)
;
object_member_spec
: identifier type_spec sqlj_object_type_attr?
| element_spec
;
sqlj_object_type_attr
: EXTERNAL NAME expression
;
element_spec
: modifier_clause? element_spec_options+ (',' pragma_clause)?
;
element_spec_options
: subprogram_spec
| constructor_spec
| map_order_function_spec
;
subprogram_spec
: (MEMBER | STATIC) (type_procedure_spec | type_function_spec)
;
type_procedure_spec
: PROCEDURE procedure_name '(' type_elements_parameter (',' type_elements_parameter)* ')' ((IS | AS) call_spec)?
;
type_function_spec
: FUNCTION function_name ('(' type_elements_parameter (',' type_elements_parameter)* ')')?
RETURN (type_spec | SELF AS RESULT) ((IS | AS) call_spec | EXTERNAL VARIABLE? NAME expression)?
;
constructor_spec
: FINAL? INSTANTIABLE? CONSTRUCTOR FUNCTION
type_spec ('(' (SELF IN OUT type_spec ',') type_elements_parameter (',' type_elements_parameter)* ')')?
RETURN SELF AS RESULT ((IS | AS) call_spec)?
;
map_order_function_spec
: (MAP | ORDER) MEMBER type_function_spec
;
pragma_clause
: PRAGMA RESTRICT_REFERENCES '(' pragma_elements (',' pragma_elements)* ')'
;
pragma_elements
: identifier
| DEFAULT
;
type_elements_parameter
: parameter_name type_spec
;
// Sequence DDLs
drop_sequence
: DROP SEQUENCE sequence_name ';'
;
alter_sequence
: ALTER SEQUENCE sequence_name sequence_spec+ ';'
;
create_sequence
: CREATE SEQUENCE sequence_name (sequence_start_clause | sequence_spec)* ';'
;
// Common Sequence
sequence_spec
: INCREMENT BY UNSIGNED_INTEGER
| MAXVALUE UNSIGNED_INTEGER
| NOMAXVALUE
| MINVALUE UNSIGNED_INTEGER
| NOMINVALUE
| CYCLE
| NOCYCLE
| CACHE UNSIGNED_INTEGER
| NOCACHE
| ORDER
| NOORDER
;
sequence_start_clause
: START WITH UNSIGNED_INTEGER
;
create_index
: CREATE (UNIQUE | BITMAP)? INDEX index_name
ON (cluster_index_clause | table_index_clause | bitmap_join_index_clause)
UNUSABLE?
';'
;
cluster_index_clause
: CLUSTER cluster_name index_attributes?
;
cluster_name
: (id_expression '.')? id_expression
;
table_index_clause
: tableview_name table_alias? '(' (','? index_expr (ASC | DESC)? )+ ')'
index_properties?
;
bitmap_join_index_clause
: tableview_name '(' (','? (tableview_name | table_alias)? column_name (ASC | DESC)? )+ ')'
FROM (','? tableview_name table_alias)+
where_clause local_partitioned_index? index_attributes?
;
index_expr
: column_name
| expression
;
index_properties
: (global_partitioned_index | local_partitioned_index | index_attributes)+
| INDEXTYPE IS (domain_index_clause | xmlindex_clause)
;
domain_index_clause
: indextype local_domain_index_clause? parallel_clause? (PARAMETERS '(' odci_parameters ')' )?
;
local_domain_index_clause
: LOCAL ('(' (','? PARTITION partition_name (PARAMETERS '(' odci_parameters ')' )? )+ ')' )?
;
xmlindex_clause
: (XDB '.')? XMLINDEX local_xmlindex_clause?
parallel_clause? //TODO xmlindex_parameters_clause?
;
local_xmlindex_clause
: LOCAL ('(' (','? PARTITION partition_name //TODO xmlindex_parameters_clause?
)+ ')')?
;
global_partitioned_index
: GLOBAL PARTITION BY (RANGE '(' (','? column_name)+ ')' '(' index_partitioning_clause ')'
| HASH '(' (','? column_name)+ ')'
(individual_hash_partitions
| hash_partitions_by_quantity
)
)
;
index_partitioning_clause
: PARTITION partition_name? VALUES LESS THAN '(' (','? literal)+ ')'
segment_attributes_clause?
;
local_partitioned_index
: LOCAL (on_range_partitioned_table
| on_list_partitioned_table
| on_hash_partitioned_table
| on_comp_partitioned_table
)?
;
on_range_partitioned_table
: '(' (','? PARTITION partition_name?
((segment_attributes_clause | key_compression)+ )?
UNUSABLE? )+
')'
;
on_list_partitioned_table
: '(' (','? PARTITION partition_name?
((segment_attributes_clause | key_compression)+ )?
UNUSABLE? )+
')'
;
on_hash_partitioned_table
: STORE IN '(' (','? tablespace)+ ')'
| '(' (','? PARTITION partition_name? (TABLESPACE tablespace)?
key_compression? UNUSABLE?)+
')'
;
on_comp_partitioned_table
: (STORE IN '(' (','? tablespace)+ ')' )?
'(' (','? PARTITION partition_name?
((segment_attributes_clause | key_compression)+)?
UNUSABLE index_subpartition_clause? )+
')'
;
index_subpartition_clause
: STORE IN '(' (','? tablespace)+ ')'
| '(' (','? SUBPARTITION subpartition_name? (TABLESPACE tablespace)?
key_compression? UNUSABLE?)+
')'
;
odci_parameters
: CHAR_STRING
;
indextype
: (id_expression '.')? id_expression
;
//https://docs.oracle.com/cd/E11882_01/server.112/e41084/statements_1010.htm#SQLRF00805
alter_index
: ALTER INDEX index_name (alter_index_ops_set1 | alter_index_ops_set2) ';'
;
alter_index_ops_set1
: ( deallocate_unused_clause
| allocate_extent_clause
| shrink_clause
| parallel_clause
| physical_attributes_clause
| logging_clause
)+
;
alter_index_ops_set2
: rebuild_clause
| PARAMETERS '(' odci_parameters ')'
| COMPILE
| enable_or_disable
| UNUSABLE
| visible_or_invisible
| RENAME TO new_index_name
| COALESCE
| monitoring_nomonitoring USAGE
| UPDATE BLOCK REFERENCES
| alter_index_partitioning
;
visible_or_invisible
: VISIBLE
| INVISIBLE
;
monitoring_nomonitoring
: MONITORING
| NOMONITORING
;
rebuild_clause
: REBUILD ( PARTITION partition_name
| SUBPARTITION subpartition_name
| REVERSE
| NOREVERSE
)?
( parallel_clause
| TABLESPACE tablespace
| PARAMETERS '(' odci_parameters ')'
//TODO | xmlindex_parameters_clause
| ONLINE
| physical_attributes_clause
| key_compression
| logging_clause
)*
;
alter_index_partitioning
: modify_index_default_attrs
| add_hash_index_partition
| modify_index_partition
| rename_index_partition
| drop_index_partition
| split_index_partition
| coalesce_index_partition
| modify_index_subpartition
;
modify_index_default_attrs
: MODIFY DEFAULT ATTRIBUTES (FOR PARTITION partition_name)?
( physical_attributes_clause
| TABLESPACE (tablespace | DEFAULT)
| logging_clause
)
;
add_hash_index_partition
: ADD PARTITION partition_name? (TABLESPACE tablespace)?
key_compression? parallel_clause?
;
coalesce_index_partition
: COALESCE PARTITION parallel_clause?
;
modify_index_partition
: MODIFY PARTITION partition_name
( modify_index_partitions_ops+
| PARAMETERS '(' odci_parameters ')'
| COALESCE
| UPDATE BLOCK REFERENCES
| UNUSABLE
)
;
modify_index_partitions_ops
: deallocate_unused_clause
| allocate_extent_clause
| physical_attributes_clause
| logging_clause
| key_compression
;
rename_index_partition
: RENAME (PARTITION partition_name | SUBPARTITION subpartition_name)
TO new_partition_name
;
drop_index_partition
: DROP PARTITION partition_name
;
split_index_partition
: SPLIT PARTITION partition_name_old AT '(' (','? literal)+ ')'
(INTO '(' index_partition_description ',' index_partition_description ')' ) ? parallel_clause?
;
index_partition_description
: PARTITION (partition_name ( (segment_attributes_clause | key_compression)+
| PARAMETERS '(' odci_parameters ')'
)
UNUSABLE?
)?
;
modify_index_subpartition
: MODIFY SUBPARTITION subpartition_name (UNUSABLE
| allocate_extent_clause
| deallocate_unused_clause
)
;
partition_name_old
: partition_name
;
new_partition_name
: partition_name
;
new_index_name
: index_name
;
create_user
: CREATE USER
user_object_name
( identified_by
| identified_other_clause
| user_tablespace_clause
| quota_clause
| profile_clause
| password_expire_clause
| user_lock_clause
| user_editions_clause
| container_clause
)+ ';'
;
// The standard clauses only permit one user per statement.
// The proxy clause allows multiple users for a proxy designation.
alter_user
: ALTER USER
user_object_name
( alter_identified_by
| identified_other_clause
| user_tablespace_clause
| quota_clause
| profile_clause
| user_default_role_clause
| password_expire_clause
| user_lock_clause
| alter_user_editions_clause
| container_clause
| container_data_clause
)+
';'
| (','? user_object_name)+ proxy_clause ';'
;
alter_identified_by
: identified_by (REPLACE id_expression)?
;
identified_by
: IDENTIFIED BY id_expression
;
identified_other_clause
: IDENTIFIED (EXTERNALLY | GLOBALLY) (AS quoted_string)?
;
user_tablespace_clause
: (DEFAULT | TEMPORARY) TABLESPACE id_expression
;
quota_clause
: QUOTA (size_clause | UNLIMITED) ON id_expression
;
profile_clause
: PROFILE id_expression
;
role_clause
: (','? role_name)+
| ALL (EXCEPT (','? role_name)+)*
;
user_default_role_clause
: DEFAULT ROLE (NONE | role_clause)
;
password_expire_clause
: PASSWORD EXPIRE
;
user_lock_clause
: ACCOUNT (LOCK | UNLOCK)
;
user_editions_clause
: ENABLE EDITIONS
;
alter_user_editions_clause
: user_editions_clause (FOR (','? regular_id)+)? FORCE?
;
proxy_clause
: REVOKE CONNECT THROUGH (ENTERPRISE USERS | user_object_name)
| GRANT CONNECT THROUGH
( ENTERPRISE USERS
| user_object_name
(WITH (NO ROLES | ROLE role_clause))?
(AUTHENTICATION REQUIRED)?
(AUTHENTICATED USING (PASSWORD | CERTIFICATE | DISTINGUISHED NAME))?
)
;
container_names
: LEFT_PAREN (','? id_expression)+ RIGHT_PAREN
;
set_container_data
: SET CONTAINER_DATA EQUALS_OP (ALL | DEFAULT | container_names)
;
add_rem_container_data
: (ADD | REMOVE) CONTAINER_DATA EQUALS_OP container_names
;
container_data_clause
: set_container_data
| add_rem_container_data (FOR container_tableview_name)?
;
// https://docs.oracle.com/cd/E11882_01/server.112/e41084/statements_4005.htm#SQLRF01105
analyze
: ( ANALYZE (TABLE tableview_name | INDEX index_name) partition_extention_clause?
| ANALYZE CLUSTER cluster_name
)
( validation_clauses
| LIST CHAINED ROWS into_clause1?
| DELETE SYSTEM? STATISTICS
)
';'
;
partition_extention_clause
: PARTITION ( '(' partition_name ')'
| FOR '(' (','? partition_key_value)+ ')'
)
| SUBPARTITION ( '(' subpartition_name ')'
| FOR '(' (','? subpartition_key_value)+ ')'
)
;
validation_clauses
: VALIDATE REF UPDATE (SET DANGLING TO NULL)?
| VALIDATE STRUCTURE
( CASCADE FAST
| CASCADE online_or_offline? into_clause?
| CASCADE
)?
online_or_offline? into_clause?
;
online_or_offline
: OFFLINE
| ONLINE
;
into_clause1
: INTO tableview_name?
;
//Making assumption on partition ad subpartition key value clauses
partition_key_value
: literal
;
subpartition_key_value
: literal
;
//https://docs.oracle.com/cd/E11882_01/server.112/e41084/statements_4006.htm#SQLRF01106
associate_statistics
: ASSOCIATE STATISTICS
WITH (column_association | function_association)
storage_table_clause?
';'
;
column_association
: COLUMNS (','? tableview_name '.' column_name)+ using_statistics_type
;
function_association
: ( FUNCTIONS (','? function_name)+
| PACKAGES (','? package_name)+
| TYPES (','? type_name)+
| INDEXES (','? index_name)+
| INDEXTYPES (','? indextype_name)+
)
( using_statistics_type
| default_cost_clause (',' default_selectivity_clause)?
| default_selectivity_clause (',' default_cost_clause)?
)
;
indextype_name
: id_expression
;
using_statistics_type
: USING (statistics_type_name | NULL)
;
statistics_type_name
: regular_id
;
default_cost_clause
: DEFAULT COST '(' cpu_cost ',' io_cost ',' network_cost ')'
;
cpu_cost
: UNSIGNED_INTEGER
;
io_cost
: UNSIGNED_INTEGER
;
network_cost
: UNSIGNED_INTEGER
;
default_selectivity_clause
: DEFAULT SELECTIVITY default_selectivity
;
default_selectivity
: UNSIGNED_INTEGER
;
storage_table_clause
: WITH (SYSTEM | USER) MANAGED STORAGE TABLES
;
// https://docs.oracle.com/database/121/SQLRF/statements_4008.htm#SQLRF56110
unified_auditing
: {version12}?
AUDIT (POLICY policy_name ((BY | EXCEPT) (','? audit_user)+ )?
(WHENEVER NOT? SUCCESSFUL)?
| CONTEXT NAMESPACE oracle_namespace
ATTRIBUTES (','? attribute_name)+ (BY (','? audit_user)+)?
)
';'
;
policy_name
: identifier
;
// https://docs.oracle.com/cd/E11882_01/server.112/e41084/statements_4007.htm#SQLRF01107
// https://docs.oracle.com/database/121/SQLRF/statements_4007.htm#SQLRF01107
audit_traditional
: AUDIT ( audit_operation_clause (auditing_by_clause | IN SESSION CURRENT)?
| audit_schema_object_clause
| NETWORK
| audit_direct_path
)
(BY (SESSION | ACCESS) )? (WHENEVER NOT? SUCCESSFUL)?
audit_container_clause?
';'
;
audit_direct_path
: {version12}? DIRECT_PATH auditing_by_clause
;
audit_container_clause
: {version12}? (CONTAINER EQUALS_OP (CURRENT | ALL))
;
audit_operation_clause
: ( (','? (sql_statement_shortcut | ALL STATEMENTS?) )+
| (','? (system_privilege | ALL PRIVILEGES) )+
)
;
auditing_by_clause
: BY (','? audit_user)+
;
audit_user
: regular_id
;
audit_schema_object_clause
: ( (','? sql_operation)+ | ALL) auditing_on_clause
;
sql_operation
: ALTER
| AUDIT
| COMMENT
| DELETE
| EXECUTE
| FLASHBACK
| GRANT
| INDEX
| INSERT
| LOCK
| READ
| RENAME
| SELECT
| UPDATE
;
auditing_on_clause
: ON ( object_name
| DIRECTORY regular_id
| MINING MODEL model_name
| {version12}? SQL TRANSLATION PROFILE profile_name
| DEFAULT
)
;
model_name
: (id_expression '.')? id_expression
;
object_name
: (id_expression '.')? id_expression
;
profile_name
: (id_expression '.')? id_expression
;
sql_statement_shortcut
: ALTER SYSTEM
| CLUSTER
| CONTEXT
| DATABASE LINK
| DIMENSION
| DIRECTORY
| INDEX
| MATERIALIZED VIEW
| NOT EXISTS
| OUTLINE
| {version12}? PLUGGABLE DATABASE
| PROCEDURE
| PROFILE
| PUBLIC DATABASE LINK
| PUBLIC SYNONYM
| ROLE
| ROLLBACK SEGMENT
| SEQUENCE
| SESSION
| SYNONYM
| SYSTEM AUDIT
| SYSTEM GRANT
| TABLE
| TABLESPACE
| TRIGGER
| TYPE
| USER
| VIEW
| ALTER SEQUENCE
| ALTER TABLE
| COMMENT TABLE
| DELETE TABLE
| EXECUTE PROCEDURE
| GRANT DIRECTORY
| GRANT PROCEDURE
| GRANT SEQUENCE
| GRANT TABLE
| GRANT TYPE
| INSERT TABLE
| LOCK TABLE
| SELECT SEQUENCE
| SELECT TABLE
| UPDATE TABLE
;
drop_index
: DROP INDEX index_name ';'
;
grant_statement
: GRANT
( ','?
(role_name
| system_privilege
| object_privilege paren_column_list?
)
)+
(ON grant_object_name)?
TO (','? grantee_name | PUBLIC)+
(WITH (ADMIN | DELEGATE) OPTION)?
(WITH HIERARCHY OPTION)?
(WITH GRANT OPTION)?
container_clause? ';'
;
container_clause
: CONTAINER EQUALS_OP (CURRENT | ALL)
;
create_directory
: CREATE (OR REPLACE)? DIRECTORY directory_name AS directory_path
';'
;
directory_name
: regular_id
;
directory_path
: CHAR_STRING
;
// https://docs.oracle.com/cd/E11882_01/appdev.112/e25519/alter_library.htm#LNPLS99946
// https://docs.oracle.com/database/121/LNPLS/alter_library.htm#LNPLS99946
alter_library
: ALTER LIBRARY library_name
( COMPILE library_debug? compiler_parameters_clause* (REUSE SETTINGS)?
| library_editionable
)
';'
;
library_editionable
: {version12}? (EDITIONABLE | NONEDITIONABLE)
;
library_debug
: {version12}? DEBUG
;
compiler_parameters_clause
: parameter_name EQUALS_OP parameter_value
;
parameter_value
: regular_id
;
library_name
: (regular_id '.')? regular_id
;
// https://docs.oracle.com/cd/E11882_01/server.112/e41084/statements_4004.htm#SQLRF01104
// https://docs.oracle.com/database/121/SQLRF/statements_4004.htm#SQLRF01104
alter_view
: ALTER VIEW tableview_name
( ADD out_of_line_constraint
| MODIFY CONSTRAINT constraint_name (RELY | NORELY)
| DROP ( CONSTRAINT constraint_name
| PRIMARY KEY
| UNIQUE '(' (','? column_name)+ ')'
)
| COMPILE
| READ (ONLY | WRITE)
| alter_view_editionable?
)
';'
;
alter_view_editionable
: {version12}? (EDITIONABLE | NONEDITIONABLE)
;
create_view
: CREATE (OR REPLACE)? (OR? FORCE)? EDITIONING? VIEW
tableview_name view_options?
AS subquery subquery_restriction_clause?
;
view_options
: view_alias_constraint
| object_view_clause
// | xmltype_view_clause //TODO
;
view_alias_constraint
: '(' ( ','? (table_alias inline_constraint* | out_of_line_constraint) )+ ')'
;
object_view_clause
: OF type_name
( WITH OBJECT (IDENTIFIER|ID|OID) ( DEFAULT | '(' (','? REGULAR_ID)+ ')' )
| UNDER tableview_name
)
( '(' ( ','? (out_of_line_constraint | REGULAR_ID inline_constraint ) )+ ')' )*
;
inline_constraint
: (CONSTRAINT constraint_name)?
( NOT? NULL
| UNIQUE
| PRIMARY KEY
| references_clause
| check_constraint
)
constraint_state?
;
inline_ref_constraint
: SCOPE IS tableview_name
| WITH ROWID
| (CONSTRAINT constraint_name)? references_clause constraint_state?
;
out_of_line_ref_constraint
: SCOPE FOR '(' ref_col_or_attr=regular_id ')' IS tableview_name
| REF '(' ref_col_or_attr=regular_id ')' WITH ROWID
| (CONSTRAINT constraint_name)? FOREIGN KEY '(' ( ','? ref_col_or_attr=regular_id)+ ')' references_clause constraint_state?
;
out_of_line_constraint
: ( (CONSTRAINT constraint_name)?
( UNIQUE '(' (','? column_name)+ ')'
| PRIMARY KEY '(' (','? column_name)+ ')'
| foreign_key_clause
| CHECK '(' expression ')'
)
)
constraint_state?
;
constraint_state
: ( NOT? DEFERRABLE
| INITIALLY (IMMEDIATE|DEFERRED)
| (RELY|NORELY)
| (ENABLE|DISABLE)
| (VALIDATE|NOVALIDATE)
| using_index_clause
)+
;
alter_tablespace
: ALTER TABLESPACE tablespace
( DEFAULT table_compression? storage_clause?
| MINIMUM EXTENT size_clause
| RESIZE size_clause
| COALESCE
| SHRINK SPACE_KEYWORD (KEEP size_clause)?
| RENAME TO new_tablespace_name
| begin_or_end BACKUP
| datafile_tempfile_clauses
| tablespace_logging_clauses
| tablespace_group_clause
| tablespace_state_clauses
| autoextend_clause
| flashback_mode_clause
| tablespace_retention_clause
)
';'
;
datafile_tempfile_clauses
: ADD (datafile_specification | tempfile_specification)
| DROP (DATAFILE | TEMPFILE) (filename | UNSIGNED_INTEGER) (KEEP size_clause)?
| SHRINK TEMPFILE (filename | UNSIGNED_INTEGER) (KEEP size_clause)?
| RENAME DATAFILE (','? filename)+ TO (','? filename)+
| (DATAFILE | TEMPFILE) (online_or_offline)
;
tablespace_logging_clauses
: logging_clause
| NO? FORCE LOGGING
;
tablespace_group_clause
: TABLESPACE GROUP (tablespace_group_name | CHAR_STRING)
;
tablespace_group_name
: regular_id
;
tablespace_state_clauses
: ONLINE
| OFFLINE (NORMAL | TEMPORARY | IMMEDIATE)?
| READ (ONLY | WRITE)
| PERMANENT
| TEMPORARY
;
flashback_mode_clause
: FLASHBACK (ON | OFF)
;
new_tablespace_name
: tablespace
;
create_tablespace
: CREATE (BIGFILE | SMALLFILE)?
( permanent_tablespace_clause
| temporary_tablespace_clause
| undo_tablespace_clause
)
';'
;
permanent_tablespace_clause
: TABLESPACE id_expression datafile_specification?
( MINIMUM EXTENT size_clause
| BLOCKSIZE size_clause
| logging_clause
| FORCE LOGGING
| (ONLINE | OFFLINE)
| ENCRYPTION tablespace_encryption_spec
| DEFAULT //TODO table_compression? storage_clause?
| extent_management_clause
| segment_management_clause
| flashback_mode_clause
)*
;
tablespace_encryption_spec
: USING encrypt_algorithm=CHAR_STRING
;
logging_clause
: LOGGING
| NOLOGGING
| FILESYSTEM_LIKE_LOGGING
;
extent_management_clause
: EXTENT MANAGEMENT LOCAL
( AUTOALLOCATE
| UNIFORM (SIZE size_clause)?
)?
;
segment_management_clause
: SEGMENT SPACE_KEYWORD MANAGEMENT (AUTO | MANUAL)
;
temporary_tablespace_clause
: TEMPORARY TABLESPACE tablespace_name=id_expression
tempfile_specification?
tablespace_group_clause? extent_management_clause?
;
undo_tablespace_clause
: UNDO TABLESPACE tablespace_name=id_expression
datafile_specification?
extent_management_clause? tablespace_retention_clause?
;
tablespace_retention_clause
: RETENTION (GUARANTEE | NOGUARANTEE)
;
datafile_specification
: DATAFILE
(','? datafile_tempfile_spec)
;
tempfile_specification
: TEMPFILE
(','? datafile_tempfile_spec)
;
datafile_tempfile_spec
: CHAR_STRING? (SIZE size_clause)? REUSE? autoextend_clause?
;
redo_log_file_spec
: DATAFILE ( CHAR_STRING
| '(' ( ','? CHAR_STRING )+ ')'
)?
(SIZE size_clause)?
(BLOCKSIZE size_clause)?
REUSE?
;
autoextend_clause
: AUTOEXTEND (OFF | ON (NEXT size_clause)? maxsize_clause? )
;
maxsize_clause
: MAXSIZE (UNLIMITED | size_clause)
;
build_clause
: BUILD (IMMEDIATE | DEFERRED)
;
parallel_clause
: NOPARALLEL
| PARALLEL parallel_count=UNSIGNED_INTEGER?
;
alter_materialized_view
: ALTER MATERIALIZED VIEW tableview_name
( physical_attributes_clause
| modify_mv_column_clause
| table_compression
| (','? lob_storage_clause)+
| (','? modify_lob_storage_clause)+
//TODO | alter_table_partitioning
| parallel_clause
| logging_clause
| allocate_extent_clause
| deallocate_unused_clause
| shrink_clause
| (cache_or_nocache)
)?
alter_iot_clauses?
(USING INDEX physical_attributes_clause)?
alter_mv_option1?
( enable_or_disable QUERY REWRITE
| COMPILE
| CONSIDER FRESH
)?
';'
;
alter_mv_option1
: alter_mv_refresh
//TODO | MODIFY scoped_table_ref_constraint
;
alter_mv_refresh
: REFRESH ( FAST
| COMPLETE
| FORCE
| ON (DEMAND | COMMIT)
| START WITH expression
| NEXT expression
| WITH PRIMARY KEY
| USING DEFAULT? MASTER ROLLBACK SEGMENT rollback_segment?
| USING (ENFORCED | TRUSTED) CONSTRAINTS
)+
;
rollback_segment
: regular_id
;
modify_mv_column_clause
: MODIFY '(' column_name (ENCRYPT encryption_spec | DECRYPT)? ')'
;
alter_materialized_view_log
: ALTER MATERIALIZED VIEW LOG FORCE? ON tableview_name
( physical_attributes_clause
| add_mv_log_column_clause
//TODO | alter_table_partitioning
| parallel_clause
| logging_clause
| allocate_extent_clause
| shrink_clause
| move_mv_log_clause
| cache_or_nocache
)?
mv_log_augmentation? mv_log_purge_clause?
';'
;
add_mv_log_column_clause
: ADD '(' column_name ')'
;
move_mv_log_clause
: MOVE segment_attributes_clause parallel_clause?
;
mv_log_augmentation
: ADD ( ( OBJECT ID
| PRIMARY KEY
| ROWID
| SEQUENCE
)
('(' (','? column_name)+ ')')?
| '(' (','? column_name)+ ')'
)
new_values_clause?
;
// Should bound this to just date/time expr
datetime_expr
: expression
;
// Should bound this to just interval expr
interval_expr
: expression
;
synchronous_or_asynchronous
: SYNCHRONOUS
| ASYNCHRONOUS
;
including_or_excluding
: INCLUDING
| EXCLUDING
;
create_materialized_view_log
: CREATE MATERIALIZED VIEW LOG ON tableview_name
( ( physical_attributes_clause
| TABLESPACE tablespace_name=id_expression
| logging_clause
| (CACHE | NOCACHE)
)+
)?
parallel_clause?
// table_partitioning_clauses TODO
( WITH
( ','?
( OBJECT ID
| PRIMARY KEY
| ROWID
| SEQUENCE
| COMMIT SCN
)
)*
('(' ( ','? regular_id )+ ')' new_values_clause? )?
mv_log_purge_clause?
)*
;
new_values_clause
: (INCLUDING | EXCLUDING ) NEW VALUES
;
mv_log_purge_clause
: PURGE
( IMMEDIATE (SYNCHRONOUS | ASYNCHRONOUS)?
// |START WITH CLAUSES TODO
)
;
create_materialized_view
: CREATE MATERIALIZED VIEW tableview_name
(OF type_name )?
//scoped_table_ref and column alias goes here TODO
( ON PREBUILT TABLE ( (WITH | WITHOUT) REDUCED PRECISION)?
| physical_properties? (CACHE | NOCACHE)? parallel_clause? build_clause?
)
( USING INDEX ( (physical_attributes_clause | TABLESPACE mv_tablespace=id_expression)+ )*
| USING NO INDEX
)?
create_mv_refresh?
(FOR UPDATE)?
( (DISABLE | ENABLE) QUERY REWRITE )?
AS subquery
';'
;
create_mv_refresh
: ( NEVER REFRESH
| REFRESH
( (FAST | COMPLETE | FORCE)
| ON (DEMAND | COMMIT)
| (START WITH | NEXT) //date goes here TODO
| WITH (PRIMARY KEY | ROWID)
| USING
( DEFAULT (MASTER | LOCAL)? ROLLBACK SEGMENT
| (MASTER | LOCAL)? ROLLBACK SEGMENT rb_segment=REGULAR_ID
)
| USING (ENFORCED | TRUSTED) CONSTRAINTS
)+
)
;
create_context
: CREATE (OR REPLACE)? CONTEXT oracle_namespace USING (schema_object_name '.')? package_name
(INITIALIZED (EXTERNALLY | GLOBALLY)
| ACCESSED GLOBALLY
)?
';'
;
oracle_namespace
: id_expression
;
//https://docs.oracle.com/cd/E11882_01/server.112/e41084/statements_5001.htm#SQLRF01201
create_cluster
: CREATE CLUSTER cluster_name '(' (','? column_name datatype SORT?)+ ')'
( physical_attributes_clause
| SIZE size_clause
| TABLESPACE tablespace
| INDEX
| (SINGLE TABLE)? HASHKEYS UNSIGNED_INTEGER (HASH IS expression)?
)*
parallel_clause? (ROWDEPENDENCIES | NOROWDEPENDENCIES)?
(CACHE | NOCACHE)?
';'
;
create_table
: CREATE (GLOBAL TEMPORARY)? TABLE tableview_name
(relational_table | object_table | xmltype_table) (AS select_statement)?
';'
;
xmltype_table
: OF XMLTYPE ('(' object_properties ')')?
(XMLTYPE xmltype_storage)? xmlschema_spec? xmltype_virtual_columns?
(ON COMMIT (DELETE | PRESERVE) ROWS)? oid_clause? oid_index_clause?
physical_properties? column_properties? table_partitioning_clauses?
(CACHE | NOCACHE)? (RESULT_CACHE '(' MODE (DEFAULT | FORCE) ')')?
parallel_clause? (ROWDEPENDENCIES | NOROWDEPENDENCIES)?
(enable_disable_clause+)? row_movement_clause?
flashback_archive_clause?
;
xmltype_virtual_columns
: VIRTUAL COLUMNS '(' (','? column_name AS '(' expression ')')+ ')'
;
xmltype_column_properties
: XMLTYPE COLUMN? column_name xmltype_storage? xmlschema_spec?
;
xmltype_storage
: STORE AS (OBJECT RELATIONAL
| (SECUREFILE | BASICFILE)? (CLOB | BINARY XML) (lob_segname ('(' lob_parameters ')')? | '(' lob_parameters ')')?
)
| STORE VARRAYS AS (LOBS | TABLES)
;
xmlschema_spec
: (XMLSCHEMA DELIMITED_ID)? ELEMENT DELIMITED_ID
(allow_or_disallow NONSCHEMA)?
(allow_or_disallow ANYSCHEMA)?
;
object_table
: OF type_name object_table_substitution?
('(' (','? object_properties)+ ')')?
(ON COMMIT (DELETE | PRESERVE) ROWS)? oid_clause? oid_index_clause?
physical_properties? column_properties? table_partitioning_clauses?
(CACHE | NOCACHE)? (RESULT_CACHE '(' MODE (DEFAULT | FORCE) ')')?
parallel_clause? (ROWDEPENDENCIES | NOROWDEPENDENCIES)?
(enable_disable_clause+)? row_movement_clause? flashback_archive_clause?
;
oid_index_clause
: OIDINDEX index_name? '(' (physical_attributes_clause | TABLESPACE tablespace)+ ')'
;
oid_clause
: OBJECT IDENTIFIER IS (SYSTEM GENERATED | PRIMARY KEY)
;
object_properties
: (column_name | attribute_name) (DEFAULT expression)? ((','? inline_constraint)+ | inline_ref_constraint)?
| out_of_line_constraint
| out_of_line_ref_constraint
| supplemental_logging_props
;
object_table_substitution
: NOT? SUBSTITUTABLE AT ALL LEVELS
;
relational_table
: ('(' relational_properties ')')?
(ON COMMIT (DELETE | PRESERVE) ROWS)?
physical_properties? column_properties? table_partitioning_clauses?
(CACHE | NOCACHE)? (RESULT_CACHE '(' MODE (DEFAULT | FORCE) ')')?
parallel_clause?
(ROWDEPENDENCIES | NOROWDEPENDENCIES)?
(enable_disable_clause+)? row_movement_clause? flashback_archive_clause?
;
relational_properties
: (','? (column_definition
| virtual_column_definition
| out_of_line_constraint
| out_of_line_ref_constraint
| supplemental_logging_props
)
)+
;
table_partitioning_clauses
: range_partitions
| list_partitions
| hash_partitions
| composite_range_partitions
| composite_list_partitions
| composite_hash_partitions
| reference_partitioning
| system_partitioning
;
range_partitions
: PARTITION BY RANGE '(' (','? column_name)+ ')'
(INTERVAL '(' expression ')' (STORE IN '(' (','? tablespace)+ ')' )? )?
'(' (','? PARTITION partition_name? range_values_clause table_partition_description)+ ')'
;
list_partitions
: PARTITION BY LIST '(' column_name ')'
'(' (','? PARTITION partition_name? list_values_clause table_partition_description )+ ')'
;
hash_partitions
: PARTITION BY HASH '(' (','? column_name)+ ')'
(individual_hash_partitions | hash_partitions_by_quantity)
;
individual_hash_partitions
: '(' (','? PARTITION partition_name? partitioning_storage_clause?)+ ')'
;
hash_partitions_by_quantity
: PARTITIONS hash_partition_quantity
(STORE IN '(' (','? tablespace)+ ')')?
(table_compression | key_compression)?
(OVERFLOW STORE IN '(' (','? tablespace)+ ')' )?
;
hash_partition_quantity
: UNSIGNED_INTEGER
;
composite_range_partitions
: PARTITION BY RANGE '(' (','? column_name)+ ')'
(INTERVAL '(' expression ')' (STORE IN '(' (','? tablespace)+ ')' )? )?
(subpartition_by_range | subpartition_by_list | subpartition_by_hash)
'(' (','? range_partition_desc)+ ')'
;
composite_list_partitions
: PARTITION BY LIST '(' column_name ')'
(subpartition_by_range | subpartition_by_list | subpartition_by_hash)
'(' (','? list_partition_desc)+ ')'
;
composite_hash_partitions
: PARTITION BY HASH '(' (',' column_name)+ ')'
(subpartition_by_range | subpartition_by_list | subpartition_by_hash)
(individual_hash_partitions | hash_partitions_by_quantity)
;
reference_partitioning
: PARTITION BY REFERENCE '(' regular_id ')'
('(' (','? reference_partition_desc)+ ')')?
;
reference_partition_desc
: PARTITION partition_name? table_partition_description
;
system_partitioning
: PARTITION BY SYSTEM
(PARTITIONS UNSIGNED_INTEGER | (','? reference_partition_desc)+)?
;
range_partition_desc
: PARTITION partition_name? range_values_clause table_partition_description
( ( '(' ( (','? range_subpartition_desc)+
| (','? list_subpartition_desc)+
| (','? individual_hash_subparts)+
)
')'
| hash_subparts_by_quantity
)
)?
;
list_partition_desc
: PARTITION partition_name? list_values_clause table_partition_description
( ( '(' ( (','? range_subpartition_desc)+
| (','? list_subpartition_desc)+
| (','? individual_hash_subparts)+
)
')'
| hash_subparts_by_quantity
)
)?
;
subpartition_template
: SUBPARTITION TEMPLATE
( ( '(' ( (','? range_subpartition_desc)+
| (','? list_subpartition_desc)+
| (','? individual_hash_subparts)+
)
')'
| hash_subpartition_quantity
)
)
;
hash_subpartition_quantity
: UNSIGNED_INTEGER
;
subpartition_by_range
: SUBPARTITION BY RANGE '(' (','? column_name)+ ')' subpartition_template?
;
subpartition_by_list
: SUBPARTITION BY LIST '(' column_name ')' subpartition_template?
;
subpartition_by_hash
: SUBPARTITION BY HASH '(' (','? column_name)+ ')'
(SUBPARTITIONS UNSIGNED_INTEGER (STORE IN '(' (','? tablespace)+ ')' )?
| subpartition_template
)?
;
subpartition_name
: partition_name
;
range_subpartition_desc
: SUBPARTITION subpartition_name? range_values_clause partitioning_storage_clause?
;
list_subpartition_desc
: SUBPARTITION subpartition_name? list_values_clause partitioning_storage_clause?
;
individual_hash_subparts
: SUBPARTITION subpartition_name? partitioning_storage_clause?
;
hash_subparts_by_quantity
: SUBPARTITIONS UNSIGNED_INTEGER (STORE IN '(' (','? tablespace)+ ')' )?
;
range_values_clause
: VALUES LESS THAN '(' (','? literal)+ ')'
;
list_values_clause
: VALUES '(' ((','? literal)+ | DEFAULT) ')'
;
table_partition_description
: deferred_segment_creation? segment_attributes_clause?
(table_compression | key_compression)?
(OVERFLOW segment_attributes_clause? )?
(lob_storage_clause | varray_col_properties | nested_table_col_properties)?
;
partitioning_storage_clause
: ( TABLESPACE tablespace
| OVERFLOW (TABLESPACE tablespace)?
| table_compression
| key_compression
| lob_partitioning_storage
| VARRAY varray_item STORE AS (BASICFILE | SECUREFILE)? LOB lob_segname
)+
;
lob_partitioning_storage
: LOB '(' lob_item ')'
STORE AS (BASICFILE | SECUREFILE)?
(lob_segname ('(' TABLESPACE tablespace ')' )?
| '(' TABLESPACE tablespace ')'
)
;
datatype_null_enable
: column_name datatype
SORT? (DEFAULT expression)? (ENCRYPT ( USING CHAR_STRING )? (IDENTIFIED BY REGULAR_ID)? CHAR_STRING? ( NO? SALT )? )?
(NOT NULL)? (ENABLE | DISABLE)?
;
//Technically, this should only allow 'K' | 'M' | 'G' | 'T' | 'P' | 'E'
// but having issues with examples/numbers01.sql line 11 "sysdate -1m"
size_clause
: UNSIGNED_INTEGER REGULAR_ID?
;
table_compression
: COMPRESS
( BASIC
| FOR ( OLTP
| (QUERY | ARCHIVE) (LOW | HIGH)?
)
)?
| NOCOMPRESS
;
physical_attributes_clause
: (PCTFREE pctfree=UNSIGNED_INTEGER
| PCTUSED pctused=UNSIGNED_INTEGER
| INITRANS inittrans=UNSIGNED_INTEGER
| storage_clause
)+
;
storage_clause
: STORAGE '('
(INITIAL initial_size=size_clause
| NEXT next_size=size_clause
| MINEXTENTS minextents=(UNSIGNED_INTEGER | UNLIMITED)
| MAXEXTENTS minextents=(UNSIGNED_INTEGER | UNLIMITED)
| PCTINCREASE pctincrease=UNSIGNED_INTEGER
| FREELISTS freelists=UNSIGNED_INTEGER
| FREELIST GROUPS freelist_groups=UNSIGNED_INTEGER
| OPTIMAL (size_clause | NULL )
| BUFFER_POOL (KEEP | RECYCLE | DEFAULT)
| FLASH_CACHE (KEEP | NONE | DEFAULT)
| ENCRYPT
)+
')'
;
deferred_segment_creation
: SEGMENT CREATION (IMMEDIATE | DEFERRED)
;
segment_attributes_clause
: ( physical_attributes_clause
| TABLESPACE tablespace_name=id_expression
| logging_clause
)+
;
physical_properties
: deferred_segment_creation? segment_attributes_clause table_compression?
;
row_movement_clause
: (ENABLE | DISABLE)? ROW MOVEMENT
;
flashback_archive_clause
: FLASHBACK ARCHIVE flashback_archive=REGULAR_ID
| NO FLASHBACK ARCHIVE
;
log_grp
: UNSIGNED_INTEGER
;
supplemental_table_logging
: ADD
(','? SUPPLEMENTAL LOG (supplemental_log_grp_clause | supplemental_id_key_clause) )*
| DROP (','? SUPPLEMENTAL LOG (supplemental_id_key_clause | GROUP log_grp) )*
;
supplemental_log_grp_clause
: GROUP log_grp '(' (','? regular_id (NO LOG)?)+ ')' ALWAYS?
;
supplemental_id_key_clause
: DATA '('( ','? ( ALL
| PRIMARY KEY
| UNIQUE
| FOREIGN KEY
)
)+
')'
COLUMNS
;
allocate_extent_clause
: ALLOCATE EXTENT
( '(' ( SIZE size_clause
| DATAFILE datafile=CHAR_STRING
| INSTANCE inst_num=UNSIGNED_INTEGER
)+
')'
)?
;
deallocate_unused_clause
: DEALLOCATE UNUSED (KEEP size_clause)?
;
shrink_clause
: SHRINK SPACE_KEYWORD COMPACT? CASCADE?
;
records_per_block_clause
: (MINIMIZE | NOMINIMIZE)? RECORDS_PER_BLOCK
;
upgrade_table_clause
: UPGRADE (NOT? INCLUDING DATA) column_properties
;
drop_table
: DROP TABLE tableview_name PURGE? SEMICOLON
;
comment_on_column
: COMMENT ON COLUMN tableview_name PERIOD column_name IS quoted_string
;
enable_or_disable
: ENABLE
| DISABLE
;
allow_or_disallow
: ALLOW
| DISALLOW
;
// Synonym DDL Clauses
create_synonym
// Synonym's schema cannot be specified for public synonyms
: CREATE (OR REPLACE)? PUBLIC SYNONYM synonym_name FOR (schema_name PERIOD)? schema_object_name (AT_SIGN link_name)?
| CREATE (OR REPLACE)? SYNONYM (schema_name PERIOD)? synonym_name FOR (schema_name PERIOD)? schema_object_name (AT_SIGN link_name)?
;
comment_on_table
: COMMENT ON TABLE tableview_name IS quoted_string
;
alter_cluster
: ALTER CLUSTER cluster_name
( physical_attributes_clause
| SIZE size_clause
| allocate_extent_clause
| deallocate_unused_clause
| cache_or_nocache
)+
parallel_clause?
';'
;
cache_or_nocache
: CACHE
| NOCACHE
;
database_name
: regular_id
;
alter_database
: ALTER DATABASE database_name?
( startup_clauses
| recovery_clauses
| database_file_clauses
| logfile_clauses
| controlfile_clauses
| standby_database_clauses
| default_settings_clause
| instance_clauses
| security_clause
)
';'
;
startup_clauses
: MOUNT ((STANDBY | CLONE) DATABASE)?
| OPEN (READ WRITE)? resetlogs_or_noresetlogs? upgrade_or_downgrade?
| OPEN READ ONLY
;
resetlogs_or_noresetlogs
: RESETLOGS
| NORESETLOGS
;
upgrade_or_downgrade
: UPGRADE
| DOWNGRADE
;
recovery_clauses
: general_recovery
| managed_standby_recovery
| begin_or_end BACKUP
;
begin_or_end
: BEGIN
| END
;
general_recovery
: RECOVER AUTOMATIC? (FROM CHAR_STRING)?
( (full_database_recovery | partial_database_recovery | LOGFILE CHAR_STRING )+
((TEST | ALLOW UNSIGNED_INTEGER CORRUPTION | parallel_clause)+ )?
| CONTINUE DEFAULT?
| CANCEL
)
;
//Need to come back to
full_database_recovery
: STANDBY? DATABASE
((UNTIL (CANCEL |TIME CHAR_STRING | CHANGE UNSIGNED_INTEGER | CONSISTENT)
| USING BACKUP CONTROLFILE
)+
)?
;
partial_database_recovery
: TABLESPACE (','? tablespace)+
| DATAFILE (','? CHAR_STRING | filenumber)+
;
managed_standby_recovery
: RECOVER (MANAGED STANDBY DATABASE
((USING CURRENT LOGFILE
| DISCONNECT (FROM SESSION)?
| NODELAY
| UNTIL CHANGE UNSIGNED_INTEGER
| UNTIL CONSISTENT
| parallel_clause
)+
| FINISH
| CANCEL
)?
| TO LOGICAL STANDBY (db_name | KEEP IDENTITY)
)
;
db_name
: regular_id
;
database_file_clauses
: RENAME FILE (','? filename)+ TO filename
| create_datafile_clause
| alter_datafile_clause
| alter_tempfile_clause
;
create_datafile_clause
: CREATE DATAFILE (','? (filename | filenumber) )+
(AS (//TODO (','? file_specification)+ |
NEW) )?
;
alter_datafile_clause
: DATAFILE (','? filename|filenumber)+
( ONLINE
| OFFLINE (FOR DROP)?
| RESIZE size_clause
| autoextend_clause
| END BACKUP
)
;
alter_tempfile_clause
: TEMPFILE (','? filename|filenumber)+
( RESIZE size_clause
| autoextend_clause
| DROP (INCLUDING DATAFILES)
| ONLINE
| OFFLINE
)
;
logfile_clauses
: (ARCHIVELOG MANUAL? | NOARCHIVELOG)
| NO? FORCE LOGGING
| RENAME FILE (','? filename)+ TO filename
| CLEAR UNARCHIVED LOGFILE (','? logfile_descriptor)+ (UNRECOVERABLE DATAFILE)?
| add_logfile_clauses
| drop_logfile_clauses
| switch_logfile_clause
| supplemental_db_logging
;
add_logfile_clauses
: ADD STANDBY? LOGFILE
//TODO ((INSTANCE CHAR_STRING | THREAD UNSIGNED_INTEGER)?
//TODO (','? (GROUP UNSIGNED_INTEGER)? //TODO redo_logfile_spec
//TODO )+
//TODO | MEMBER (','? filename REUSE?)+ TO (','? logfile_descriptor)+
//TODO )
;
drop_logfile_clauses
: DROP STANDBY?
LOGFILE ((','? logfile_descriptor)+
| MEMBER (','? filename)+
)
;
switch_logfile_clause
: SWITCH ALL LOGFILES TO BLOCKSIZE UNSIGNED_INTEGER
;
supplemental_db_logging
: add_or_drop
SUPPLEMENTAL LOG (DATA
| supplemental_id_key_clause
| supplemental_plsql_clause
)
;
add_or_drop
: ADD
| DROP
;
supplemental_plsql_clause
: DATA FOR PROCEDURAL REPLICATION
;
logfile_descriptor
: GROUP UNSIGNED_INTEGER
| '(' (','? filename)+ ')'
| filename
;
controlfile_clauses
: CREATE (LOGICAL | PHYSICAL)? STANDBY CONTROLFILE AS filename REUSE?
| BACKUP CONTROLFILE TO (filename REUSE? | trace_file_clause)
;
trace_file_clause
: TRACE (AS filename REUSE?)? (RESETLOGS|NORESETLOGS)?
;
standby_database_clauses
: ( activate_standby_db_clause
| maximize_standby_db_clause
| register_logfile_clause
| commit_switchover_clause
| start_standby_clause
| stop_standby_clause
| convert_database_clause
)
parallel_clause?
;
activate_standby_db_clause
: ACTIVATE (PHYSICAL | LOGICAL)? STANDBY DATABASE (FINISH APPLY)?
;
maximize_standby_db_clause
: SET STANDBY DATABASE TO MAXIMIZE (PROTECTION | AVAILABILITY | PERFORMANCE)
;
register_logfile_clause
: REGISTER (OR REPLACE)? (PHYSICAL | LOGICAL) LOGFILE //TODO (','? file_specification)+
//TODO (FOR logminer_session_name)?
;
commit_switchover_clause
: (PREPARE | COMMIT) TO SWITCHOVER
((TO (((PHYSICAL | LOGICAL)? PRIMARY | PHYSICAL? STANDBY)
((WITH | WITHOUT)? SESSION SHUTDOWN (WAIT | NOWAIT) )?
| LOGICAL STANDBY
)
| LOGICAL STANDBY
)
| CANCEL
)?
;
start_standby_clause
: START LOGICAL STANDBY APPLY IMMEDIATE? NODELAY?
( NEW PRIMARY regular_id
| INITIAL scn_value=UNSIGNED_INTEGER?
| SKIP_ FAILED TRANSACTION
| FINISH
)?
;
stop_standby_clause
: (STOP | ABORT) LOGICAL STANDBY APPLY
;
convert_database_clause
: CONVERT TO (PHYSICAL | SNAPSHOT) STANDBY
;
default_settings_clause
: DEFAULT EDITION EQUALS_OP edition_name
| SET DEFAULT (BIGFILE | SMALLFILE) TABLESPACE
| DEFAULT TABLESPACE tablespace
| DEFAULT TEMPORARY TABLESPACE (tablespace | tablespace_group_name)
| RENAME GLOBAL_NAME TO database ('.' domain)+
| ENABLE BLOCK CHANGE TRACKING (USING FILE filename REUSE?)?
| DISABLE BLOCK CHANGE TRACKING
| flashback_mode_clause
| set_time_zone_clause
;
set_time_zone_clause
: SET TIMEZONE EQUALS_OP CHAR_STRING
;
instance_clauses
: enable_or_disable INSTANCE CHAR_STRING
;
security_clause
: GUARD (ALL | STANDBY | NONE)
;
domain
: regular_id
;
database
: regular_id
;
edition_name
: regular_id
;
filenumber
: UNSIGNED_INTEGER
;
filename
: CHAR_STRING
;
alter_table
: ALTER TABLE tableview_name
(
| alter_table_properties
| constraint_clauses
| column_clauses
//TODO | alter_table_partitioning
//TODO | alter_external_table
| move_table_clause
)
((enable_disable_clause | enable_or_disable (TABLE LOCK | ALL TRIGGERS) )+)?
';'
;
alter_table_properties
: alter_table_properties_1
| RENAME TO tableview_name
| shrink_clause
| READ ONLY
| READ WRITE
| REKEY CHAR_STRING
;
alter_table_properties_1
: ( physical_attributes_clause
| logging_clause
| table_compression
| supplemental_table_logging
| allocate_extent_clause
| deallocate_unused_clause
| (CACHE | NOCACHE)
| RESULT_CACHE '(' MODE (DEFAULT | FORCE) ')'
| upgrade_table_clause
| records_per_block_clause
| parallel_clause
| row_movement_clause
| flashback_archive_clause
)+
alter_iot_clauses?
;
alter_iot_clauses
: index_org_table_clause
| alter_overflow_clause
| alter_mapping_table_clause
| COALESCE
;
alter_mapping_table_clause
: MAPPING TABLE (allocate_extent_clause | deallocate_unused_clause)
;
alter_overflow_clause
: add_overflow_clause
| OVERFLOW (segment_attributes_clause | allocate_extent_clause | shrink_clause | deallocate_unused_clause)+
;
add_overflow_clause
: ADD OVERFLOW segment_attributes_clause? ('(' (','? PARTITION segment_attributes_clause?)+ ')' )?
;
enable_disable_clause
: (ENABLE | DISABLE) (VALIDATE | NOVALIDATE)?
(UNIQUE '(' (','? column_name)+ ')'
| PRIMARY KEY
| CONSTRAINT constraint_name
) using_index_clause? exceptions_clause?
CASCADE? ((KEEP | DROP) INDEX)?
;
using_index_clause
: USING INDEX (index_name | '(' create_index ')' | index_attributes )?
;
index_attributes
: ( physical_attributes_clause
| logging_clause
| TABLESPACE (tablespace | DEFAULT)
| key_compression
| sort_or_nosort
| REVERSE
| visible_or_invisible
| parallel_clause
)+
;
sort_or_nosort
: SORT
| NOSORT
;
exceptions_clause
: EXCEPTIONS INTO tableview_name
;
move_table_clause
: MOVE ONLINE? segment_attributes_clause? table_compression? index_org_table_clause? ((lob_storage_clause | varray_col_properties)+)? parallel_clause?
;
index_org_table_clause
: (mapping_table_clause | PCTTHRESHOLD UNSIGNED_INTEGER | key_compression) index_org_overflow_clause?
;
mapping_table_clause
: MAPPING TABLE
| NOMAPPING
;
key_compression
: NOCOMPRESS
| COMPRESS UNSIGNED_INTEGER
;
index_org_overflow_clause
: (INCLUDING column_name)? OVERFLOW segment_attributes_clause?
;
column_clauses
: add_modify_drop_column_clauses
| rename_column_clause
| modify_collection_retrieval
| modify_lob_storage_clause
;
modify_collection_retrieval
: MODIFY NESTED TABLE collection_item RETURN AS (LOCATOR | VALUE)
;
collection_item
: tableview_name
;
rename_column_clause
: RENAME COLUMN old_column_name TO new_column_name
;
old_column_name
: column_name
;
new_column_name
: column_name
;
add_modify_drop_column_clauses
: (add_column_clause
|modify_column_clauses
|drop_column_clause
)+
;
drop_column_clause
: SET UNUSED (COLUMN column_name| ('(' (','? column_name)+ ')' )) (CASCADE CONSTRAINTS | INVALIDATE)*
| DROP (COLUMN column_name | '(' (','? column_name)+ ')' ) (CASCADE CONSTRAINTS | INVALIDATE)* (CHECKPOINT UNSIGNED_INTEGER)?
| DROP (UNUSED COLUMNS | COLUMNS CONTINUE) (CHECKPOINT UNSIGNED_INTEGER)
;
modify_column_clauses
: MODIFY ('(' (','? modify_col_properties)+ ')'
| modify_col_substitutable
)
;
modify_col_properties
: column_name datatype? (DEFAULT expression)? (ENCRYPT encryption_spec | DECRYPT)? inline_constraint* lob_storage_clause? //TODO alter_xmlschema_clause
;
modify_col_substitutable
: COLUMN column_name NOT? SUBSTITUTABLE AT ALL LEVELS FORCE?
;
add_column_clause
: ADD '(' (','? column_definition
|','? virtual_column_definition
)+
')'
column_properties?
//TODO (','? out_of_line_part_storage )
;
alter_varray_col_properties
: MODIFY VARRAY varray_item '(' modify_lob_parameters ')'
;
varray_col_properties
: VARRAY varray_item ( substitutable_column_clause? varray_storage_clause
| substitutable_column_clause
)
;
varray_storage_clause
: STORE AS (SECUREFILE|BASICFILE)? LOB ( lob_segname? '(' lob_storage_parameters ')'
| lob_segname
)
;
lob_segname
: regular_id
;
lob_item
: regular_id
;
lob_storage_parameters
: TABLESPACE tablespace | (lob_parameters storage_clause? )
| storage_clause
;
lob_storage_clause
: LOB ( '(' (','? lob_item)+ ')' STORE AS ( (SECUREFILE|BASICFILE) | '(' lob_storage_parameters ')' )+
| '(' lob_item ')' STORE AS ( (SECUREFILE | BASICFILE) | lob_segname | '(' lob_storage_parameters ')' )+
)
;
modify_lob_storage_clause
: MODIFY LOB '(' lob_item ')' '(' modify_lob_parameters ')'
;
modify_lob_parameters
: ( storage_clause
| (PCTVERSION | FREEPOOLS) UNSIGNED_INTEGER
| REBUILD FREEPOOLS
| lob_retention_clause
| lob_deduplicate_clause
| lob_compression_clause
| ENCRYPT encryption_spec
| DECRYPT
| CACHE
| (CACHE | NOCACHE | CACHE READS) logging_clause?
| allocate_extent_clause
| shrink_clause
| deallocate_unused_clause
)+
;
lob_parameters
: ( (ENABLE | DISABLE) STORAGE IN ROW
| CHUNK UNSIGNED_INTEGER
| PCTVERSION UNSIGNED_INTEGER
| FREEPOOLS UNSIGNED_INTEGER
| lob_retention_clause
| lob_deduplicate_clause
| lob_compression_clause
| ENCRYPT encryption_spec
| DECRYPT
| (CACHE | NOCACHE | CACHE READS) logging_clause?
)+
;
lob_deduplicate_clause
: DEDUPLICATE
| KEEP_DUPLICATES
;
lob_compression_clause
: NOCOMPRESS
| COMPRESS (HIGH | MEDIUM | LOW)?
;
lob_retention_clause
: RETENTION (MAX | MIN UNSIGNED_INTEGER | AUTO | NONE)?
;
encryption_spec
: (USING CHAR_STRING)? (IDENTIFIED BY REGULAR_ID)? CHAR_STRING? (NO? SALT)?
;
tablespace
: regular_id
;
varray_item
: (id_expression '.')? (id_expression '.')? id_expression
;
column_properties
: object_type_col_properties
| nested_table_col_properties
| (varray_col_properties | lob_storage_clause) //TODO '(' ( ','? lob_partition_storage)+ ')'
| xmltype_column_properties
;
period_definition
: {version12}? PERIOD FOR column_name
( '(' start_time_column ',' end_time_column ')' )?
;
start_time_column
: column_name
;
end_time_column
: column_name
;
column_definition
: column_name (datatype | type_name)
SORT? (DEFAULT expression)? (ENCRYPT (USING CHAR_STRING)? (IDENTIFIED BY regular_id)? CHAR_STRING? (NO? SALT)? )? (inline_constraint* | inline_ref_constraint)
;
virtual_column_definition
: column_name datatype? (GENERATED ALWAYS)? AS '(' expression ')'
VIRTUAL? inline_constraint*
;
out_of_line_part_storage
: PARTITION partition_name
;
nested_table_col_properties
: NESTED TABLE (nested_item | COLUMN_VALUE) substitutable_column_clause? (LOCAL | GLOBAL)?
STORE AS tableview_name ( '(' ( '(' object_properties ')'
| physical_properties
| column_properties
)+
')'
)?
(RETURN AS? (LOCATOR | VALUE) )?
;
nested_item
: regular_id
;
substitutable_column_clause
: ELEMENT? IS OF TYPE? '(' type_name ')'
| NOT? SUBSTITUTABLE AT ALL LEVELS
;
partition_name
: regular_id
;
supplemental_logging_props
: SUPPLEMENTAL LOG (supplemental_log_grp_clause | supplemental_id_key_clause)
;
column_or_attribute
: regular_id
;
object_type_col_properties
: COLUMN column=regular_id substitutable_column_clause
;
constraint_clauses
: ADD '(' (out_of_line_constraint* | out_of_line_ref_constraint) ')'
| ADD (out_of_line_constraint* | out_of_line_ref_constraint)
| MODIFY (CONSTRAINT constraint_name | PRIMARY KEY | UNIQUE '(' (','? column_name)+ ')') constraint_state CASCADE?
| RENAME CONSTRAINT old_constraint_name TO new_constraint_name
| drop_constraint_clause+
;
old_constraint_name
: constraint_name
;
new_constraint_name
: constraint_name
;
drop_constraint_clause
: DROP drop_primary_key_or_unique_or_generic_clause
;
drop_primary_key_or_unique_or_generic_clause
: (PRIMARY KEY | UNIQUE '(' (','? column_name)+ ')') CASCADE? (KEEP | DROP)?
| CONSTRAINT constraint_name CASCADE?
;
add_constraint
: ADD (','? (CONSTRAINT constraint_name)?
( primary_key_clause
| foreign_key_clause
| unique_key_clause
| check_constraint
) )+
;
check_constraint
: CHECK '(' condition ')' DISABLE?
;
drop_constraint
: DROP CONSTRAINT constraint_name
;
enable_constraint
: ENABLE CONSTRAINT constraint_name
;
disable_constraint
: DISABLE CONSTRAINT constraint_name
;
foreign_key_clause
: FOREIGN KEY paren_column_list references_clause on_delete_clause?
;
references_clause
: REFERENCES tableview_name paren_column_list
;
on_delete_clause
: ON DELETE (CASCADE | SET NULL)
;
unique_key_clause
: UNIQUE paren_column_list using_index_clause?
;
primary_key_clause
: PRIMARY KEY paren_column_list using_index_clause?
;
// Anonymous PL/SQL code block
anonymous_block
: (DECLARE seq_of_declare_specs)? BEGIN seq_of_statements (EXCEPTION exception_handler+)? END SEMICOLON
;
// Common DDL Clauses
invoker_rights_clause
: AUTHID (CURRENT_USER | DEFINER)
;
call_spec
: LANGUAGE (java_spec | c_spec)
;
// Call Spec Specific Clauses
java_spec
: JAVA NAME CHAR_STRING
;
c_spec
: C_LETTER (NAME CHAR_STRING)? LIBRARY identifier c_agent_in_clause? (WITH CONTEXT)? c_parameters_clause?
;
c_agent_in_clause
: AGENT IN '(' expressions ')'
;
c_parameters_clause
: PARAMETERS '(' (expressions | '.' '.' '.') ')'
;
parameter
: parameter_name (IN | OUT | INOUT | NOCOPY)* type_spec? default_value_part?
;
default_value_part
: (ASSIGN_OP | DEFAULT) expression
;
// Elements Declarations
seq_of_declare_specs
: declare_spec+
;
declare_spec
: pragma_declaration
| variable_declaration
| subtype_declaration
| cursor_declaration
| exception_declaration
| type_declaration
| procedure_spec
| function_spec
| procedure_body
| function_body
;
// incorporates constant_declaration
variable_declaration
: identifier CONSTANT? type_spec (NOT NULL)? default_value_part? ';'
;
subtype_declaration
: SUBTYPE identifier IS type_spec (RANGE expression '..' expression)? (NOT NULL)? ';'
;
// cursor_declaration incorportates curscursor_body and cursor_spec
cursor_declaration
: CURSOR identifier ('(' (','? parameter_spec)+ ')' )? (RETURN type_spec)? (IS select_statement)? ';'
;
parameter_spec
: parameter_name (IN? type_spec)? default_value_part?
;
exception_declaration
: identifier EXCEPTION ';'
;
pragma_declaration
: PRAGMA (SERIALLY_REUSABLE
| AUTONOMOUS_TRANSACTION
| EXCEPTION_INIT '(' exception_name ',' numeric_negative ')'
| INLINE '(' id1=identifier ',' expression ')'
| RESTRICT_REFERENCES '(' (identifier | DEFAULT) (',' identifier)+ ')') ';'
;
// Record Declaration Specific Clauses
// incorporates ref_cursor_type_definition
record_type_def
: RECORD '(' (','? field_spec)+ ')'
;
field_spec
: column_name type_spec? (NOT NULL)? default_value_part?
;
ref_cursor_type_def
: REF CURSOR (RETURN type_spec)?
;
type_declaration
: TYPE identifier IS (table_type_def | varray_type_def | record_type_def | ref_cursor_type_def) ';'
;
table_type_def
: TABLE OF type_spec table_indexed_by_part? (NOT NULL)?
;
table_indexed_by_part
: (idx1=INDEXED | idx2=INDEX) BY type_spec
;
varray_type_def
: (VARRAY | VARYING ARRAY) '(' expression ')' OF type_spec (NOT NULL)?
;
// Statements
seq_of_statements
: (statement (';' | EOF) | label_declaration)+
;
label_declaration
: ltp1= '<' '<' label_name '>' '>'
;
statement
: CREATE swallow_to_semi
| TRUNCATE swallow_to_semi
| body
| block
| assignment_statement
| continue_statement
| exit_statement
| goto_statement
| if_statement
| loop_statement
| forall_statement
| null_statement
| raise_statement
| return_statement
| case_statement/*[true]*/
| sql_statement
| function_call
| pipe_row_statement
;
swallow_to_semi
: ~';'+
;
assignment_statement
: (general_element | bind_variable) ASSIGN_OP expression
;
continue_statement
: CONTINUE label_name? (WHEN condition)?
;
exit_statement
: EXIT label_name? (WHEN condition)?
;
goto_statement
: GOTO label_name
;
if_statement
: IF condition THEN seq_of_statements elsif_part* else_part? END IF
;
elsif_part
: ELSIF condition THEN seq_of_statements
;
else_part
: ELSE seq_of_statements
;
loop_statement
: label_declaration? (WHILE condition | FOR cursor_loop_param)? LOOP seq_of_statements END LOOP label_name?
;
// Loop Specific Clause
cursor_loop_param
: index_name IN REVERSE? lower_bound range_separator='..' upper_bound
| record_name IN (cursor_name ('(' expressions? ')')? | '(' select_statement ')')
;
forall_statement
: FORALL index_name IN bounds_clause sql_statement (SAVE EXCEPTIONS)?
;
bounds_clause
: lower_bound '..' upper_bound
| INDICES OF collection_name between_bound?
| VALUES OF index_name
;
between_bound
: BETWEEN lower_bound AND upper_bound
;
lower_bound
: concatenation
;
upper_bound
: concatenation
;
null_statement
: NULL
;
raise_statement
: RAISE exception_name?
;
return_statement
: RETURN expression?
;
function_call
: CALL? routine_name function_argument?
;
pipe_row_statement
: PIPE ROW '(' expression ')';
body
: BEGIN seq_of_statements (EXCEPTION exception_handler+)? END label_name?
;
// Body Specific Clause
exception_handler
: WHEN exception_name (OR exception_name)* THEN seq_of_statements
;
trigger_block
: (DECLARE? declare_spec+)? body
;
block
: DECLARE? declare_spec+ body
;
// SQL Statements
sql_statement
: execute_immediate
| data_manipulation_language_statements
| cursor_manipulation_statements
| transaction_control_statements
;
execute_immediate
: EXECUTE IMMEDIATE expression (into_clause using_clause? | using_clause dynamic_returning_clause? | dynamic_returning_clause)?
;
// Execute Immediate Specific Clause
dynamic_returning_clause
: (RETURNING | RETURN) into_clause
;
// DML Statements
data_manipulation_language_statements
: merge_statement
| lock_table_statement
| select_statement
| update_statement
| delete_statement
| insert_statement
| explain_statement
;
// Cursor Manipulation Statements
cursor_manipulation_statements
: close_statement
| open_statement
| fetch_statement
| open_for_statement
;
close_statement
: CLOSE cursor_name
;
open_statement
: OPEN cursor_name ('(' expressions? ')')?
;
fetch_statement
: FETCH cursor_name (it1=INTO (','? variable_name)+ | BULK COLLECT INTO (','? variable_name)+)
;
open_for_statement
: OPEN variable_name FOR (select_statement | expression) using_clause?
;
// Transaction Control SQL Statements
transaction_control_statements
: set_transaction_command
| set_constraint_command
| commit_statement
| rollback_statement
| savepoint_statement
;
set_transaction_command
: SET TRANSACTION
(READ (ONLY | WRITE) | ISOLATION LEVEL (SERIALIZABLE | READ COMMITTED) | USE ROLLBACK SEGMENT rollback_segment_name)?
(NAME quoted_string)?
;
set_constraint_command
: SET (CONSTRAINT | CONSTRAINTS) (ALL | (','? constraint_name)+) (IMMEDIATE | DEFERRED)
;
commit_statement
: COMMIT WORK?
(COMMENT expression | FORCE (CORRUPT_XID expression | CORRUPT_XID_ALL | expression (',' expression)?))?
write_clause?
;
write_clause
: WRITE (WAIT | NOWAIT)? (IMMEDIATE | BATCH)?
;
rollback_statement
: ROLLBACK WORK? (TO SAVEPOINT? savepoint_name | FORCE quoted_string)?
;
savepoint_statement
: SAVEPOINT savepoint_name
;
// Dml
/* TODO
//SHOULD BE OVERRIDEN!
compilation_unit
: seq_of_statements* EOF
;
//SHOULD BE OVERRIDEN!
seq_of_statements
: select_statement
| update_statement
| delete_statement
| insert_statement
| lock_table_statement
| merge_statement
| explain_statement
// | case_statement[true]
;
*/
explain_statement
: EXPLAIN PLAN (SET STATEMENT_ID '=' quoted_string)? (INTO tableview_name)?
FOR (select_statement | update_statement | delete_statement | insert_statement | merge_statement)
;
select_statement
: subquery_factoring_clause? subquery (for_update_clause | order_by_clause | offset_clause | fetch_clause)*
;
// Select Specific Clauses
subquery_factoring_clause
: WITH (','? factoring_element)+
;
factoring_element
: query_name paren_column_list? AS '(' subquery order_by_clause? ')'
search_clause? cycle_clause?
;
search_clause
: SEARCH (DEPTH | BREADTH) FIRST BY column_name ASC? DESC? (NULLS FIRST)? (NULLS LAST)?
(',' column_name ASC? DESC? (NULLS FIRST)? (NULLS LAST)?)* SET column_name
;
cycle_clause
: CYCLE column_list SET column_name TO expression DEFAULT expression
;
subquery
: subquery_basic_elements subquery_operation_part*
;
subquery_basic_elements
: query_block
| '(' subquery ')'
;
subquery_operation_part
: (UNION ALL? | INTERSECT | MINUS) subquery_basic_elements
;
query_block
: SELECT (DISTINCT | UNIQUE | ALL)? ('*' | (','? selected_element)+)
into_clause? from_clause where_clause? hierarchical_query_clause? group_by_clause? model_clause?
;
selected_element
: select_list_elements column_alias?
;
from_clause
: FROM table_ref_list
;
select_list_elements
: tableview_name '.' '*'
| (regular_id '.')? expressions
;
table_ref_list
: (','? table_ref)+
;
// NOTE to PIVOT clause
// according the SQL reference this should not be possible
// according to he reality it is. Here we probably apply pivot/unpivot onto whole join clause
// eventhough it is not enclosed in parenthesis. See pivot examples 09,10,11
table_ref
: table_ref_aux join_clause* (pivot_clause | unpivot_clause)?
;
table_ref_aux
: table_ref_aux_internal flashback_query_clause* (/*{isTableAlias()}?*/ table_alias)?
;
table_ref_aux_internal
: dml_table_expression_clause (pivot_clause | unpivot_clause)? # table_ref_aux_internal_one
| '(' table_ref subquery_operation_part* ')' (pivot_clause | unpivot_clause)? # table_ref_aux_internal_two
| ONLY '(' dml_table_expression_clause ')' # table_ref_aux_internal_three
;
join_clause
: query_partition_clause? (CROSS | NATURAL)? (INNER | outer_join_type)?
JOIN table_ref_aux query_partition_clause? (join_on_part | join_using_part)*
;
join_on_part
: ON condition
;
join_using_part
: USING paren_column_list
;
outer_join_type
: (FULL | LEFT | RIGHT) OUTER?
;
query_partition_clause
: PARTITION BY (('(' (subquery | expressions)? ')') | expressions)
;
flashback_query_clause
: VERSIONS BETWEEN (SCN | TIMESTAMP) expression
| AS OF (SCN | TIMESTAMP | SNAPSHOT) expression
;
pivot_clause
: PIVOT XML? '(' (','? pivot_element)+ pivot_for_clause pivot_in_clause ')'
;
pivot_element
: aggregate_function_name '(' expression ')' column_alias?
;
pivot_for_clause
: FOR (column_name | paren_column_list)
;
pivot_in_clause
: IN '(' (subquery | (','? ANY)+ | (','? pivot_in_clause_element)+) ')'
;
pivot_in_clause_element
: pivot_in_clause_elements column_alias?
;
pivot_in_clause_elements
: expression
| '(' expressions? ')'
;
unpivot_clause
: UNPIVOT ((INCLUDE | EXCLUDE) NULLS)?
'(' (column_name | paren_column_list) pivot_for_clause unpivot_in_clause ')'
;
unpivot_in_clause
: IN '(' (','? unpivot_in_elements)+ ')'
;
unpivot_in_elements
: (column_name | paren_column_list)
(AS (constant | '(' (','? constant)+ ')'))?
;
hierarchical_query_clause
: CONNECT BY NOCYCLE? condition start_part?
| start_part CONNECT BY NOCYCLE? condition
;
start_part
: START WITH condition
;
group_by_clause
: GROUP BY (','? group_by_elements)+ having_clause?
| having_clause (GROUP BY (','? group_by_elements)+)?
;
group_by_elements
: grouping_sets_clause
| rollup_cube_clause
| expression
;
rollup_cube_clause
: (ROLLUP | CUBE) '(' (','? grouping_sets_elements)+ ')'
;
grouping_sets_clause
: GROUPING SETS '(' (','? grouping_sets_elements)+ ')'
;
grouping_sets_elements
: rollup_cube_clause
| '(' expressions? ')'
| expression
;
having_clause
: HAVING condition
;
model_clause
: MODEL cell_reference_options* return_rows_clause? reference_model* main_model
;
cell_reference_options
: (IGNORE | KEEP) NAV
| UNIQUE (DIMENSION | SINGLE REFERENCE)
;
return_rows_clause
: RETURN (UPDATED | ALL) ROWS
;
reference_model
: REFERENCE reference_model_name ON '(' subquery ')' model_column_clauses cell_reference_options*
;
main_model
: (MAIN main_model_name)? model_column_clauses cell_reference_options* model_rules_clause
;
model_column_clauses
: model_column_partition_part? DIMENSION BY model_column_list MEASURES model_column_list
;
model_column_partition_part
: PARTITION BY model_column_list
;
model_column_list
: '(' (','? model_column)+ ')'
;
model_column
: (expression | query_block) column_alias?
;
model_rules_clause
: model_rules_part? '(' (','? model_rules_element)* ')'
;
model_rules_part
: RULES (UPDATE | UPSERT ALL?)? ((AUTOMATIC | SEQUENTIAL) ORDER)? model_iterate_clause?
;
model_rules_element
: (UPDATE | UPSERT ALL?)? cell_assignment order_by_clause? '=' expression
;
cell_assignment
: model_expression
;
model_iterate_clause
: ITERATE '(' expression ')' until_part?
;
until_part
: UNTIL '(' condition ')'
;
order_by_clause
: ORDER SIBLINGS? BY (','? order_by_elements)+
;
order_by_elements
: expression (ASC | DESC)? (NULLS (FIRST | LAST))?
;
offset_clause
: OFFSET expression (ROW | ROWS)
;
fetch_clause
: FETCH (FIRST | NEXT) (expression PERCENT_KEYWORD?)? (ROW | ROWS) (ONLY | WITH TIES)
;
for_update_clause
: FOR UPDATE for_update_of_part? for_update_options?
;
for_update_of_part
: OF column_list
;
for_update_options
: SKIP_ LOCKED
| NOWAIT
| WAIT expression
;
update_statement
: UPDATE general_table_ref update_set_clause where_clause? static_returning_clause? error_logging_clause?
;
// Update Specific Clauses
update_set_clause
: SET
((','? column_based_update_set_clause)+ | VALUE '(' identifier ')' '=' expression)
;
column_based_update_set_clause
: column_name '=' expression
| paren_column_list '=' subquery
;
delete_statement
: DELETE FROM? general_table_ref where_clause? static_returning_clause? error_logging_clause?
;
insert_statement
: INSERT (single_table_insert | multi_table_insert)
;
// Insert Specific Clauses
single_table_insert
: insert_into_clause (values_clause static_returning_clause? | select_statement) error_logging_clause?
;
multi_table_insert
: (ALL multi_table_element+ | conditional_insert_clause) select_statement
;
multi_table_element
: insert_into_clause values_clause? error_logging_clause?
;
conditional_insert_clause
: (ALL | FIRST)? conditional_insert_when_part+ conditional_insert_else_part?
;
conditional_insert_when_part
: WHEN condition THEN multi_table_element+
;
conditional_insert_else_part
: ELSE multi_table_element+
;
insert_into_clause
: INTO general_table_ref paren_column_list?
;
values_clause
: VALUES '(' expressions? ')'
;
merge_statement
: MERGE INTO tableview_name table_alias? USING selected_tableview ON '(' condition ')'
(merge_update_clause merge_insert_clause? | merge_insert_clause merge_update_clause?)?
error_logging_clause?
;
// Merge Specific Clauses
merge_update_clause
: WHEN MATCHED THEN UPDATE SET merge_element (',' merge_element)* where_clause? merge_update_delete_part?
;
merge_element
: column_name '=' expression
;
merge_update_delete_part
: DELETE where_clause
;
merge_insert_clause
: WHEN NOT MATCHED THEN INSERT paren_column_list?
VALUES '(' expressions? ')' where_clause?
;
selected_tableview
: (tableview_name | '(' select_statement ')') table_alias?
;
lock_table_statement
: LOCK TABLE lock_table_element (',' lock_table_element)* IN lock_mode MODE wait_nowait_part?
;
wait_nowait_part
: WAIT expression
| NOWAIT
;
// Lock Specific Clauses
lock_table_element
: tableview_name partition_extension_clause?
;
lock_mode
: ROW SHARE
| ROW EXCLUSIVE
| SHARE UPDATE?
| SHARE ROW EXCLUSIVE
| EXCLUSIVE
;
// Common DDL Clauses
general_table_ref
: (dml_table_expression_clause | ONLY '(' dml_table_expression_clause ')') table_alias?
;
static_returning_clause
: (RETURNING | RETURN) expressions into_clause
;
error_logging_clause
: LOG ERRORS error_logging_into_part? expression? error_logging_reject_part?
;
error_logging_into_part
: INTO tableview_name
;
error_logging_reject_part
: REJECT LIMIT (UNLIMITED | expression)
;
dml_table_expression_clause
: table_collection_expression
| '(' select_statement subquery_restriction_clause? ')'
| tableview_name sample_clause?
;
table_collection_expression
: (TABLE | THE) ('(' subquery ')' | '(' expression ')' ('(' '+' ')')?)
;
subquery_restriction_clause
: WITH (READ ONLY | CHECK OPTION (CONSTRAINT constraint_name)?)
;
sample_clause
: SAMPLE BLOCK? '(' expression (',' expression)? ')' seed_part?
;
seed_part
: SEED '(' expression ')'
;
// Expression & Condition
condition
: expression
;
expressions
: expression (',' expression)*
;
expression
: cursor_expression
| logical_expression
;
cursor_expression
: CURSOR '(' subquery ')'
;
logical_expression
: multiset_expression (IS NOT?
(NULL | NAN | PRESENT | INFINITE | A_LETTER SET | EMPTY | OF TYPE?
'(' ONLY? type_spec (',' type_spec)* ')'))*
| NOT logical_expression
| logical_expression AND logical_expression
| logical_expression OR logical_expression
;
multiset_expression
: relational_expression (multiset_type=(MEMBER | SUBMULTISET) OF? concatenation)?
;
relational_expression
: relational_expression relational_operator relational_expression
| compound_expression
;
compound_expression
: concatenation
(NOT? ( IN in_elements
| BETWEEN between_elements
| like_type=(LIKE | LIKEC | LIKE2 | LIKE4) concatenation (ESCAPE concatenation)?))?
;
relational_operator
: '='
| (NOT_EQUAL_OP | '<' '>' | '!' '=' | '^' '=')
| ('<' | '>') '='?
;
in_elements
: '(' subquery ')'
| '(' concatenation (',' concatenation)* ')'
| constant
| bind_variable
| general_element
;
between_elements
: concatenation AND concatenation
;
concatenation
: model_expression
(AT (LOCAL | TIME ZONE concatenation) | interval_expression)?
| concatenation op=(ASTERISK | SOLIDUS) concatenation
| concatenation op=(PLUS_SIGN | MINUS_SIGN) concatenation
| concatenation BAR BAR concatenation
;
interval_expression
: DAY ('(' concatenation ')')? TO SECOND ('(' concatenation ')')?
| YEAR ('(' concatenation ')')? TO MONTH
;
model_expression
: unary_expression ('[' model_expression_element ']')?
;
model_expression_element
: (ANY | expression) (',' (ANY | expression))*
| single_column_for_loop (',' single_column_for_loop)*
| multi_column_for_loop
;
single_column_for_loop
: FOR column_name
( IN '(' expressions? ')'
| (LIKE expression)? FROM fromExpr=expression TO toExpr=expression
action_type=(INCREMENT | DECREMENT) action_expr=expression)
;
multi_column_for_loop
: FOR paren_column_list
IN '(' (subquery | '(' expressions? ')') ')'
;
unary_expression
: ('-' | '+') unary_expression
| PRIOR unary_expression
| CONNECT_BY_ROOT unary_expression
| /*TODO {input.LT(1).getText().equalsIgnoreCase("new") && !input.LT(2).getText().equals(".")}?*/ NEW unary_expression
| DISTINCT unary_expression
| ALL unary_expression
| /*TODO{(input.LA(1) == CASE || input.LA(2) == CASE)}?*/ case_statement/*[false]*/
| quantified_expression
| standard_function
| atom
;
case_statement /*TODO [boolean isStatementParameter]
TODO scope {
boolean isStatement;
}
@init {$case_statement::isStatement = $isStatementParameter;}*/
: searched_case_statement
| simple_case_statement
;
// CASE
simple_case_statement
: label_name? ck1=CASE expression simple_case_when_part+ case_else_part? END CASE? label_name?
;
simple_case_when_part
: WHEN expression THEN (/*TODO{$case_statement::isStatement}?*/ seq_of_statements | expression)
;
searched_case_statement
: label_name? ck1=CASE searched_case_when_part+ case_else_part? END CASE? label_name?
;
searched_case_when_part
: WHEN expression THEN (/*TODO{$case_statement::isStatement}?*/ seq_of_statements | expression)
;
case_else_part
: ELSE (/*{$case_statement::isStatement}?*/ seq_of_statements | expression)
;
atom
: table_element outer_join_sign
| bind_variable
| constant
| general_element
| '(' subquery ')' subquery_operation_part*
| '(' expressions ')'
;
quantified_expression
: (SOME | EXISTS | ALL | ANY) ('(' subquery ')' | '(' expression ')')
;
string_function
: SUBSTR '(' expression ',' expression (',' expression)? ')'
| TO_CHAR '(' (table_element | standard_function | expression)
(',' quoted_string)? (',' quoted_string)? ')'
| DECODE '(' expressions ')'
| CHR '(' concatenation USING NCHAR_CS ')'
| NVL '(' expression ',' expression ')'
| TRIM '(' ((LEADING | TRAILING | BOTH)? quoted_string? FROM)? concatenation ')'
| TO_DATE '(' expression (',' quoted_string)? ')'
;
standard_function
: string_function
| numeric_function_wrapper
| other_function
;
literal
: CHAR_STRING
| string_function
| numeric
| MAXVALUE
;
numeric_function_wrapper
: numeric_function (single_column_for_loop | multi_column_for_loop)?
;
numeric_function
: SUM '(' (DISTINCT | ALL)? expression ')'
| COUNT '(' ( '*' | ((DISTINCT | UNIQUE | ALL)? concatenation)? ) ')' over_clause?
| ROUND '(' expression (',' UNSIGNED_INTEGER)? ')'
| AVG '(' (DISTINCT | ALL)? expression ')'
| MAX '(' (DISTINCT | ALL)? expression ')'
| LEAST '(' expressions ')'
| GREATEST '(' expressions ')'
;
other_function
: over_clause_keyword function_argument_analytic over_clause?
| /*TODO stantard_function_enabling_using*/ regular_id function_argument_modeling using_clause?
| COUNT '(' ( '*' | (DISTINCT | UNIQUE | ALL)? concatenation) ')' over_clause?
| (CAST | XMLCAST) '(' (MULTISET '(' subquery ')' | concatenation) AS type_spec ')'
| COALESCE '(' table_element (',' (numeric | quoted_string))? ')'
| COLLECT '(' (DISTINCT | UNIQUE)? concatenation collect_order_by_part? ')'
| within_or_over_clause_keyword function_argument within_or_over_part+
| cursor_name ( PERCENT_ISOPEN | PERCENT_FOUND | PERCENT_NOTFOUND | PERCENT_ROWCOUNT )
| DECOMPOSE '(' concatenation (CANONICAL | COMPATIBILITY)? ')'
| EXTRACT '(' regular_id FROM concatenation ')'
| (FIRST_VALUE | LAST_VALUE) function_argument_analytic respect_or_ignore_nulls? over_clause
| standard_prediction_function_keyword
'(' expressions cost_matrix_clause? using_clause? ')'
| TRANSLATE '(' expression (USING (CHAR_CS | NCHAR_CS))? (',' expression)* ')'
| TREAT '(' expression AS REF? type_spec ')'
| TRIM '(' ((LEADING | TRAILING | BOTH)? quoted_string? FROM)? concatenation ')'
| XMLAGG '(' expression order_by_clause? ')' ('.' general_element_part)?
| (XMLCOLATTVAL | XMLFOREST)
'(' (','? xml_multiuse_expression_element)+ ')' ('.' general_element_part)?
| XMLELEMENT
'(' (ENTITYESCAPING | NOENTITYESCAPING)? (NAME | EVALNAME)? expression
(/*TODO{input.LT(2).getText().equalsIgnoreCase("xmlattributes")}?*/ ',' xml_attributes_clause)?
(',' expression column_alias?)* ')' ('.' general_element_part)?
| XMLEXISTS '(' expression xml_passing_clause? ')'
| XMLPARSE '(' (DOCUMENT | CONTENT) concatenation WELLFORMED? ')' ('.' general_element_part)?
| XMLPI
'(' (NAME identifier | EVALNAME concatenation) (',' concatenation)? ')' ('.' general_element_part)?
| XMLQUERY
'(' concatenation xml_passing_clause? RETURNING CONTENT (NULL ON EMPTY)? ')' ('.' general_element_part)?
| XMLROOT
'(' concatenation (',' xmlroot_param_version_part)? (',' xmlroot_param_standalone_part)? ')' ('.' general_element_part)?
| XMLSERIALIZE
'(' (DOCUMENT | CONTENT) concatenation (AS type_spec)?
xmlserialize_param_enconding_part? xmlserialize_param_version_part? xmlserialize_param_ident_part? ((HIDE | SHOW) DEFAULTS)? ')'
('.' general_element_part)?
| XMLTABLE
'(' xml_namespaces_clause? concatenation xml_passing_clause? (COLUMNS xml_table_column (',' xml_table_column))? ')' ('.' general_element_part)?
;
over_clause_keyword
: AVG
| CORR
| LAG
| LEAD
| MAX
| MEDIAN
| MIN
| NTILE
| RATIO_TO_REPORT
| ROW_NUMBER
| SUM
| VARIANCE
| REGR_
| STDDEV
| VAR_
| COVAR_
;
within_or_over_clause_keyword
: CUME_DIST
| DENSE_RANK
| LISTAGG
| PERCENT_RANK
| PERCENTILE_CONT
| PERCENTILE_DISC
| RANK
;
standard_prediction_function_keyword
: PREDICTION
| PREDICTION_BOUNDS
| PREDICTION_COST
| PREDICTION_DETAILS
| PREDICTION_PROBABILITY
| PREDICTION_SET
;
over_clause
: OVER '(' query_partition_clause? (order_by_clause windowing_clause?)? ')'
;
windowing_clause
: windowing_type
(BETWEEN windowing_elements AND windowing_elements | windowing_elements)
;
windowing_type
: ROWS
| RANGE
;
windowing_elements
: UNBOUNDED PRECEDING
| CURRENT ROW
| concatenation (PRECEDING | FOLLOWING)
;
using_clause
: USING ('*' | (','? using_element)+)
;
using_element
: (IN OUT? | OUT)? select_list_elements column_alias?
;
collect_order_by_part
: ORDER BY concatenation
;
within_or_over_part
: WITHIN GROUP '(' order_by_clause ')'
| over_clause
;
cost_matrix_clause
: COST (MODEL AUTO? | '(' (','? cost_class_name)+ ')' VALUES '(' expressions? ')')
;
xml_passing_clause
: PASSING (BY VALUE)? expression column_alias? (',' expression column_alias?)
;
xml_attributes_clause
: XMLATTRIBUTES
'(' (ENTITYESCAPING | NOENTITYESCAPING)? (SCHEMACHECK | NOSCHEMACHECK)?
(','? xml_multiuse_expression_element)+ ')'
;
xml_namespaces_clause
: XMLNAMESPACES
'(' (concatenation column_alias)? (',' concatenation column_alias)*
xml_general_default_part? ')'
;
xml_table_column
: xml_column_name
(FOR ORDINALITY | type_spec (PATH concatenation)? xml_general_default_part?)
;
xml_general_default_part
: DEFAULT concatenation
;
xml_multiuse_expression_element
: expression (AS (id_expression | EVALNAME concatenation))?
;
xmlroot_param_version_part
: VERSION (NO VALUE | expression)
;
xmlroot_param_standalone_part
: STANDALONE (YES | NO VALUE?)
;
xmlserialize_param_enconding_part
: ENCODING concatenation
;
xmlserialize_param_version_part
: VERSION concatenation
;
xmlserialize_param_ident_part
: NO INDENT
| INDENT (SIZE '=' concatenation)?
;
// SqlPlus
sql_plus_command
: '/'
| EXIT
| PROMPT
| SHOW (ERR | ERRORS)
| START_CMD
| whenever_command
| set_command
;
whenever_command
: WHENEVER (SQLERROR | OSERROR)
( EXIT (SUCCESS | FAILURE | WARNING) (COMMIT | ROLLBACK)
| CONTINUE (COMMIT | ROLLBACK | NONE))
;
set_command
: SET regular_id (CHAR_STRING | ON | OFF | /*EXACT_NUM_LIT*/numeric | regular_id)
;
// Common
partition_extension_clause
: (SUBPARTITION | PARTITION) FOR? '(' expressions? ')'
;
column_alias
: AS? (identifier | quoted_string)
| AS
;
table_alias
: identifier
| quoted_string
;
where_clause
: WHERE (CURRENT OF cursor_name | expression)
;
into_clause
: (BULK COLLECT)? INTO (','? variable_name)+
;
// Common Named Elements
xml_column_name
: identifier
| quoted_string
;
cost_class_name
: identifier
;
attribute_name
: identifier
;
savepoint_name
: identifier
;
rollback_segment_name
: identifier
;
table_var_name
: identifier
;
schema_name
: identifier
;
routine_name
: identifier ('.' id_expression)* ('@' link_name)?
;
package_name
: identifier
;
implementation_type_name
: identifier ('.' id_expression)?
;
parameter_name
: identifier
;
reference_model_name
: identifier
;
main_model_name
: identifier
;
container_tableview_name
: identifier ('.' id_expression)?
;
aggregate_function_name
: identifier ('.' id_expression)*
;
query_name
: identifier
;
grantee_name
: id_expression identified_by?
;
role_name
: id_expression
| CONNECT
;
constraint_name
: identifier ('.' id_expression)* ('@' link_name)?
;
label_name
: id_expression
;
type_name
: id_expression ('.' id_expression)*
;
sequence_name
: id_expression ('.' id_expression)*
;
exception_name
: identifier ('.' id_expression)*
;
function_name
: identifier ('.' id_expression)?
;
procedure_name
: identifier ('.' id_expression)?
;
trigger_name
: identifier ('.' id_expression)?
;
variable_name
: (INTRODUCER char_set_name)? id_expression ('.' id_expression)?
| bind_variable
;
index_name
: identifier ('.' id_expression)?
;
cursor_name
: general_element
| bind_variable
;
record_name
: identifier
| bind_variable
;
collection_name
: identifier ('.' id_expression)?
;
link_name
: identifier
;
column_name
: identifier ('.' id_expression)*
;
tableview_name
: identifier ('.' id_expression)?
('@' link_name | /*TODO{!(input.LA(2) == BY)}?*/ partition_extension_clause)?
;
char_set_name
: id_expression ('.' id_expression)*
;
synonym_name
: identifier
;
// Represents a valid DB object name in DDL commands which are valid for several DB (or schema) objects.
// For instance, create synonym ... for <DB object name>, or rename <old DB object name> to <new DB object name>.
// Both are valid for sequences, tables, views, etc.
schema_object_name
: id_expression
;
dir_object_name
: id_expression
;
user_object_name
: id_expression
;
grant_object_name
: tableview_name
| USER (','? user_object_name)+
| DIRECTORY dir_object_name
| EDITION schema_object_name
| MINING MODEL schema_object_name
| JAVA (SOURCE | RESOURCE) schema_object_name
| SQL TRANSLATION PROFILE schema_object_name
;
column_list
: (','? column_name)+
;
paren_column_list
: LEFT_PAREN column_list RIGHT_PAREN
;
// PL/SQL Specs
// NOTE: In reality this applies to aggregate functions only
keep_clause
: KEEP '(' DENSE_RANK (FIRST | LAST) order_by_clause ')' over_clause?
;
function_argument
: '(' (','? argument)* ')' keep_clause?
;
function_argument_analytic
: '(' (','? argument respect_or_ignore_nulls?)* ')' keep_clause?
;
function_argument_modeling
: '(' column_name (',' (numeric | NULL) (',' (numeric | NULL))?)?
USING (tableview_name '.' '*' | '*' | (','? expression column_alias?)+)
')' keep_clause?
;
respect_or_ignore_nulls
: (RESPECT | IGNORE) NULLS
;
argument
: (identifier '=' '>')? expression
;
type_spec
: datatype
| REF? type_name (PERCENT_ROWTYPE | PERCENT_TYPE)?
;
datatype
: native_datatype_element precision_part? (WITH LOCAL? TIME ZONE | CHARACTER SET char_set_name)?
| INTERVAL (YEAR | DAY) ('(' expression ')')? TO (MONTH | SECOND) ('(' expression ')')?
;
precision_part
: '(' numeric (',' numeric)? (CHAR | BYTE)? ')'
;
native_datatype_element
: BINARY_INTEGER
| PLS_INTEGER
| NATURAL
| BINARY_FLOAT
| BINARY_DOUBLE
| NATURALN
| POSITIVE
| POSITIVEN
| SIGNTYPE
| SIMPLE_INTEGER
| NVARCHAR2
| DEC
| INTEGER
| INT
| NUMERIC
| SMALLINT
| NUMBER
| DECIMAL
| DOUBLE PRECISION?
| FLOAT
| REAL
| NCHAR
| LONG RAW?
| CHAR
| CHARACTER
| VARCHAR2
| VARCHAR
| STRING
| RAW
| BOOLEAN
| DATE
| ROWID
| UROWID
| YEAR
| MONTH
| DAY
| HOUR
| MINUTE
| SECOND
| TIMEZONE_HOUR
| TIMEZONE_MINUTE
| TIMEZONE_REGION
| TIMEZONE_ABBR
| TIMESTAMP
| TIMESTAMP_UNCONSTRAINED
| TIMESTAMP_TZ_UNCONSTRAINED
| TIMESTAMP_LTZ_UNCONSTRAINED
| YMINTERVAL_UNCONSTRAINED
| DSINTERVAL_UNCONSTRAINED
| BFILE
| BLOB
| CLOB
| NCLOB
| MLSLABEL
;
bind_variable
: (BINDVAR | ':' UNSIGNED_INTEGER)
// Pro*C/C++ indicator variables
(INDICATOR? (BINDVAR | ':' UNSIGNED_INTEGER))?
('.' general_element_part)*
;
general_element
: general_element_part ('.' general_element_part)*
;
general_element_part
: (INTRODUCER char_set_name)? id_expression ('.' id_expression)* ('@' link_name)? function_argument?
;
table_element
: (INTRODUCER char_set_name)? id_expression ('.' id_expression)*
;
object_privilege
: ALL PRIVILEGES?
| ALTER
| DEBUG
| DELETE
| EXECUTE
| FLASHBACK ARCHIVE
| INDEX
| INHERIT PRIVILEGES
| INSERT
| KEEP SEQUENCE
| MERGE VIEW
| ON COMMIT REFRESH
| QUERY REWRITE
| READ
| REFERENCES
| SELECT
| TRANSLATE SQL
| UNDER
| UPDATE
| USE
| WRITE
;
//Ordered by type rather than alphabetically
system_privilege
: ALL PRIVILEGES
| ADVISOR
| ADMINISTER ANY? SQL TUNING SET
| (ALTER | CREATE | DROP) ANY SQL PROFILE
| ADMINISTER SQL MANAGEMENT OBJECT
| CREATE ANY? CLUSTER
| (ALTER | DROP) ANY CLUSTER
| (CREATE | DROP) ANY CONTEXT
| EXEMPT REDACTION POLICY
| ALTER DATABASE
| (ALTER | CREATE) PUBLIC? DATABASE LINK
| DROP PUBLIC DATABASE LINK
| DEBUG CONNECT SESSION
| DEBUG ANY PROCEDURE
| ANALYZE ANY DICTIONARY
| CREATE ANY? DIMENSION
| (ALTER | DROP) ANY DIMENSION
| (CREATE | DROP) ANY DIRECTORY
| (CREATE | DROP) ANY EDITION
| FLASHBACK (ARCHIVE ADMINISTER | ANY TABLE)
| (ALTER | CREATE | DROP) ANY INDEX
| CREATE ANY? INDEXTYPE
| (ALTER | DROP | EXECUTE) ANY INDEXTYPE
| CREATE (ANY | EXTERNAL)? JOB
| EXECUTE ANY (CLASS | PROGRAM)
| MANAGE SCHEDULER
| ADMINISTER KEY MANAGEMENT
| CREATE ANY? LIBRARY
| (ALTER | DROP | EXECUTE) ANY LIBRARY
| LOGMINING
| CREATE ANY? MATERIALIZED VIEW
| (ALTER | DROP) ANY MATERIALIZED VIEW
| GLOBAL? QUERY REWRITE
| ON COMMIT REFRESH
| CREATE ANY? MINING MODEL
| (ALTER | DROP | SELECT | COMMENT) ANY MINING MODEL
| CREATE ANY? CUBE
| (ALTER | DROP | SELECT | UPDATE) ANY CUBE
| CREATE ANY? MEASURE FOLDER
| (DELETE | DROP | INSERT) ANY MEASURE FOLDER
| CREATE ANY? CUBE DIMENSION
| (ALTER | DELETE | DROP | INSERT | SELECT | UPDATE) ANY CUBE DIMENSION
| CREATE ANY? CUBE BUILD PROCESS
| (DROP | UPDATE) ANY CUBE BUILD PROCESS
| CREATE ANY? OPERATOR
| (ALTER | DROP | EXECUTE) ANY OPERATOR
| (CREATE | ALTER | DROP) ANY OUTLINE
| CREATE PLUGGABLE DATABASE
| SET CONTAINER
| CREATE ANY? PROCEDURE
| (ALTER | DROP | EXECUTE) ANY PROCEDURE
| (CREATE | ALTER | DROP ) PROFILE
| CREATE ROLE
| (ALTER | DROP | GRANT) ANY ROLE
| (CREATE | ALTER | DROP) ROLLBACK SEGMENT
| CREATE ANY? SEQUENCE
| (ALTER | DROP | SELECT) ANY SEQUENCE
| (ALTER | CREATE | RESTRICTED) SESSION
| ALTER RESOURCE COST
| CREATE ANY? SQL TRANSLATION PROFILE
| (ALTER | DROP | USE) ANY SQL TRANSLATION PROFILE
| TRANSLATE ANY SQL
| CREATE ANY? SYNONYM
| DROP ANY SYNONYM
| (CREATE | DROP) PUBLIC SYNONYM
| CREATE ANY? TABLE
| (ALTER | BACKUP | COMMENT | DELETE | DROP | INSERT | LOCK | READ | SELECT | UPDATE) ANY TABLE
| (CREATE | ALTER | DROP | MANAGE | UNLIMITED) TABLESPACE
| CREATE ANY? TRIGGER
| (ALTER | DROP) ANY TRIGGER
| ADMINISTER DATABASE TRIGGER
| CREATE ANY? TYPE
| (ALTER | DROP | EXECUTE | UNDER) ANY TYPE
| (CREATE | ALTER | DROP) USER
| CREATE ANY? VIEW
| (DROP | UNDER | MERGE) ANY VIEW
| (ANALYZE | AUDIT) ANY
| BECOME USER
| CHANGE NOTIFICATION
| EXEMPT ACCESS POLICY
| FORCE ANY? TRANSACTION
| GRANT ANY OBJECT? PRIVILEGE
| INHERIT ANY PRIVILEGES
| KEEP DATE TIME
| KEEP SYSGUID
| PURGE DBA_RECYCLEBIN
| RESUMABLE
| SELECT ANY (DICTIONARY | TRANSACTION)
| SYSBACKUP
| SYSDBA
| SYSDG
| SYSKM
| SYSOPER
;
// $>
// $<Lexer Mappings
constant
: TIMESTAMP (quoted_string | bind_variable) (AT TIME ZONE quoted_string)?
| INTERVAL (quoted_string | bind_variable | general_element_part)
(YEAR | MONTH | DAY | HOUR | MINUTE | SECOND)
('(' (UNSIGNED_INTEGER | bind_variable) (',' (UNSIGNED_INTEGER | bind_variable) )? ')')?
(TO ( DAY | HOUR | MINUTE | SECOND ('(' (UNSIGNED_INTEGER | bind_variable) ')')?))?
| numeric
| DATE quoted_string
| quoted_string
| NULL
| TRUE
| FALSE
| DBTIMEZONE
| SESSIONTIMEZONE
| MINVALUE
| MAXVALUE
| DEFAULT
;
numeric
: UNSIGNED_INTEGER
| APPROXIMATE_NUM_LIT
;
numeric_negative
: MINUS_SIGN numeric
;
quoted_string
: CHAR_STRING
//| CHAR_STRING_PERL
| NATIONAL_CHAR_STRING_LIT
;
identifier
: (INTRODUCER char_set_name)? id_expression
;
id_expression
: regular_id
| DELIMITED_ID
;
outer_join_sign
: '(' '+' ')'
;
regular_id
: non_reserved_keywords_pre12c
| non_reserved_keywords_in_12c
| REGULAR_ID
| A_LETTER
| AGENT
| AGGREGATE
| ANALYZE
| AUTONOMOUS_TRANSACTION
| BATCH
| BINARY_INTEGER
| BOOLEAN
| C_LETTER
| CHAR
| CLUSTER
| CONSTRUCTOR
| CUSTOMDATUM
| DECIMAL
| DELETE
| DETERMINISTIC
| DSINTERVAL_UNCONSTRAINED
| ERR
| EXCEPTION
| EXCEPTION_INIT
| EXCEPTIONS
| EXIT
| FLOAT
| FORALL
| INDICES
| INOUT
| INTEGER
| LANGUAGE
| LONG
| LOOP
| NUMBER
| ORADATA
| OSERROR
| OUT
| OVERRIDING
| PARALLEL_ENABLE
| PIPELINED
| PLS_INTEGER
| POSITIVE
| POSITIVEN
| PRAGMA
| RAISE
| RAW
| RECORD
| REF
| RENAME
| RESTRICT_REFERENCES
| RESULT
| SELF
| SERIALLY_REUSABLE
| SET
| SIGNTYPE
| SIMPLE_INTEGER
| SMALLINT
| SQLDATA
| SQLERROR
| SUBTYPE
| TIMESTAMP_LTZ_UNCONSTRAINED
| TIMESTAMP_TZ_UNCONSTRAINED
| TIMESTAMP_UNCONSTRAINED
| TRIGGER
| VARCHAR
| VARCHAR2
| VARIABLE
| WARNING
| WHILE
| XMLAGG
| YMINTERVAL_UNCONSTRAINED
| REGR_
| VAR_
| COVAR_
;
non_reserved_keywords_in_12c
: ACL
| ACTION
| ACTIONS
| ACTIVE
| ACTIVE_DATA
| ACTIVITY
| ADAPTIVE_PLAN
| ADVANCED
| AFD_DISKSTRING
| ANOMALY
| ANSI_REARCH
| APPLICATION
| APPROX_COUNT_DISTINCT
| ARCHIVAL
| ARCHIVED
| ASIS
| ASSIGN
| AUTO_LOGIN
| AUTO_REOPTIMIZE
| AVRO
| BACKGROUND
| BATCHSIZE
| BATCH_TABLE_ACCESS_BY_ROWID
| BEGINNING
| BEQUEATH
| BITMAP_AND
| BSON
| CACHING
| CALCULATED
| CALLBACK
| CAPACITY
| CDBDEFAULT
| CLASSIFIER
| CLEANUP
| CLIENT
| CLUSTER_DETAILS
| CLUSTER_DISTANCE
| CLUSTERING
| COMMON_DATA
| COMPONENT
| COMPONENTS
| CON_DBID_TO_ID
| CONDITION
| CONDITIONAL
| CON_GUID_TO_ID
| CON_ID
| CON_NAME_TO_ID
| CONTAINER_DATA
| CONTAINERS
| CON_UID_TO_ID
| COOKIE
| COPY
| CREATE_FILE_DEST
| CREDENTIAL
| CRITICAL
| CUBE_AJ
| CUBE_SJ
| DATAMOVEMENT
| DATAOBJ_TO_MAT_PARTITION
| DATAPUMP
| DATA_SECURITY_REWRITE_LIMIT
| DAYS
| DB_UNIQUE_NAME
| DECORRELATE
| DEFINE
| DELEGATE
| DELETE_ALL
| DESTROY
| DIMENSIONS
| DISABLE_ALL
| DISABLE_PARALLEL_DML
| DISCARD
| DISTRIBUTE
| DUPLICATE
| DV
| EDITIONABLE
| ELIM_GROUPBY
| EM
| ENABLE_ALL
| ENABLE_PARALLEL_DML
| EQUIPART
| EVAL
| EVALUATE
| EXISTING
| EXPRESS
| EXTRACTCLOBXML
| FACTOR
| FAILOVER
| FAILURE
| FAMILY
| FAR
| FASTSTART
| FEATURE_DETAILS
| FETCH
| FILE_NAME_CONVERT
| FIXED_VIEW_DATA
| FORMAT
| GATHER_OPTIMIZER_STATISTICS
| GET
| ILM
| INACTIVE
| INDEXING
| INHERIT
| INMEMORY
| INMEMORY_PRUNING
| INPLACE
| INTERLEAVED
| JSON
| JSON_ARRAY
| JSON_ARRAYAGG
| JSON_EQUAL
| JSON_EXISTS
| JSON_EXISTS2
| JSONGET
| JSON_OBJECT
| JSON_OBJECTAGG
| JSONPARSE
| JSON_QUERY
| JSON_SERIALIZE
| JSON_TABLE
| JSON_TEXTCONTAINS
| JSON_TEXTCONTAINS2
| JSON_VALUE
| KEYSTORE
| LABEL
| LAX
| LIFECYCLE
| LINEAR
| LOCKING
| LOGMINING
| MAP
| MATCH
| MATCHES
| MATCH_NUMBER
| MATCH_RECOGNIZE
| MAX_SHARED_TEMP_SIZE
| MEMCOMPRESS
| METADATA
| MODEL_NB
| MODEL_SV
| MODIFICATION
| MODULE
| MONTHS
| MULTIDIMENSIONAL
| NEG
| NO_ADAPTIVE_PLAN
| NO_ANSI_REARCH
| NO_AUTO_REOPTIMIZE
| NO_BATCH_TABLE_ACCESS_BY_ROWID
| NO_CLUSTERING
| NO_COMMON_DATA
| NOCOPY
| NO_DATA_SECURITY_REWRITE
| NO_DECORRELATE
| NO_ELIM_GROUPBY
| NO_GATHER_OPTIMIZER_STATISTICS
| NO_INMEMORY
| NO_INMEMORY_PRUNING
| NOKEEP
| NONEDITIONABLE
| NO_OBJECT_LINK
| NO_PARTIAL_JOIN
| NO_PARTIAL_ROLLUP_PUSHDOWN
| NOPARTITION
| NO_PQ_CONCURRENT_UNION
| NO_PQ_REPLICATE
| NO_PQ_SKEW
| NO_PX_FAULT_TOLERANCE
| NORELOCATE
| NOREPLAY
| NO_ROOT_SW_FOR_LOCAL
| NO_SQL_TRANSLATION
| NO_USE_CUBE
| NO_USE_VECTOR_AGGREGATION
| NO_VECTOR_TRANSFORM
| NO_VECTOR_TRANSFORM_DIMS
| NO_VECTOR_TRANSFORM_FACT
| NO_ZONEMAP
| OBJ_ID
| OFFSET
| OLS
| OMIT
| ONE
| ORA_CHECK_ACL
| ORA_CHECK_PRIVILEGE
| ORA_CLUSTERING
| ORA_INVOKING_USER
| ORA_INVOKING_USERID
| ORA_INVOKING_XS_USER
| ORA_INVOKING_XS_USER_GUID
| ORA_RAWCOMPARE
| ORA_RAWCONCAT
| ORA_WRITE_TIME
| PARTIAL
| PARTIAL_JOIN
| PARTIAL_ROLLUP_PUSHDOWN
| PAST
| PATCH
| PATH_PREFIX
| PATTERN
| PER
| PERIOD
| PERMUTE
| PLUGGABLE
| POOL_16K
| POOL_2K
| POOL_32K
| POOL_4K
| POOL_8K
| PQ_CONCURRENT_UNION
| PQ_DISTRIBUTE_WINDOW
| PQ_FILTER
| PQ_REPLICATE
| PQ_SKEW
| PRELOAD
| PRETTY
| PREV
| PRINTBLOBTOCLOB
| PRIORITY
| PRIVILEGED
| PROXY
| PRUNING
| PX_FAULT_TOLERANCE
| REALM
| REDEFINE
| RELOCATE
| RESTART
| ROLESET
| ROWID_MAPPING_TABLE
| RUNNING
| SAVE
| SCRUB
| SDO_GEOM_MBR
| SECRET
| SERIAL
| SERVICE_NAME_CONVERT
| SERVICES
| SHARING
| SHELFLIFE
| SOURCE_FILE_DIRECTORY
| SOURCE_FILE_NAME_CONVERT
| SQL_TRANSLATION_PROFILE
| STANDARD_HASH
| STANDBYS
| STATE
| STATEMENT
| STREAM
| SUBSCRIBE
| SUBSET
| SUCCESS
| SYSBACKUP
| SYS_CHECK_PRIVILEGE
| SYSDG
| SYS_GET_COL_ACLIDS
| SYSGUID
| SYSKM
| SYS_MKXTI
| SYSOBJ
| SYS_OP_CYCLED_SEQ
| SYS_OP_HASH
| SYS_OP_KEY_VECTOR_CREATE
| SYS_OP_KEY_VECTOR_FILTER
| SYS_OP_KEY_VECTOR_FILTER_LIST
| SYS_OP_KEY_VECTOR_SUCCEEDED
| SYS_OP_KEY_VECTOR_USE
| SYS_OP_PART_ID
| SYS_OP_ZONE_ID
| SYS_RAW_TO_XSID
| SYS_XSID_TO_RAW
| SYS_ZMAP_FILTER
| SYS_ZMAP_REFRESH
| TAG
| TEXT
| TIER
| TIES
| TO_ACLID
| TRANSLATION
| TRUST
| UCS2
| UNCONDITIONAL
| UNMATCHED
| UNPLUG
| UNSUBSCRIBE
| USABLE
| USE_CUBE
| USE_HIDDEN_PARTITIONS
| USER_DATA
| USER_TABLESPACES
| USE_VECTOR_AGGREGATION
| USING_NO_EXPAND
| UTF16BE
| UTF16LE
| UTF32
| UTF8
| V1
| V2
| VALID_TIME_END
| VECTOR_TRANSFORM
| VECTOR_TRANSFORM_DIMS
| VECTOR_TRANSFORM_FACT
| VERIFIER
| VIOLATION
| VISIBILITY
| WEEK
| WEEKS
| WITH_PLSQL
| WRAPPER
| XS
| YEARS
| ZONEMAP
;
non_reserved_keywords_pre12c
: ABORT
| ABS
| ACCESSED
| ACCESS
| ACCOUNT
| ACOS
| ACTIVATE
| ACTIVE_COMPONENT
| ACTIVE_FUNCTION
| ACTIVE_TAG
| ADD_COLUMN
| ADD_GROUP
| ADD_MONTHS
| ADD
| ADJ_DATE
| ADMINISTER
| ADMINISTRATOR
| ADMIN
| ADVISE
| ADVISOR
| AFTER
| ALIAS
| ALLOCATE
| ALLOW
| ALL_ROWS
| ALWAYS
| ANALYZE
| ANCILLARY
| AND_EQUAL
| ANTIJOIN
| ANYSCHEMA
| APPENDCHILDXML
| APPEND
| APPEND_VALUES
| APPLY
| ARCHIVELOG
| ARCHIVE
| ARRAY
| ASCII
| ASCIISTR
| ASIN
| ASSEMBLY
| ASSOCIATE
| ASYNCHRONOUS
| ASYNC
| ATAN2
| ATAN
| AT
| ATTRIBUTE
| ATTRIBUTES
| AUTHENTICATED
| AUTHENTICATION
| AUTHID
| AUTHORIZATION
| AUTOALLOCATE
| AUTOEXTEND
| AUTOMATIC
| AUTO
| AVAILABILITY
| AVG
| BACKUP
| BASICFILE
| BASIC
| BATCH
| BECOME
| BEFORE
| BEGIN
| BEGIN_OUTLINE_DATA
| BEHALF
| BFILE
| BFILENAME
| BIGFILE
| BINARY_DOUBLE_INFINITY
| BINARY_DOUBLE
| BINARY_DOUBLE_NAN
| BINARY_FLOAT_INFINITY
| BINARY_FLOAT
| BINARY_FLOAT_NAN
| BINARY
| BIND_AWARE
| BINDING
| BIN_TO_NUM
| BITAND
| BITMAP
| BITMAPS
| BITMAP_TREE
| BITS
| BLOB
| BLOCK
| BLOCK_RANGE
| BLOCKSIZE
| BLOCKS
| BODY
| BOTH
| BOUND
| BRANCH
| BREADTH
| BROADCAST
| BUFFER_CACHE
| BUFFER
| BUFFER_POOL
| BUILD
| BULK
| BYPASS_RECURSIVE_CHECK
| BYPASS_UJVC
| BYTE
| CACHE_CB
| CACHE_INSTANCES
| CACHE
| CACHE_TEMP_TABLE
| CALL
| CANCEL
| CARDINALITY
| CASCADE
| CASE
| CAST
| CATEGORY
| CEIL
| CELL_FLASH_CACHE
| CERTIFICATE
| CFILE
| CHAINED
| CHANGE_DUPKEY_ERROR_INDEX
| CHANGE
| CHARACTER
| CHAR_CS
| CHARTOROWID
| CHECK_ACL_REWRITE
| CHECKPOINT
| CHILD
| CHOOSE
| CHR
| CHUNK
| CLASS
| CLEAR
| CLOB
| CLONE
| CLOSE_CACHED_OPEN_CURSORS
| CLOSE
| CLUSTER_BY_ROWID
| CLUSTER_ID
| CLUSTERING_FACTOR
| CLUSTER_PROBABILITY
| CLUSTER_SET
| COALESCE
| COALESCE_SQ
| COARSE
| CO_AUTH_IND
| COLD
| COLLECT
| COLUMNAR
| COLUMN_AUTH_INDICATOR
| COLUMN
| COLUMNS
| COLUMN_STATS
| COLUMN_VALUE
| COMMENT
| COMMIT
| COMMITTED
| COMPACT
| COMPATIBILITY
| COMPILE
| COMPLETE
| COMPLIANCE
| COMPOSE
| COMPOSITE_LIMIT
| COMPOSITE
| COMPOUND
| COMPUTE
| CONCAT
| CONFIRM
| CONFORMING
| CONNECT_BY_CB_WHR_ONLY
| CONNECT_BY_COMBINE_SW
| CONNECT_BY_COST_BASED
| CONNECT_BY_ELIM_DUPS
| CONNECT_BY_FILTERING
| CONNECT_BY_ISCYCLE
| CONNECT_BY_ISLEAF
| CONNECT_BY_ROOT
| CONNECT_TIME
| CONSIDER
| CONSISTENT
| CONSTANT
| CONST
| CONSTRAINT
| CONSTRAINTS
| CONTAINER
| CONTENT
| CONTENTS
| CONTEXT
| CONTINUE
| CONTROLFILE
| CONVERT
| CORR_K
| CORR
| CORR_S
| CORRUPTION
| CORRUPT_XID_ALL
| CORRUPT_XID
| COSH
| COS
| COST
| COST_XML_QUERY_REWRITE
| COUNT
| COVAR_POP
| COVAR_SAMP
| CPU_COSTING
| CPU_PER_CALL
| CPU_PER_SESSION
| CRASH
| CREATE_STORED_OUTLINES
| CREATION
| CROSSEDITION
| CROSS
| CSCONVERT
| CUBE_GB
| CUBE
| CUME_DISTM
| CUME_DIST
| CURRENT_DATE
| CURRENT
| CURRENT_SCHEMA
| CURRENT_TIME
| CURRENT_TIMESTAMP
| CURRENT_USER
| CURRENTV
| CURSOR
| CURSOR_SHARING_EXACT
| CURSOR_SPECIFIC_SEGMENT
| CV
| CYCLE
| DANGLING
| DATABASE
| DATAFILE
| DATAFILES
| DATA
| DATAOBJNO
| DATAOBJ_TO_PARTITION
| DATE_MODE
| DAY
| DBA
| DBA_RECYCLEBIN
| DBMS_STATS
| DB_ROLE_CHANGE
| DBTIMEZONE
| DB_VERSION
| DDL
| DEALLOCATE
| DEBUGGER
| DEBUG
| DECLARE
| DEC
| DECOMPOSE
| DECREMENT
| DECR
| DECRYPT
| DEDUPLICATE
| DEFAULTS
| DEFERRABLE
| DEFERRED
| DEFINED
| DEFINER
| DEGREE
| DELAY
| DELETEXML
| DEMAND
| DENSE_RANKM
| DENSE_RANK
| DEPENDENT
| DEPTH
| DEQUEUE
| DEREF
| DEREF_NO_REWRITE
| DETACHED
| DETERMINES
| DICTIONARY
| DIMENSION
| DIRECT_LOAD
| DIRECTORY
| DIRECT_PATH
| DISABLE
| DISABLE_PRESET
| DISABLE_RPKE
| DISALLOW
| DISASSOCIATE
| DISCONNECT
| DISKGROUP
| DISK
| DISKS
| DISMOUNT
| DISTINGUISHED
| DISTRIBUTED
| DML
| DML_UPDATE
| DOCFIDELITY
| DOCUMENT
| DOMAIN_INDEX_FILTER
| DOMAIN_INDEX_NO_SORT
| DOMAIN_INDEX_SORT
| DOUBLE
| DOWNGRADE
| DRIVING_SITE
| DROP_COLUMN
| DROP_GROUP
| DST_UPGRADE_INSERT_CONV
| DUMP
| DYNAMIC
| DYNAMIC_SAMPLING_EST_CDN
| DYNAMIC_SAMPLING
| EACH
| EDITIONING
| EDITION
| EDITIONS
| ELEMENT
| ELIMINATE_JOIN
| ELIMINATE_OBY
| ELIMINATE_OUTER_JOIN
| EMPTY_BLOB
| EMPTY_CLOB
| EMPTY
| ENABLE
| ENABLE_PRESET
| ENCODING
| ENCRYPTION
| ENCRYPT
| END
| END_OUTLINE_DATA
| ENFORCED
| ENFORCE
| ENQUEUE
| ENTERPRISE
| ENTITYESCAPING
| ENTRY
| ERROR_ARGUMENT
| ERROR
| ERROR_ON_OVERLAP_TIME
| ERRORS
| ESCAPE
| ESTIMATE
| EVALNAME
| EVALUATION
| EVENTS
| EVERY
| EXCEPTIONS
| EXCEPT
| EXCHANGE
| EXCLUDE
| EXCLUDING
| EXECUTE
| EXEMPT
| EXISTSNODE
| EXPAND_GSET_TO_UNION
| EXPAND_TABLE
| EXPIRE
| EXPLAIN
| EXPLOSION
| EXP
| EXPORT
| EXPR_CORR_CHECK
| EXTENDS
| EXTENT
| EXTENTS
| EXTERNALLY
| EXTERNAL
| EXTRACT
| EXTRACTVALUE
| EXTRA
| FACILITY
| FACT
| FACTORIZE_JOIN
| FAILED_LOGIN_ATTEMPTS
| FAILED
| FAILGROUP
| FALSE
| FAST
| FBTSCAN
| FEATURE_ID
| FEATURE_SET
| FEATURE_VALUE
| FILE
| FILESYSTEM_LIKE_LOGGING
| FILTER
| FINAL
| FINE
| FINISH
| FIRSTM
| FIRST
| FIRST_ROWS
| FIRST_VALUE
| FLAGGER
| FLASHBACK
| FLASH_CACHE
| FLOB
| FLOOR
| FLUSH
| FOLDER
| FOLLOWING
| FOLLOWS
| FORCE
| FORCE_XML_QUERY_REWRITE
| FOREIGN
| FOREVER
| FORWARD
| FRAGMENT_NUMBER
| FREELIST
| FREELISTS
| FREEPOOLS
| FRESH
| FROM_TZ
| FULL
| FULL_OUTER_JOIN_TO_OUTER
| FUNCTION
| FUNCTIONS
| GATHER_PLAN_STATISTICS
| GBY_CONC_ROLLUP
| GBY_PUSHDOWN
| GENERATED
| GLOBALLY
| GLOBAL
| GLOBAL_NAME
| GLOBAL_TOPIC_ENABLED
| GREATEST
| GROUP_BY
| GROUP_ID
| GROUPING_ID
| GROUPING
| GROUPS
| GUARANTEED
| GUARANTEE
| GUARD
| HASH_AJ
| HASHKEYS
| HASH
| HASH_SJ
| HEADER
| HEAP
| HELP
| HEXTORAW
| HEXTOREF
| HIDDEN_KEYWORD
| HIDE
| HIERARCHY
| HIGH
| HINTSET_BEGIN
| HINTSET_END
| HOT
| HOUR
| HWM_BROKERED
| HYBRID
| IDENTIFIER
| IDENTITY
| IDGENERATORS
| IDLE_TIME
| ID
| IF
| IGNORE
| IGNORE_OPTIM_EMBEDDED_HINTS
| IGNORE_ROW_ON_DUPKEY_INDEX
| IGNORE_WHERE_CLAUSE
| IMMEDIATE
| IMPACT
| IMPORT
| INCLUDE
| INCLUDE_VERSION
| INCLUDING
| INCREMENTAL
| INCREMENT
| INCR
| INDENT
| INDEX_ASC
| INDEX_COMBINE
| INDEX_DESC
| INDEXED
| INDEXES
| INDEX_FFS
| INDEX_FILTER
| INDEX_JOIN
| INDEX_ROWS
| INDEX_RRS
| INDEX_RS_ASC
| INDEX_RS_DESC
| INDEX_RS
| INDEX_SCAN
| INDEX_SKIP_SCAN
| INDEX_SS_ASC
| INDEX_SS_DESC
| INDEX_SS
| INDEX_STATS
| INDEXTYPE
| INDEXTYPES
| INDICATOR
| INFINITE
| INFORMATIONAL
| INITCAP
| INITIALIZED
| INITIALLY
| INITIAL
| INITRANS
| INLINE
| INLINE_XMLTYPE_NT
| IN_MEMORY_METADATA
| INNER
| INSERTCHILDXMLAFTER
| INSERTCHILDXMLBEFORE
| INSERTCHILDXML
| INSERTXMLAFTER
| INSERTXMLBEFORE
| INSTANCE
| INSTANCES
| INSTANTIABLE
| INSTANTLY
| INSTEAD
| INSTR2
| INSTR4
| INSTRB
| INSTRC
| INSTR
| INTERMEDIATE
| INTERNAL_CONVERT
| INTERNAL_USE
| INTERPRETED
| INTERVAL
| INT
| INVALIDATE
| INVISIBLE
| IN_XQUERY
| ISOLATION_LEVEL
| ISOLATION
| ITERATE
| ITERATION_NUMBER
| JAVA
| JOB
| JOIN
| KEEP_DUPLICATES
| KEEP
| KERBEROS
| KEY_LENGTH
| KEY
| KEYSIZE
| KEYS
| KILL
| LAG
| LAST_DAY
| LAST
| LAST_VALUE
| LATERAL
| LAYER
| LDAP_REGISTRATION_ENABLED
| LDAP_REGISTRATION
| LDAP_REG_SYNC_INTERVAL
| LEADING
| LEAD
| LEAST
| LEFT
| LENGTH2
| LENGTH4
| LENGTHB
| LENGTHC
| LENGTH
| LESS
| LEVEL
| LEVELS
| LIBRARY
| LIFE
| LIFETIME
| LIKE2
| LIKE4
| LIKEC
| LIKE_EXPAND
| LIMIT
| LINK
| LISTAGG
| LIST
| LN
| LNNVL
| LOAD
| LOB
| LOBNVL
| LOBS
| LOCAL_INDEXES
| LOCAL
| LOCALTIME
| LOCALTIMESTAMP
| LOCATION
| LOCATOR
| LOCKED
| LOGFILE
| LOGFILES
| LOGGING
| LOGICAL
| LOGICAL_READS_PER_CALL
| LOGICAL_READS_PER_SESSION
| LOG
| LOGOFF
| LOGON
| LOG_READ_ONLY_VIOLATIONS
| LOWER
| LOW
| LPAD
| LTRIM
| MAIN
| MAKE_REF
| MANAGED
| MANAGEMENT
| MANAGE
| MANAGER
| MANUAL
| MAPPING
| MASTER
| MATCHED
| MATERIALIZED
| MATERIALIZE
| MAXARCHLOGS
| MAXDATAFILES
| MAXEXTENTS
| MAXIMIZE
| MAXINSTANCES
| MAXLOGFILES
| MAXLOGHISTORY
| MAXLOGMEMBERS
| MAX
| MAXSIZE
| MAXTRANS
| MAXVALUE
| MEASURE
| MEASURES
| MEDIAN
| MEDIUM
| MEMBER
| MEMORY
| MERGEACTIONS
| MERGE_AJ
| MERGE_CONST_ON
| MERGE
| MERGE_SJ
| METHOD
| MIGRATE
| MIGRATION
| MINEXTENTS
| MINIMIZE
| MINIMUM
| MINING
| MIN
| MINUS_NULL
| MINUTE
| MINVALUE
| MIRRORCOLD
| MIRRORHOT
| MIRROR
| MLSLABEL
| MODEL_COMPILE_SUBQUERY
| MODEL_DONTVERIFY_UNIQUENESS
| MODEL_DYNAMIC_SUBQUERY
| MODEL_MIN_ANALYSIS
| MODEL
| MODEL_NO_ANALYSIS
| MODEL_PBY
| MODEL_PUSH_REF
| MODIFY_COLUMN_TYPE
| MODIFY
| MOD
| MONITORING
| MONITOR
| MONTH
| MONTHS_BETWEEN
| MOUNT
| MOUNTPATH
| MOVEMENT
| MOVE
| MULTISET
| MV_MERGE
| NAMED
| NAME
| NAMESPACE
| NAN
| NANVL
| NATIONAL
| NATIVE_FULL_OUTER_JOIN
| NATIVE
| NATURAL
| NAV
| NCHAR_CS
| NCHAR
| NCHR
| NCLOB
| NEEDED
| NESTED
| NESTED_TABLE_FAST_INSERT
| NESTED_TABLE_GET_REFS
| NESTED_TABLE_ID
| NESTED_TABLE_SET_REFS
| NESTED_TABLE_SET_SETID
| NETWORK
| NEVER
| NEW
| NEW_TIME
| NEXT_DAY
| NEXT
| NL_AJ
| NLJ_BATCHING
| NLJ_INDEX_FILTER
| NLJ_INDEX_SCAN
| NLJ_PREFETCH
| NLS_CALENDAR
| NLS_CHARACTERSET
| NLS_CHARSET_DECL_LEN
| NLS_CHARSET_ID
| NLS_CHARSET_NAME
| NLS_COMP
| NLS_CURRENCY
| NLS_DATE_FORMAT
| NLS_DATE_LANGUAGE
| NLS_INITCAP
| NLS_ISO_CURRENCY
| NL_SJ
| NLS_LANG
| NLS_LANGUAGE
| NLS_LENGTH_SEMANTICS
| NLS_LOWER
| NLS_NCHAR_CONV_EXCP
| NLS_NUMERIC_CHARACTERS
| NLS_SORT
| NLSSORT
| NLS_SPECIAL_CHARS
| NLS_TERRITORY
| NLS_UPPER
| NO_ACCESS
| NOAPPEND
| NOARCHIVELOG
| NOAUDIT
| NO_BASETABLE_MULTIMV_REWRITE
| NO_BIND_AWARE
| NO_BUFFER
| NOCACHE
| NO_CARTESIAN
| NO_CHECK_ACL_REWRITE
| NO_CLUSTER_BY_ROWID
| NO_COALESCE_SQ
| NO_CONNECT_BY_CB_WHR_ONLY
| NO_CONNECT_BY_COMBINE_SW
| NO_CONNECT_BY_COST_BASED
| NO_CONNECT_BY_ELIM_DUPS
| NO_CONNECT_BY_FILTERING
| NO_COST_XML_QUERY_REWRITE
| NO_CPU_COSTING
| NOCPU_COSTING
| NOCYCLE
| NODELAY
| NO_DOMAIN_INDEX_FILTER
| NO_DST_UPGRADE_INSERT_CONV
| NO_ELIMINATE_JOIN
| NO_ELIMINATE_OBY
| NO_ELIMINATE_OUTER_JOIN
| NOENTITYESCAPING
| NO_EXPAND_GSET_TO_UNION
| NO_EXPAND
| NO_EXPAND_TABLE
| NO_FACT
| NO_FACTORIZE_JOIN
| NO_FILTERING
| NOFORCE
| NO_FULL_OUTER_JOIN_TO_OUTER
| NO_GBY_PUSHDOWN
| NOGUARANTEE
| NO_INDEX_FFS
| NO_INDEX
| NO_INDEX_SS
| NO_LOAD
| NOLOCAL
| NOLOGGING
| NOMAPPING
| NOMAXVALUE
| NO_MERGE
| NOMINIMIZE
| NOMINVALUE
| NO_MODEL_PUSH_REF
| NO_MONITORING
| NOMONITORING
| NO_MONITOR
| NO_MULTIMV_REWRITE
| NO
| NO_NATIVE_FULL_OUTER_JOIN
| NONBLOCKING
| NONE
| NO_NLJ_BATCHING
| NO_NLJ_PREFETCH
| NONSCHEMA
| NOORDER
| NO_ORDER_ROLLUPS
| NO_OUTER_JOIN_TO_ANTI
| NO_OUTER_JOIN_TO_INNER
| NOOVERRIDE
| NO_PARALLEL_INDEX
| NOPARALLEL_INDEX
| NO_PARALLEL
| NOPARALLEL
| NO_PARTIAL_COMMIT
| NO_PLACE_DISTINCT
| NO_PLACE_GROUP_BY
| NO_PQ_MAP
| NO_PRUNE_GSETS
| NO_PULL_PRED
| NO_PUSH_PRED
| NO_PUSH_SUBQ
| NO_PX_JOIN_FILTER
| NO_QKN_BUFF
| NO_QUERY_TRANSFORMATION
| NO_REF_CASCADE
| NORELY
| NOREPAIR
| NORESETLOGS
| NO_RESULT_CACHE
| NOREVERSE
| NO_REWRITE
| NOREWRITE
| NORMAL
| NOROWDEPENDENCIES
| NOSCHEMACHECK
| NOSEGMENT
| NO_SEMIJOIN
| NO_SEMI_TO_INNER
| NO_SET_TO_JOIN
| NOSORT
| NO_SQL_TUNE
| NO_STAR_TRANSFORMATION
| NO_STATEMENT_QUEUING
| NO_STATS_GSETS
| NOSTRICT
| NO_SUBQUERY_PRUNING
| NO_SUBSTRB_PAD
| NO_SWAP_JOIN_INPUTS
| NOSWITCH
| NO_TABLE_LOOKUP_BY_NL
| NO_TEMP_TABLE
| NOTHING
| NOTIFICATION
| NO_TRANSFORM_DISTINCT_AGG
| NO_UNNEST
| NO_USE_HASH_AGGREGATION
| NO_USE_HASH_GBY_FOR_PUSHDOWN
| NO_USE_HASH
| NO_USE_INVISIBLE_INDEXES
| NO_USE_MERGE
| NO_USE_NL
| NOVALIDATE
| NO_XDB_FASTPATH_INSERT
| NO_XML_DML_REWRITE
| NO_XMLINDEX_REWRITE_IN_SELECT
| NO_XMLINDEX_REWRITE
| NO_XML_QUERY_REWRITE
| NTH_VALUE
| NTILE
| NULLIF
| NULLS
| NUMERIC
| NUM_INDEX_KEYS
| NUMTODSINTERVAL
| NUMTOYMINTERVAL
| NVARCHAR2
| NVL2
| NVL
| OBJECT2XML
| OBJECT
| OBJNO
| OBJNO_REUSE
| OCCURENCES
| OFFLINE
| OFF
| OIDINDEX
| OID
| OLAP
| OLD
| OLD_PUSH_PRED
| OLTP
| ONLINE
| ONLY
| OPAQUE
| OPAQUE_TRANSFORM
| OPAQUE_XCANONICAL
| OPCODE
| OPEN
| OPERATIONS
| OPERATOR
| OPT_ESTIMATE
| OPTIMAL
| OPTIMIZE
| OPTIMIZER_FEATURES_ENABLE
| OPTIMIZER_GOAL
| OPT_PARAM
| ORA_BRANCH
| ORADEBUG
| ORA_DST_AFFECTED
| ORA_DST_CONVERT
| ORA_DST_ERROR
| ORA_GET_ACLIDS
| ORA_GET_PRIVILEGES
| ORA_HASH
| ORA_ROWSCN
| ORA_ROWSCN_RAW
| ORA_ROWVERSION
| ORA_TABVERSION
| ORDERED
| ORDERED_PREDICATES
| ORDINALITY
| OR_EXPAND
| ORGANIZATION
| OR_PREDICATES
| OTHER
| OUTER_JOIN_TO_ANTI
| OUTER_JOIN_TO_INNER
| OUTER
| OUTLINE_LEAF
| OUTLINE
| OUT_OF_LINE
| OVERFLOW
| OVERFLOW_NOMOVE
| OVERLAPS
| OVER
| OWNER
| OWNERSHIP
| OWN
| PACKAGE
| PACKAGES
| PARALLEL_INDEX
| PARALLEL
| PARAMETERS
| PARAM
| PARENT
| PARITY
| PARTIALLY
| PARTITION_HASH
| PARTITION_LIST
| PARTITION
| PARTITION_RANGE
| PARTITIONS
| PARTNUMINST
| PASSING
| PASSWORD_GRACE_TIME
| PASSWORD_LIFE_TIME
| PASSWORD_LOCK_TIME
| PASSWORD
| PASSWORD_REUSE_MAX
| PASSWORD_REUSE_TIME
| PASSWORD_VERIFY_FUNCTION
| PATH
| PATHS
| PBL_HS_BEGIN
| PBL_HS_END
| PCTINCREASE
| PCTTHRESHOLD
| PCTUSED
| PCTVERSION
| PENDING
| PERCENTILE_CONT
| PERCENTILE_DISC
| PERCENT_KEYWORD
| PERCENT_RANKM
| PERCENT_RANK
| PERFORMANCE
| PERMANENT
| PERMISSION
| PFILE
| PHYSICAL
| PIKEY
| PIV_GB
| PIVOT
| PIV_SSF
| PLACE_DISTINCT
| PLACE_GROUP_BY
| PLAN
| PLSCOPE_SETTINGS
| PLSQL_CCFLAGS
| PLSQL_CODE_TYPE
| PLSQL_DEBUG
| PLSQL_OPTIMIZE_LEVEL
| PLSQL_WARNINGS
| POINT
| POLICY
| POST_TRANSACTION
| POWERMULTISET_BY_CARDINALITY
| POWERMULTISET
| POWER
| PQ_DISTRIBUTE
| PQ_MAP
| PQ_NOMAP
| PREBUILT
| PRECEDES
| PRECEDING
| PRECISION
| PRECOMPUTE_SUBQUERY
| PREDICATE_REORDERS
| PREDICTION_BOUNDS
| PREDICTION_COST
| PREDICTION_DETAILS
| PREDICTION
| PREDICTION_PROBABILITY
| PREDICTION_SET
| PREPARE
| PRESENT
| PRESENTNNV
| PRESENTV
| PRESERVE
| PRESERVE_OID
| PREVIOUS
| PRIMARY
| PRIVATE
| PRIVATE_SGA
| PRIVILEGE
| PRIVILEGES
| PROCEDURAL
| PROCEDURE
| PROCESS
| PROFILE
| PROGRAM
| PROJECT
| PROPAGATE
| PROTECTED
| PROTECTION
| PULL_PRED
| PURGE
| PUSH_PRED
| PUSH_SUBQ
| PX_GRANULE
| PX_JOIN_FILTER
| QB_NAME
| QUERY_BLOCK
| QUERY
| QUEUE_CURR
| QUEUE
| QUEUE_ROWP
| QUIESCE
| QUORUM
| QUOTA
| RANDOM_LOCAL
| RANDOM
| RANGE
| RANKM
| RANK
| RAPIDLY
| RATIO_TO_REPORT
| RAWTOHEX
| RAWTONHEX
| RBA
| RBO_OUTLINE
| RDBA
| READ
| READS
| REAL
| REBALANCE
| REBUILD
| RECORDS_PER_BLOCK
| RECOVERABLE
| RECOVER
| RECOVERY
| RECYCLEBIN
| RECYCLE
| REDACTION
| REDO
| REDUCED
| REDUNDANCY
| REF_CASCADE_CURSOR
| REFERENCED
| REFERENCE
| REFERENCES
| REFERENCING
| REF
| REFRESH
| REFTOHEX
| REGEXP_COUNT
| REGEXP_INSTR
| REGEXP_LIKE
| REGEXP_REPLACE
| REGEXP_SUBSTR
| REGISTER
| REGR_AVGX
| REGR_AVGY
| REGR_COUNT
| REGR_INTERCEPT
| REGR_R2
| REGR_SLOPE
| REGR_SXX
| REGR_SXY
| REGR_SYY
| REGULAR
| REJECT
| REKEY
| RELATIONAL
| RELY
| REMAINDER
| REMOTE_MAPPED
| REMOVE
| REPAIR
| REPEAT
| REPLACE
| REPLICATION
| REQUIRED
| RESETLOGS
| RESET
| RESIZE
| RESOLVE
| RESOLVER
| RESPECT
| RESTORE_AS_INTERVALS
| RESTORE
| RESTRICT_ALL_REF_CONS
| RESTRICTED
| RESTRICT
| RESULT_CACHE
| RESUMABLE
| RESUME
| RETENTION
| RETRY_ON_ROW_CHANGE
| RETURNING
| RETURN
| REUSE
| REVERSE
| REWRITE
| REWRITE_OR_ERROR
| RIGHT
| ROLE
| ROLES
| ROLLBACK
| ROLLING
| ROLLUP
| ROUND
| ROWDEPENDENCIES
| ROWID
| ROWIDTOCHAR
| ROWIDTONCHAR
| ROW_LENGTH
| ROW
| ROW_NUMBER
| ROWNUM
| ROWS
| RPAD
| RTRIM
| RULE
| RULES
| SALT
| SAMPLE
| SAVE_AS_INTERVALS
| SAVEPOINT
| SB4
| SCALE
| SCALE_ROWS
| SCAN_INSTANCES
| SCAN
| SCHEDULER
| SCHEMACHECK
| SCHEMA
| SCN_ASCENDING
| SCN
| SCOPE
| SD_ALL
| SD_INHIBIT
| SD_SHOW
| SEARCH
| SECOND
| SECUREFILE_DBA
| SECUREFILE
| SECURITY
| SEED
| SEG_BLOCK
| SEG_FILE
| SEGMENT
| SELECTIVITY
| SEMIJOIN_DRIVER
| SEMIJOIN
| SEMI_TO_INNER
| SEQUENCED
| SEQUENCE
| SEQUENTIAL
| SERIALIZABLE
| SERVERERROR
| SESSION_CACHED_CURSORS
| SESSION
| SESSIONS_PER_USER
| SESSIONTIMEZONE
| SESSIONTZNAME
| SETS
| SETTINGS
| SET_TO_JOIN
| SEVERE
| SHARED
| SHARED_POOL
| SHOW
| SHRINK
| SHUTDOWN
| SIBLINGS
| SID
| SIGNAL_COMPONENT
| SIGNAL_FUNCTION
| SIGN
| SIMPLE
| SINGLE
| SINGLETASK
| SINH
| SIN
| SKIP_EXT_OPTIMIZER
| SKIP_
| SKIP_UNQ_UNUSABLE_IDX
| SKIP_UNUSABLE_INDEXES
| SMALLFILE
| SNAPSHOT
| SOME
| SORT
| SOUNDEX
| SOURCE
| SPACE_KEYWORD
| SPECIFICATION
| SPFILE
| SPLIT
| SPREADSHEET
| SQLLDR
| SQL
| SQL_TRACE
| SQRT
| STALE
| STANDALONE
| STANDBY_MAX_DATA_DELAY
| STANDBY
| STAR
| STAR_TRANSFORMATION
| STARTUP
| STATEMENT_ID
| STATEMENT_QUEUING
| STATEMENTS
| STATIC
| STATISTICS
| STATS_BINOMIAL_TEST
| STATS_CROSSTAB
| STATS_F_TEST
| STATS_KS_TEST
| STATS_MODE
| STATS_MW_TEST
| STATS_ONE_WAY_ANOVA
| STATS_T_TEST_INDEP
| STATS_T_TEST_INDEPU
| STATS_T_TEST_ONE
| STATS_T_TEST_PAIRED
| STATS_WSR_TEST
| STDDEV
| STDDEV_POP
| STDDEV_SAMP
| STOP
| STORAGE
| STORE
| STREAMS
| STRICT
| STRING
| STRIPE_COLUMNS
| STRIPE_WIDTH
| STRIP
| STRUCTURE
| SUBMULTISET
| SUBPARTITION
| SUBPARTITION_REL
| SUBPARTITIONS
| SUBQUERIES
| SUBQUERY_PRUNING
| SUBSTITUTABLE
| SUBSTR2
| SUBSTR4
| SUBSTRB
| SUBSTRC
| SUBSTR
| SUCCESSFUL
| SUMMARY
| SUM
| SUPPLEMENTAL
| SUSPEND
| SWAP_JOIN_INPUTS
| SWITCH
| SWITCHOVER
| SYNCHRONOUS
| SYNC
| SYSASM
| SYS_AUDIT
| SYSAUX
| SYS_CHECKACL
| SYS_CONNECT_BY_PATH
| SYS_CONTEXT
| SYSDATE
| SYSDBA
| SYS_DBURIGEN
| SYS_DL_CURSOR
| SYS_DM_RXFORM_CHR
| SYS_DM_RXFORM_NUM
| SYS_DOM_COMPARE
| SYS_DST_PRIM2SEC
| SYS_DST_SEC2PRIM
| SYS_ET_BFILE_TO_RAW
| SYS_ET_BLOB_TO_IMAGE
| SYS_ET_IMAGE_TO_BLOB
| SYS_ET_RAW_TO_BFILE
| SYS_EXTPDTXT
| SYS_EXTRACT_UTC
| SYS_FBT_INSDEL
| SYS_FILTER_ACLS
| SYS_FNMATCHES
| SYS_FNREPLACE
| SYS_GET_ACLIDS
| SYS_GET_PRIVILEGES
| SYS_GETTOKENID
| SYS_GETXTIVAL
| SYS_GUID
| SYS_MAKEXML
| SYS_MAKE_XMLNODEID
| SYS_MKXMLATTR
| SYS_OP_ADT2BIN
| SYS_OP_ADTCONS
| SYS_OP_ALSCRVAL
| SYS_OP_ATG
| SYS_OP_BIN2ADT
| SYS_OP_BITVEC
| SYS_OP_BL2R
| SYS_OP_BLOOM_FILTER_LIST
| SYS_OP_BLOOM_FILTER
| SYS_OP_C2C
| SYS_OP_CAST
| SYS_OP_CEG
| SYS_OP_CL2C
| SYS_OP_COMBINED_HASH
| SYS_OP_COMP
| SYS_OP_CONVERT
| SYS_OP_COUNTCHG
| SYS_OP_CSCONV
| SYS_OP_CSCONVTEST
| SYS_OP_CSR
| SYS_OP_CSX_PATCH
| SYS_OP_DECOMP
| SYS_OP_DESCEND
| SYS_OP_DISTINCT
| SYS_OP_DRA
| SYS_OP_DUMP
| SYS_OP_DV_CHECK
| SYS_OP_ENFORCE_NOT_NULL
| SYSOPER
| SYS_OP_EXTRACT
| SYS_OP_GROUPING
| SYS_OP_GUID
| SYS_OP_IIX
| SYS_OP_ITR
| SYS_OP_LBID
| SYS_OP_LOBLOC2BLOB
| SYS_OP_LOBLOC2CLOB
| SYS_OP_LOBLOC2ID
| SYS_OP_LOBLOC2NCLOB
| SYS_OP_LOBLOC2TYP
| SYS_OP_LSVI
| SYS_OP_LVL
| SYS_OP_MAKEOID
| SYS_OP_MAP_NONNULL
| SYS_OP_MSR
| SYS_OP_NICOMBINE
| SYS_OP_NIEXTRACT
| SYS_OP_NII
| SYS_OP_NIX
| SYS_OP_NOEXPAND
| SYS_OP_NTCIMG
| SYS_OP_NUMTORAW
| SYS_OP_OIDVALUE
| SYS_OP_OPNSIZE
| SYS_OP_PAR_1
| SYS_OP_PARGID_1
| SYS_OP_PARGID
| SYS_OP_PAR
| SYS_OP_PIVOT
| SYS_OP_R2O
| SYS_OP_RAWTONUM
| SYS_OP_RDTM
| SYS_OP_REF
| SYS_OP_RMTD
| SYS_OP_ROWIDTOOBJ
| SYS_OP_RPB
| SYS_OPTLOBPRBSC
| SYS_OP_TOSETID
| SYS_OP_TPR
| SYS_OP_TRTB
| SYS_OPTXICMP
| SYS_OPTXQCASTASNQ
| SYS_OP_UNDESCEND
| SYS_OP_VECAND
| SYS_OP_VECBIT
| SYS_OP_VECOR
| SYS_OP_VECXOR
| SYS_OP_VERSION
| SYS_OP_VREF
| SYS_OP_VVD
| SYS_OP_XMLCONS_FOR_CSX
| SYS_OP_XPTHATG
| SYS_OP_XPTHIDX
| SYS_OP_XPTHOP
| SYS_OP_XTXT2SQLT
| SYS_ORDERKEY_DEPTH
| SYS_ORDERKEY_MAXCHILD
| SYS_ORDERKEY_PARENT
| SYS_PARALLEL_TXN
| SYS_PATHID_IS_ATTR
| SYS_PATHID_IS_NMSPC
| SYS_PATHID_LASTNAME
| SYS_PATHID_LASTNMSPC
| SYS_PATH_REVERSE
| SYS_PXQEXTRACT
| SYS_RID_ORDER
| SYS_ROW_DELTA
| SYS_SC_2_XMLT
| SYS_SYNRCIREDO
| SYSTEM_DEFINED
| SYSTEM
| SYSTIMESTAMP
| SYS_TYPEID
| SYS_UMAKEXML
| SYS_XMLANALYZE
| SYS_XMLCONTAINS
| SYS_XMLCONV
| SYS_XMLEXNSURI
| SYS_XMLGEN
| SYS_XMLI_LOC_ISNODE
| SYS_XMLI_LOC_ISTEXT
| SYS_XMLINSTR
| SYS_XMLLOCATOR_GETSVAL
| SYS_XMLNODEID_GETCID
| SYS_XMLNODEID_GETLOCATOR
| SYS_XMLNODEID_GETOKEY
| SYS_XMLNODEID_GETPATHID
| SYS_XMLNODEID_GETPTRID
| SYS_XMLNODEID_GETRID
| SYS_XMLNODEID_GETSVAL
| SYS_XMLNODEID_GETTID
| SYS_XMLNODEID
| SYS_XMLT_2_SC
| SYS_XMLTRANSLATE
| SYS_XMLTYPE2SQL
| SYS_XQ_ASQLCNV
| SYS_XQ_ATOMCNVCHK
| SYS_XQBASEURI
| SYS_XQCASTABLEERRH
| SYS_XQCODEP2STR
| SYS_XQCODEPEQ
| SYS_XQCON2SEQ
| SYS_XQCONCAT
| SYS_XQDELETE
| SYS_XQDFLTCOLATION
| SYS_XQDOC
| SYS_XQDOCURI
| SYS_XQDURDIV
| SYS_XQED4URI
| SYS_XQENDSWITH
| SYS_XQERRH
| SYS_XQERR
| SYS_XQESHTMLURI
| SYS_XQEXLOBVAL
| SYS_XQEXSTWRP
| SYS_XQEXTRACT
| SYS_XQEXTRREF
| SYS_XQEXVAL
| SYS_XQFB2STR
| SYS_XQFNBOOL
| SYS_XQFNCMP
| SYS_XQFNDATIM
| SYS_XQFNLNAME
| SYS_XQFNNM
| SYS_XQFNNSURI
| SYS_XQFNPREDTRUTH
| SYS_XQFNQNM
| SYS_XQFNROOT
| SYS_XQFORMATNUM
| SYS_XQFTCONTAIN
| SYS_XQFUNCR
| SYS_XQGETCONTENT
| SYS_XQINDXOF
| SYS_XQINSERT
| SYS_XQINSPFX
| SYS_XQIRI2URI
| SYS_XQLANG
| SYS_XQLLNMFRMQNM
| SYS_XQMKNODEREF
| SYS_XQNILLED
| SYS_XQNODENAME
| SYS_XQNORMSPACE
| SYS_XQNORMUCODE
| SYS_XQ_NRNG
| SYS_XQNSP4PFX
| SYS_XQNSPFRMQNM
| SYS_XQPFXFRMQNM
| SYS_XQ_PKSQL2XML
| SYS_XQPOLYABS
| SYS_XQPOLYADD
| SYS_XQPOLYCEL
| SYS_XQPOLYCSTBL
| SYS_XQPOLYCST
| SYS_XQPOLYDIV
| SYS_XQPOLYFLR
| SYS_XQPOLYMOD
| SYS_XQPOLYMUL
| SYS_XQPOLYRND
| SYS_XQPOLYSQRT
| SYS_XQPOLYSUB
| SYS_XQPOLYUMUS
| SYS_XQPOLYUPLS
| SYS_XQPOLYVEQ
| SYS_XQPOLYVGE
| SYS_XQPOLYVGT
| SYS_XQPOLYVLE
| SYS_XQPOLYVLT
| SYS_XQPOLYVNE
| SYS_XQREF2VAL
| SYS_XQRENAME
| SYS_XQREPLACE
| SYS_XQRESVURI
| SYS_XQRNDHALF2EVN
| SYS_XQRSLVQNM
| SYS_XQRYENVPGET
| SYS_XQRYVARGET
| SYS_XQRYWRP
| SYS_XQSEQ2CON4XC
| SYS_XQSEQ2CON
| SYS_XQSEQDEEPEQ
| SYS_XQSEQINSB
| SYS_XQSEQRM
| SYS_XQSEQRVS
| SYS_XQSEQSUB
| SYS_XQSEQTYPMATCH
| SYS_XQSTARTSWITH
| SYS_XQSTATBURI
| SYS_XQSTR2CODEP
| SYS_XQSTRJOIN
| SYS_XQSUBSTRAFT
| SYS_XQSUBSTRBEF
| SYS_XQTOKENIZE
| SYS_XQTREATAS
| SYS_XQ_UPKXML2SQL
| SYS_XQXFORM
| TABLE
| TABLE_LOOKUP_BY_NL
| TABLES
| TABLESPACE
| TABLESPACE_NO
| TABLE_STATS
| TABNO
| TANH
| TAN
| TBLORIDXPARTNUM
| TEMPFILE
| TEMPLATE
| TEMPORARY
| TEMP_TABLE
| TEST
| THAN
| THE
| THEN
| THREAD
| THROUGH
| TIME
| TIMEOUT
| TIMES
| TIMESTAMP
| TIMEZONE_ABBR
| TIMEZONE_HOUR
| TIMEZONE_MINUTE
| TIME_ZONE
| TIMEZONE_OFFSET
| TIMEZONE_REGION
| TIV_GB
| TIV_SSF
| TO_BINARY_DOUBLE
| TO_BINARY_FLOAT
| TO_BLOB
| TO_CHAR
| TO_CLOB
| TO_DATE
| TO_DSINTERVAL
| TO_LOB
| TO_MULTI_BYTE
| TO_NCHAR
| TO_NCLOB
| TO_NUMBER
| TOPLEVEL
| TO_SINGLE_BYTE
| TO_TIME
| TO_TIMESTAMP
| TO_TIMESTAMP_TZ
| TO_TIME_TZ
| TO_YMINTERVAL
| TRACE
| TRACING
| TRACKING
| TRAILING
| TRANSACTION
| TRANSFORM_DISTINCT_AGG
| TRANSITIONAL
| TRANSITION
| TRANSLATE
| TREAT
| TRIGGERS
| TRIM
| TRUE
| TRUNCATE
| TRUNC
| TRUSTED
| TUNING
| TX
| TYPE
| TYPES
| TZ_OFFSET
| UB2
| UBA
| UID
| UNARCHIVED
| UNBOUNDED
| UNBOUND
| UNDER
| UNDO
| UNDROP
| UNIFORM
| UNISTR
| UNLIMITED
| UNLOAD
| UNLOCK
| UNNEST_INNERJ_DISTINCT_VIEW
| UNNEST
| UNNEST_NOSEMIJ_NODISTINCTVIEW
| UNNEST_SEMIJ_VIEW
| UNPACKED
| UNPIVOT
| UNPROTECTED
| UNQUIESCE
| UNRECOVERABLE
| UNRESTRICTED
| UNTIL
| UNUSABLE
| UNUSED
| UPDATABLE
| UPDATED
| UPDATEXML
| UPD_INDEXES
| UPD_JOININDEX
| UPGRADE
| UPPER
| UPSERT
| UROWID
| USAGE
| USE_ANTI
| USE_CONCAT
| USE_HASH_AGGREGATION
| USE_HASH_GBY_FOR_PUSHDOWN
| USE_HASH
| USE_INVISIBLE_INDEXES
| USE_MERGE_CARTESIAN
| USE_MERGE
| USE
| USE_NL
| USE_NL_WITH_INDEX
| USE_PRIVATE_OUTLINES
| USER_DEFINED
| USERENV
| USERGROUP
| USER
| USER_RECYCLEBIN
| USERS
| USE_SEMI
| USE_STORED_OUTLINES
| USE_TTT_FOR_GSETS
| USE_WEAK_NAME_RESL
| USING
| VALIDATE
| VALIDATION
| VALUE
| VARIANCE
| VAR_POP
| VARRAY
| VARRAYS
| VAR_SAMP
| VARYING
| VECTOR_READ
| VECTOR_READ_TRACE
| VERIFY
| VERSIONING
| VERSION
| VERSIONS_ENDSCN
| VERSIONS_ENDTIME
| VERSIONS
| VERSIONS_OPERATION
| VERSIONS_STARTSCN
| VERSIONS_STARTTIME
| VERSIONS_XID
| VIRTUAL
| VISIBLE
| VOLUME
| VSIZE
| WAIT
| WALLET
| WELLFORMED
| WHENEVER
| WHEN
| WHITESPACE
| WIDTH_BUCKET
| WITHIN
| WITHOUT
| WORK
| WRAPPED
| WRITE
| XDB_FASTPATH_INSERT
| X_DYN_PRUNE
| XID
| XML2OBJECT
| XMLATTRIBUTES
| XMLCAST
| XMLCDATA
| XMLCOLATTVAL
| XMLCOMMENT
| XMLCONCAT
| XMLDIFF
| XML_DML_RWT_STMT
| XMLELEMENT
| XMLEXISTS2
| XMLEXISTS
| XMLFOREST
| XMLINDEX_REWRITE_IN_SELECT
| XMLINDEX_REWRITE
| XMLINDEX_SEL_IDX_TBL
| XMLISNODE
| XMLISVALID
| XML
| XMLNAMESPACES
| XMLPARSE
| XMLPATCH
| XMLPI
| XMLQUERY
| XMLQUERYVAL
| XMLROOT
| XMLSCHEMA
| XMLSERIALIZE
| XMLTABLE
| XMLTRANSFORMBLOB
| XMLTRANSFORM
| XMLTYPE
| XPATHTABLE
| XS_SYS_CONTEXT
| YEAR
| YES
| ZONE
;
string_function_name
: CHR
| DECODE
| SUBSTR
| TO_CHAR
| TRIM
;
numeric_function_name
: AVG
| COUNT
| NVL
| ROUND
| SUM
;

View File

@ -0,0 +1,133 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.relational.RelationalDatabaseSchema;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.schema.SchemaChangeEvent;
import io.debezium.schema.SchemaChangeEvent.SchemaChangeEventType;
import io.debezium.util.Clock;
import oracle.streams.ChunkColumnValue;
import oracle.streams.DDLLCR;
import oracle.streams.LCR;
import oracle.streams.RowLCR;
import oracle.streams.StreamsException;
import oracle.streams.XStreamLCRCallbackHandler;
class LcrEventHandler implements XStreamLCRCallbackHandler {
private static final Logger LOGGER = LoggerFactory.getLogger(OracleStreamingChangeEventSource.class);
private final ErrorHandler errorHandler;
private final EventDispatcher dispatcher;
private final Clock clock;
private final RelationalDatabaseSchema schema;
private final OracleOffsetContext offsetContext;
public LcrEventHandler(ErrorHandler errorHandler, EventDispatcher dispatcher, Clock clock, RelationalDatabaseSchema schema, OracleOffsetContext offsetContext) {
this.errorHandler = errorHandler;
this.dispatcher = dispatcher;
this.clock = clock;
this.schema = schema;
this.offsetContext = offsetContext;
}
@Override
public void processLCR(LCR lcr) throws StreamsException {
LOGGER.debug("Processing event {}", lcr);
offsetContext.setPosition(lcr.getPosition());
offsetContext.setTransactionId(lcr.getTransactionId());
offsetContext.setSourceTime(lcr.getSourceTime().timestampValue().toInstant());
try {
if(lcr instanceof RowLCR) {
dispatchDataChangeEvent((RowLCR) lcr);
}
else if (lcr instanceof DDLLCR) {
dispatchSchemaChangeEvent((DDLLCR) lcr);
}
}
catch(Exception e) {
LOGGER.error("Exception occurred when handling event {}", lcr);
errorHandler.setProducerThrowable(e);
}
}
private void dispatchDataChangeEvent(RowLCR lcr) {
try {
if(RowLCR.COMMIT.equals(lcr.getCommandType())) {
return;
}
TableId tableId = getTableId(lcr);
dispatcher.dispatchDataChangeEvent(
offsetContext,
tableId,
() -> new OracleChangeRecordEmitter(lcr, schema.getTable(tableId), clock),
DataChangeEvent::new
);
}
catch (InterruptedException e) {
Thread.interrupted();
}
}
private void dispatchSchemaChangeEvent(DDLLCR ddlLcr) {
try {
TableId tableId = getTableId(ddlLcr);
dispatcher.dispatchSchemaChangeEvent(
tableId,
(tid, r) -> {
SchemaChangeEventType eventType = getSchemaChangeEventType(ddlLcr);
if (eventType != null) {
Table table = new OracleDdlParser().parseCreateTable(tid, ddlLcr.getDDLText());
r.schemaChangeEvent(new SchemaChangeEvent(ddlLcr.getDDLText(), table, eventType));
}
}
);
}
catch (InterruptedException e) {
Thread.interrupted();
}
}
private SchemaChangeEventType getSchemaChangeEventType(DDLLCR ddlLcr) {
switch(ddlLcr.getCommandType()) {
case "CREATE TABLE": return SchemaChangeEventType.CREATE;
case "ALTER TABLE": throw new UnsupportedOperationException("ALTER TABLE not yet implemented");
case "DROP TABLE": throw new UnsupportedOperationException("DROP TABLE not yet implemented");
default: return null;
}
}
private TableId getTableId(LCR lcr) {
return new TableId(lcr.getSourceDatabaseName(), lcr.getObjectOwner(), lcr.getObjectName());
}
@Override
public void processChunk(ChunkColumnValue arg0) throws StreamsException {
throw new UnsupportedOperationException("Not yet implemented");
}
@Override
public LCR createLCR() throws StreamsException {
throw new UnsupportedOperationException("Should never be called");
}
@Override
public ChunkColumnValue createChunk() throws StreamsException {
throw new UnsupportedOperationException("Should never be called");
}
}

View File

@ -0,0 +1,24 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.util.Properties;
import io.debezium.util.IoUtil;
/**
* Information about this module.
*
* @author Gunnar Morling
*/
public final class Module {
private static final Properties INFO = IoUtil.loadProperties(Module.class, "io/debezium/connector/oracle/build.version");
public static String version() {
return INFO.getProperty("version");
}
}

View File

@ -0,0 +1,52 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.spi.ChangeEventSourceFactory;
import io.debezium.pipeline.source.spi.SnapshotChangeEventSource;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.util.Clock;
public class OracleChangeEventSourceFactory implements ChangeEventSourceFactory {
private final OracleConnectorConfig configuration;
private final OracleConnection jdbcConnection;
private final ErrorHandler errorHandler;
private final EventDispatcher dispatcher;
private final Clock clock;
private final OracleDatabaseSchema schema;
public OracleChangeEventSourceFactory(OracleConnectorConfig configuration, OracleConnection jdbcConnection,
ErrorHandler errorHandler, EventDispatcher dispatcher, Clock clock, OracleDatabaseSchema schema) {
this.configuration = configuration;
this.jdbcConnection = jdbcConnection;
this.errorHandler = errorHandler;
this.dispatcher = dispatcher;
this.clock = clock;
this.schema = schema;
}
@Override
public SnapshotChangeEventSource getSnapshotChangeEventSource() {
return new OracleSnapshotChangeEventSource(configuration, jdbcConnection, schema);
}
@Override
public StreamingChangeEventSource getStreamingChangeEventSource(OffsetContext offsetContext) {
return new OracleStreamingChangeEventSource(
configuration,
(OracleOffsetContext) offsetContext,
jdbcConnection,
dispatcher,
errorHandler,
clock,
schema
);
}
}

View File

@ -0,0 +1,76 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import org.apache.kafka.connect.data.Struct;
import io.debezium.data.Envelope.Operation;
import io.debezium.pipeline.spi.ChangeRecordEmitter;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.relational.Table;
import io.debezium.relational.TableSchema;
import io.debezium.schema.DataCollectionSchema;
import io.debezium.util.Clock;
import oracle.streams.ColumnValue;
import oracle.streams.RowLCR;
// TODO extract RelationalChangeRecordEmitter
public class OracleChangeRecordEmitter implements ChangeRecordEmitter {
private final RowLCR lcr;
private final Table table;
private final Clock clock;
public OracleChangeRecordEmitter(RowLCR lcr, Table table, Clock clock) {
this.lcr = lcr;
this.table = table;
this.clock = clock;
}
@Override
public void emitChangeRecords(OffsetContext offsetContext, DataCollectionSchema schema, Receiver receiver) throws InterruptedException {
TableSchema tableSchema = (TableSchema) schema;
Operation operation = getOperation();
if (operation == Operation.CREATE) {
Object[] columnValues = getColumnValues(lcr.getNewValues());
Object key = tableSchema.keyFromColumnData(columnValues);
Struct value = tableSchema.valueFromColumnData(columnValues);
Struct envelope = tableSchema.getEnvelopeSchema().create(value, offsetContext.getSourceInfo(), clock.currentTimeInMillis());
receiver.changeRecord(operation, key, envelope, offsetContext);
}
else if (operation == Operation.UPDATE) {
throw new UnsupportedOperationException("Not yet implemented");
// TODO handle PK change
}
else if (operation == Operation.DELETE) {
throw new UnsupportedOperationException("Not yet implemented");
}
}
private Operation getOperation() {
switch(lcr.getCommandType()) {
case "INSERT": return Operation.CREATE;
case "DELETE": return Operation.DELETE;
case "UPDATE": return Operation.UPDATE;
default: throw new IllegalArgumentException("Received event of unexpected command type: " + lcr);
}
}
private Object[] getColumnValues(ColumnValue[] columnValues) {
Object[] values = new Object[table.columnNames().size()];
for (ColumnValue columnValue : columnValues) {
int index = table.columnNames().indexOf(columnValue.getColumnName());
values[index] = columnValue.getColumnData();
}
return values;
}
}

View File

@ -0,0 +1,120 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.sql.SQLException;
import java.sql.Statement;
import java.sql.Types;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.config.Configuration;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.Column;
import io.debezium.relational.TableEditor;
import io.debezium.relational.TableId;
import io.debezium.relational.Tables;
import io.debezium.relational.Tables.ColumnNameFilter;
import io.debezium.relational.Tables.TableNameFilter;
public class OracleConnection extends JdbcConnection {
private final static Logger LOGGER = LoggerFactory.getLogger(OracleConnection.class);
public OracleConnection(Configuration config, ConnectionFactory connectionFactory) {
super(config, connectionFactory);
}
public void setSessionToPdb(String pdbName) {
Statement statement = null;
try {
statement = connection().createStatement();
statement.execute("alter session set container=" + pdbName);
}
catch (SQLException e) {
throw new RuntimeException(e);
}
finally {
if (statement != null) {
try {
statement.close();
}
catch (SQLException e) {
LOGGER.error("Couldn't close statement", e);
}
}
}
}
public void resetSessionToCdb() {
Statement statement = null;
try {
statement = connection().createStatement();
statement.execute("alter session set container=cdb$root");
}
catch (SQLException e) {
throw new RuntimeException(e);
}
finally {
if (statement != null) {
try {
statement.close();
}
catch (SQLException e) {
LOGGER.error("Couldn't close statement", e);
}
}
}
}
@Override
public Set<TableId> readTableNames(String databaseCatalog, String schemaNamePattern, String tableNamePattern,
String[] tableTypes) throws SQLException {
Set<TableId> tableIds = super.readTableNames(null, schemaNamePattern, tableNamePattern, tableTypes);
return tableIds.stream()
.map(t -> new TableId(databaseCatalog, t.schema(), t.table()))
.collect(Collectors.toSet());
}
@Override
public void readSchema(Tables tables, String databaseCatalog, String schemaNamePattern, TableNameFilter tableFilter,
ColumnNameFilter columnFilter, boolean removeTablesNotFoundInJdbc) throws SQLException {
super.readSchema(tables, null, schemaNamePattern, tableFilter, columnFilter, removeTablesNotFoundInJdbc);
Set<TableId> tableIds = new HashSet<>(tables.tableIds());
for (TableId tableId : tableIds) {
TableEditor editor = tables.editTable(tableId);
editor.tableId(new TableId(databaseCatalog, tableId.schema(), tableId.table()));
List<String> columnNames = new ArrayList<>(editor.columnNames());
for (String columnName : columnNames) {
Column column = editor.columnWithName(columnName);
if (column.jdbcType() == Types.TIMESTAMP) {
editor.addColumn(
column.edit()
.length(column.scale())
.scale(-1)
.create()
);
}
}
tables.overwriteTable(editor.create());
tables.removeTable(tableId);
}
}
}

View File

@ -0,0 +1,29 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
import io.debezium.jdbc.JdbcConfiguration;
import io.debezium.jdbc.JdbcConnection.ConnectionFactory;
public class OracleConnectionFactory implements ConnectionFactory {
@Override
public Connection connect(JdbcConfiguration config) throws SQLException {
String hostName = config.getHostname();
int port = config.getPort();
String database = config.getDatabase();
String user = config.getUser();
String password = config.getPassword();
return DriverManager.getConnection(
"jdbc:oracle:oci:@" + hostName + ":" + port + "/" + database, user, password
);
}
}

View File

@ -0,0 +1,53 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.connect.connector.Task;
import org.apache.kafka.connect.source.SourceConnector;
public class OracleConnector extends SourceConnector {
private Map<String, String> properties;
@Override
public String version() {
return Module.version();
}
@Override
public void start(Map<String, String> props) {
this.properties = Collections.unmodifiableMap(new HashMap<>(props));
}
@Override
public Class<? extends Task> taskClass() {
return OracleConnectorTask.class;
}
@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
if (maxTasks > 1) {
throw new IllegalArgumentException("Only a single connector task may be started");
}
return Collections.singletonList(properties);
}
@Override
public void stop() {
}
@Override
public ConfigDef config() {
return OracleConnectorConfig.configDef();
}
}

View File

@ -0,0 +1,105 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.ConfigDef.Width;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration;
import io.debezium.config.Field;
import io.debezium.jdbc.JdbcConfiguration;
public class OracleConnectorConfig extends CommonConnectorConfig {
// TODO pull up to RelationalConnectorConfig
public static final String DATABASE_CONFIG_PREFIX = "database.";
public static final Field LOGICAL_NAME = Field.create("database.server.name")
.withDisplayName("Namespace")
.withType(Type.STRING)
.withWidth(Width.MEDIUM)
.withImportance(Importance.HIGH)
.withValidation(Field::isRequired)
// TODO
//.withValidation(Field::isRequired, MySqlConnectorConfig::validateServerNameIsDifferentFromHistoryTopicName)
.withDescription("Unique name that identifies the database server and all recorded offsets, and"
+ "that is used as a prefix for all schemas and topics. "
+ "Each distinct MySQL installation should have a separate namespace and monitored by "
+ "at most one Debezium connector.");
public static final Field DATABASE_NAME = Field.create(DATABASE_CONFIG_PREFIX + JdbcConfiguration.DATABASE)
.withDisplayName("Database name")
.withType(Type.STRING)
.withWidth(Width.MEDIUM)
.withImportance(Importance.HIGH)
.withValidation(Field::isRequired)
.withDescription("The name of the database the connector should be monitoring. When working with a "
+ "multi-tenant set-up, must be set to the CDB name.");
public static final Field PDB_NAME = Field.create(DATABASE_CONFIG_PREFIX + "pdb.name")
.withDisplayName("PDB name")
.withType(Type.STRING)
.withWidth(Width.MEDIUM)
.withImportance(Importance.HIGH)
.withDescription("Name of the pluggable database when working with a multi-tenant set-up. "
+ "The CDB name must be given via " + DATABASE_NAME.name() + " in this case.");
public static final Field XSTREAM_SERVER_NAME = Field.create(DATABASE_CONFIG_PREFIX + "out.server.name")
.withDisplayName("XStream out server name")
.withType(Type.STRING)
.withWidth(Width.MEDIUM)
.withImportance(Importance.HIGH)
.withDescription("Name of the XStream Out server to connect to.");
/**
* The set of {@link Field}s defined as part of this configuration.
*/
public static Field.Set ALL_FIELDS = Field.setOf(
LOGICAL_NAME,
DATABASE_NAME,
PDB_NAME,
XSTREAM_SERVER_NAME,
CommonConnectorConfig.POLL_INTERVAL_MS,
CommonConnectorConfig.MAX_BATCH_SIZE,
CommonConnectorConfig.MAX_QUEUE_SIZE
);
private final String databaseName;
private final String pdbName;
private final String xoutServerName;
public OracleConnectorConfig(Configuration config) {
super(config, LOGICAL_NAME);
this.databaseName = config.getString(DATABASE_NAME);
this.pdbName = config.getString(PDB_NAME);
this.xoutServerName = config.getString(XSTREAM_SERVER_NAME);
}
public static ConfigDef configDef() {
ConfigDef config = new ConfigDef();
Field.group(config, "Oracle", LOGICAL_NAME, DATABASE_NAME, PDB_NAME, XSTREAM_SERVER_NAME);
Field.group(config, "Connector", CommonConnectorConfig.POLL_INTERVAL_MS, CommonConnectorConfig.MAX_BATCH_SIZE, CommonConnectorConfig.MAX_QUEUE_SIZE);
return config;
}
public String getDatabaseName() {
return databaseName;
}
public String getPdbName() {
return pdbName;
}
public String getXoutServerName() {
return xoutServerName;
}
}

View File

@ -0,0 +1,146 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.sql.SQLException;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import org.apache.kafka.connect.source.SourceRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.config.Configuration;
import io.debezium.config.Field;
import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.common.BaseSourceTask;
import io.debezium.pipeline.ChangeEventSourceCoordinator;
import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.schema.TopicSelector;
import io.debezium.util.Clock;
import io.debezium.util.SchemaNameAdjuster;
public class OracleConnectorTask extends BaseSourceTask {
private static final Logger LOGGER = LoggerFactory.getLogger(OracleConnectorTask.class);
private static final String CONTEXT_NAME = "oracle-connector-task";
private static enum State {
RUNNING, STOPPED;
}
private volatile OracleTaskContext taskContext;
private volatile ChangeEventQueue<Object> queue;
private volatile OracleConnection jdbcConnection;
private volatile ChangeEventSourceCoordinator coordinator;
private volatile ErrorHandler errorHandler;
private final AtomicReference<State> state = new AtomicReference<State>(State.STOPPED);
@Override
public String version() {
return Module.version();
}
@Override
public void start(Configuration config) {
if (!state.compareAndSet(State.STOPPED, State.RUNNING)) {
LOGGER.info("Connector has already been started");
return;
}
OracleConnectorConfig connectorConfig = new OracleConnectorConfig(config);
taskContext = new OracleTaskContext(connectorConfig);
Clock clock = Clock.system();
// Set up the task record queue ...
this.queue = new ChangeEventQueue.Builder<Object>()
.pollInterval(connectorConfig.getPollInterval())
.maxBatchSize(connectorConfig.getMaxBatchSize())
.maxQueueSize(connectorConfig.getMaxQueueSize())
.loggingContextSupplier(() -> taskContext.configureLoggingContext(CONTEXT_NAME))
.build();
errorHandler = new ErrorHandler(OracleConnector.class, connectorConfig.getLogicalName(), queue, this::cleanupResources);
TopicSelector topicSelector = OracleTopicSelector.defaultSelector(connectorConfig.getLogicalName());
Configuration jdbcConfig = config.subset("database.", true);
jdbcConnection = new OracleConnection(jdbcConfig, new OracleConnectionFactory());
SchemaNameAdjuster schemaNameAdjuster = SchemaNameAdjuster.create(LOGGER);
OracleDatabaseSchema schema = new OracleDatabaseSchema(schemaNameAdjuster, topicSelector);
EventDispatcher dispatcher = new EventDispatcher(errorHandler, topicSelector, schema, queue);
coordinator = new ChangeEventSourceCoordinator(
errorHandler,
OracleConnector.class,
connectorConfig.getLogicalName(),
new OracleChangeEventSourceFactory(connectorConfig, jdbcConnection, errorHandler, dispatcher, clock, schema)
);
coordinator.start();
}
@Override
public List<SourceRecord> poll() throws InterruptedException {
// TODO
List records = queue.poll();
return ((List<DataChangeEvent>)records).stream()
.map(DataChangeEvent::getRecord)
.collect(Collectors.toList());
}
@Override
public void stop() {
cleanupResources();
}
private void cleanupResources() {
if (!state.compareAndSet(State.RUNNING, State.STOPPED)) {
LOGGER.info("Connector has already been stopped");
return;
}
try {
if (coordinator != null) {
coordinator.stop();
}
}
catch (InterruptedException e) {
Thread.interrupted();
LOGGER.error("Interrupted while stopping coordinator", e);
}
try {
if (errorHandler != null) {
errorHandler.stop();
}
}
catch (InterruptedException e) {
Thread.interrupted();
LOGGER.error("Interrupted while stopping", e);
}
try {
if (jdbcConnection != null) {
jdbcConnection.close();
}
}
catch (SQLException e) {
LOGGER.error("Exception while closing JDBC connection", e);
}
}
@Override
protected Iterable<Field> getAllConfigurationFields() {
return OracleConnectorConfig.ALL_FIELDS;
}
}

View File

@ -0,0 +1,69 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.util.HashMap;
import java.util.Map;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.relational.RelationalDatabaseSchema;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.relational.TableSchema;
import io.debezium.relational.TableSchemaBuilder;
import io.debezium.relational.Tables;
import io.debezium.schema.DataCollectionId;
import io.debezium.schema.DataCollectionSchema;
import io.debezium.schema.SchemaChangeEvent;
import io.debezium.schema.TopicSelector;
import io.debezium.util.SchemaNameAdjuster;
// TODO generify into HistorizedRelationalDatabaseSchema
public class OracleDatabaseSchema implements RelationalDatabaseSchema {
private static final Logger LOGGER = LoggerFactory.getLogger(OracleDatabaseSchema.class);
private final TopicSelector topicSelector;
private final Tables tables;
private final Map<TableId, TableSchema> schemas;
private final TableSchemaBuilder tableSchemaBuilder;
public OracleDatabaseSchema(SchemaNameAdjuster schemaNameAdjuster, TopicSelector topicSelector) {
this.topicSelector = topicSelector;
this.tables = new Tables();
this.schemas = new HashMap<>();
this.tableSchemaBuilder = new TableSchemaBuilder(new OracleValueConverters(), schemaNameAdjuster, SourceInfo.SCHEMA);
}
@Override
public void applySchemaChange(SchemaChangeEvent schemaChange) {
LOGGER.debug("Applying schema change event {}", schemaChange);
Table table = schemaChange.getTable();
tables.overwriteTable(table);
schemas.put(table.id(), tableSchemaBuilder.create(null, getEnvelopeSchemaName(table), table, null, null));
}
private String getEnvelopeSchemaName(Table table) {
return topicSelector.topicNameFor(table.id()) + ".Envelope";
}
@Override
public DataCollectionSchema getDataCollectionSchema(DataCollectionId id) {
return schemas.get(id);
}
@Override
public Table getTable(TableId id) {
return tables.forTable(id);
}
}

View File

@ -0,0 +1,138 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.sql.Types;
import java.util.List;
import java.util.Locale;
import java.util.stream.Collectors;
import org.antlr.v4.runtime.ANTLRInputStream;
import org.antlr.v4.runtime.CommonTokenStream;
import org.antlr.v4.runtime.tree.ParseTreeWalker;
import io.debezium.connector.oracle.parser.PlSqlLexer;
import io.debezium.connector.oracle.parser.PlSqlParser;
import io.debezium.connector.oracle.parser.PlSqlParser.Column_definitionContext;
import io.debezium.connector.oracle.parser.PlSqlParser.Column_nameContext;
import io.debezium.connector.oracle.parser.PlSqlParser.Create_tableContext;
import io.debezium.connector.oracle.parser.PlSqlParser.Out_of_line_constraintContext;
import io.debezium.connector.oracle.parser.PlSqlParser.Precision_partContext;
import io.debezium.connector.oracle.parser.PlSqlParserBaseListener;
import io.debezium.relational.Column;
import io.debezium.relational.ColumnEditor;
import io.debezium.relational.Table;
import io.debezium.relational.TableEditor;
import io.debezium.relational.TableId;
public class OracleDdlParser {
public Table parseCreateTable(TableId tableId, String createTableDdl) {
if (!createTableDdl.endsWith(";")) {
createTableDdl = createTableDdl + ";";
}
PlSqlLexer lexer = new PlSqlLexer(new ANTLRInputStream(toUpperCase(createTableDdl)));
CommonTokenStream tokens = new CommonTokenStream(lexer);
PlSqlParser parser = new PlSqlParser(tokens);
Create_tableContext ast = parser.create_table();
CreateTableListener listener = new CreateTableListener(tableId);
ParseTreeWalker.DEFAULT.walk(listener, ast);
return listener.getTable();
}
// TODO excluded quoted identifiers
private String toUpperCase(String ddl) {
return ddl.toUpperCase(Locale.ENGLISH);
}
private static class CreateTableListener extends PlSqlParserBaseListener {
private final TableEditor editor;
public CreateTableListener(TableId tableId) {
editor = Table.editor();
editor.tableId(tableId);
}
public Table getTable() {
return editor.create();
}
@Override
public void exitColumn_definition(Column_definitionContext ctx) {
ColumnEditor columnEditor = Column.editor();
columnEditor.name(getColumnName(ctx.column_name()));
if (ctx.datatype().native_datatype_element().INT() != null || ctx.datatype().native_datatype_element().INTEGER() != null) {
columnEditor.jdbcType(Types.NUMERIC);
columnEditor.type("NUMBER");
columnEditor.length(38);
columnEditor.scale(0);
}
else if (ctx.datatype().native_datatype_element().DATE() != null) {
columnEditor.jdbcType(Types.DATE);
columnEditor.type("DATE");
}
else if (ctx.datatype().native_datatype_element().TIMESTAMP() != null) {
columnEditor.jdbcType(Types.TIMESTAMP);
columnEditor.type("TIMESTAMP");
columnEditor.length(6);
columnEditor.scale(0);
}
else if (ctx.datatype().native_datatype_element().VARCHAR2() != null) {
columnEditor.jdbcType(Types.VARCHAR);
columnEditor.type("VARCHAR2");
}
else if (ctx.datatype().native_datatype_element().DECIMAL() != null) {
columnEditor.jdbcType(Types.DECIMAL);
columnEditor.type("DECIMAL");
}
else {
throw new IllegalArgumentException("Unsupported column type: " + ctx.datatype().native_datatype_element().getText());
}
Precision_partContext precisionPart = ctx.datatype().precision_part();
if (precisionPart != null) {
columnEditor.length(Integer.valueOf(precisionPart.numeric(0).getText()));
if (precisionPart.numeric().size() > 1) {
columnEditor.scale(Integer.valueOf(precisionPart.numeric(1).getText()));
}
}
boolean hasNotNullConstraint = ctx.inline_constraint().stream()
.filter(c -> c.NOT() != null)
.findFirst()
.isPresent();
columnEditor.optional(!hasNotNullConstraint);
editor.addColumn(columnEditor.create());
super.exitColumn_definition(ctx);
}
@Override
public void exitOut_of_line_constraint(Out_of_line_constraintContext ctx) {
if(ctx.PRIMARY() != null) {
List<String> pkColumnNames = ctx.column_name().stream()
.map(this::getColumnName)
.collect(Collectors.toList());
editor.setPrimaryKeyNames(pkColumnNames);
}
super.exitOut_of_line_constraint(ctx);
}
private String getColumnName(Column_nameContext ctx) {
return ctx.identifier().id_expression().getText();
}
}
}

View File

@ -0,0 +1,67 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.time.Instant;
import java.util.Collections;
import java.util.Map;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Struct;
import io.debezium.pipeline.spi.OffsetContext;
public class OracleOffsetContext implements OffsetContext {
private static final String SERVER_PARTITION_KEY = "server";
private final Schema sourceInfoSchema;
private final Map<String, String> partition;
private final SourceInfo sourceInfo;
public OracleOffsetContext(String serverName) {
partition = Collections.singletonMap(SERVER_PARTITION_KEY, serverName);
sourceInfo = new SourceInfo(serverName);
sourceInfoSchema = sourceInfo.schema();
}
@Override
public Map<String, ?> getPartition() {
return partition;
}
@Override
public Map<String, ?> getOffset() {
return Collections.singletonMap(SourceInfo.POSITION_KEY, sourceInfo.getPosition());
}
@Override
public Schema getSourceInfoSchema() {
return sourceInfoSchema;
}
@Override
public Struct getSourceInfo() {
return sourceInfo.struct();
}
public void setPosition(byte[] position) {
sourceInfo.setPosition(position);
}
public byte[] getPosition() {
return sourceInfo.getPosition();
}
public void setTransactionId(String transactionId) {
sourceInfo.setTransactionId(transactionId);
}
public void setSourceTime(Instant instant) {
sourceInfo.setSourceTime(instant);
}
}

View File

@ -0,0 +1,140 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.math.BigDecimal;
import java.sql.Clob;
import java.sql.Connection;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Set;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.pipeline.source.spi.SnapshotChangeEventSource;
import io.debezium.pipeline.spi.SnapshotResult;
import io.debezium.pipeline.spi.SnapshotResult.SnapshotResultStatus;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.relational.Tables;
import io.debezium.schema.DatabaseSchema;
import io.debezium.schema.SchemaChangeEvent;
import io.debezium.schema.SchemaChangeEvent.SchemaChangeEventType;
import oracle.sql.NUMBER;
import oracle.streams.StreamsException;
import oracle.streams.XStreamUtility;
public class OracleSnapshotChangeEventSource implements SnapshotChangeEventSource {
private static final Logger LOGGER = LoggerFactory.getLogger(OracleSnapshotChangeEventSource.class);
private final OracleConnectorConfig connectorConfig;
private final OracleConnection jdbcConnection;
private final DatabaseSchema schema;
public OracleSnapshotChangeEventSource(OracleConnectorConfig connectorConfig, OracleConnection jdbcConnection, DatabaseSchema schema) {
this.connectorConfig = connectorConfig;
this.jdbcConnection = jdbcConnection;
this.schema = schema;
}
@Override
public SnapshotResult execute(SnapshotContext context) {
Connection connection = null;
try {
connection = jdbcConnection.connection();
connection.setAutoCommit(false);
Statement statement = connection.createStatement();
if (connectorConfig.getPdbName() != null) {
jdbcConnection.setSessionToPdb(connectorConfig.getPdbName());
}
String catalogName = connectorConfig.getPdbName() != null ? connectorConfig.getPdbName() : connectorConfig.getDatabaseName();
Set<TableId> tableNames = jdbcConnection.readTableNames(catalogName, "%DEBEZIUM%", null, new String[] {"TABLE"} );
for (TableId tableId : tableNames) {
if (context.isAborted()) {
return new SnapshotResult(SnapshotResultStatus.ABORTED, null);
}
LOGGER.debug("Locking table {}", tableId);
statement.execute("LOCK TABLE " + tableId.schema() + "." + tableId.table() + " IN EXCLUSIVE MODE");
}
ResultSet rs = statement.executeQuery("select DBMS_FLASHBACK.GET_SYSTEM_CHANGE_NUMBER from dual");
if (!rs.next()) {
throw new IllegalStateException("Couldn't get SCN");
}
BigDecimal scn = rs.getBigDecimal(1);
rs.close();
Tables tables = new Tables();
jdbcConnection.readSchema(tables, catalogName, "%DEBEZIUM%", null, null, false);
for (TableId tableId : tableNames) {
if (context.isAborted()) {
return new SnapshotResult(SnapshotResultStatus.ABORTED, null);
}
LOGGER.debug("Capturing structure of table {}", tableId);
Table table = tables.forTable(tableId);
rs = statement.executeQuery("select dbms_metadata.get_ddl( 'TABLE', '" + tableId.table() + "', '" + tableId.schema() + "' ) from dual");
if (!rs.next()) {
throw new IllegalStateException("Couldn't get metadata");
}
Object res = rs.getObject(1);
String ddl = ((Clob)res).getSubString(1, (int) ((Clob)res).length());
rs.close();
schema.applySchemaChange(new SchemaChangeEvent(ddl, table, SchemaChangeEventType.CREATE));
}
OracleOffsetContext offset = new OracleOffsetContext(connectorConfig.getLogicalName());
offset.setPosition(convertScnToPosition(scn));
return new SnapshotResult(SnapshotResultStatus.COMPLETED, offset);
}
catch(SQLException e) {
throw new RuntimeException(e);
}
finally {
rollbackTransaction(connection);
if (connectorConfig.getPdbName() != null) {
jdbcConnection.resetSessionToCdb();
}
}
}
private byte[] convertScnToPosition(BigDecimal scn) {
try {
return XStreamUtility.convertSCNToPosition(new NUMBER(scn), XStreamUtility.POS_VERSION_V2);
}
catch (StreamsException | SQLException e) {
throw new RuntimeException(e);
}
}
private void rollbackTransaction(Connection connection) {
if(connection != null) {
try {
connection.rollback();
}
catch (SQLException e) {
throw new RuntimeException(e);
}
}
}
}

View File

@ -0,0 +1,98 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.sql.Connection;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.util.Clock;
import io.debezium.util.Threads;
import oracle.jdbc.OracleConnection;
import oracle.streams.StreamsException;
import oracle.streams.XStreamOut;
public class OracleStreamingChangeEventSource implements StreamingChangeEventSource {
private static final Logger LOGGER = LoggerFactory.getLogger(OracleStreamingChangeEventSource.class);
private final JdbcConnection jdbcConnection;
private final EventDispatcher dispatcher;
private final ErrorHandler errorHandler;
private final Clock clock;
private final OracleDatabaseSchema schema;
private final ExecutorService executor;
private final OracleOffsetContext offsetContext;
private final String xStreamServerName;
public OracleStreamingChangeEventSource(OracleConnectorConfig connectorConfig, OracleOffsetContext offsetContext, JdbcConnection jdbcConnection, EventDispatcher dispatcher, ErrorHandler errorHandler, Clock clock, OracleDatabaseSchema schema) {
this.jdbcConnection = jdbcConnection;
this.dispatcher = dispatcher;
this.errorHandler = errorHandler;
this.clock = clock;
this.schema = schema;
this.offsetContext = offsetContext;
this.xStreamServerName = connectorConfig.getXoutServerName();
this.executor = Threads.newSingleThreadExecutor(OracleConnector.class, connectorConfig.getLogicalName(), "xstream-handler");
}
@Override
public void start() {
Connection connection;
try {
connection = jdbcConnection.connection();
}
catch(Exception e) {
throw new RuntimeException(e);
}
LcrEventHandler hdlr = new LcrEventHandler(errorHandler, dispatcher, clock, schema, offsetContext);
executor.execute(() -> {
XStreamOut xsOut = null;
try {
xsOut = XStreamOut.attach((OracleConnection) connection, xStreamServerName, offsetContext.getPosition(), 1, 1, XStreamOut.DEFAULT_MODE);
while(isRunning()) {
LOGGER.trace("Receiving LCR");
xsOut.receiveLCRCallback(hdlr, XStreamOut.DEFAULT_MODE);
}
}
catch (Exception e) {
errorHandler.setProducerThrowable(e);
}
finally {
if (xsOut != null) {
try {
xsOut.detach(XStreamOut.DEFAULT_MODE);
}
catch (StreamsException e) {
LOGGER.error("Couldn't detach from XStream outbound server " + xStreamServerName, e);
}
}
}
});
}
private boolean isRunning() {
return !Thread.currentThread().isInterrupted();
}
@Override
public void stop() throws InterruptedException {
executor.shutdownNow();
executor.awaitTermination(60, TimeUnit.SECONDS);
};
}

View File

@ -0,0 +1,15 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import io.debezium.connector.common.CdcSourceTaskContext;
public class OracleTaskContext extends CdcSourceTaskContext {
public OracleTaskContext(OracleConnectorConfig config) {
super("Oracle", config.getLogicalName());
}
}

View File

@ -0,0 +1,29 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import io.debezium.relational.TableId;
import io.debezium.schema.DataCollectionId;
import io.debezium.schema.TopicSelector;
public class OracleTopicSelector implements TopicSelector {
private final String prefix;
public OracleTopicSelector(String prefix) {
this.prefix = prefix;
}
public static OracleTopicSelector defaultSelector(String prefix) {
return new OracleTopicSelector(prefix);
}
@Override
public String topicNameFor(DataCollectionId id) {
TableId tableId = (TableId) id;
return String.join(".", prefix, tableId.catalog(), tableId.schema(), tableId.table());
}
}

View File

@ -0,0 +1,86 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.sql.SQLException;
import java.sql.Types;
import org.apache.kafka.connect.data.Field;
import io.debezium.jdbc.JdbcValueConverters;
import io.debezium.relational.Column;
import io.debezium.relational.ValueConverter;
import oracle.sql.NUMBER;
import oracle.sql.TIMESTAMP;
public class OracleValueConverters extends JdbcValueConverters {
@Override
public ValueConverter converter(Column column, Field fieldDefn) {
switch(column.jdbcType()) {
case Types.NUMERIC:
return data -> convertNumeric(column, fieldDefn, data);
}
return super.converter(column, fieldDefn);
}
@Override
protected Object convertInteger(Column column, Field fieldDefn, Object data) {
if (data instanceof NUMBER) {
try {
data = ((NUMBER)data).intValue();
}
catch (SQLException e) {
throw new RuntimeException("Couldn't convert value for column " + column.name(), e);
}
}
return super.convertInteger(column, fieldDefn, data);
}
@Override
protected Object convertDecimal(Column column, Field fieldDefn, Object data) {
if (data instanceof NUMBER) {
try {
data = ((NUMBER)data).bigDecimalValue();
}
catch (SQLException e) {
throw new RuntimeException("Couldn't convert value for column " + column.name(), e);
}
}
return super.convertDecimal(column, fieldDefn, data);
}
@Override
protected Object convertNumeric(Column column, Field fieldDefn, Object data) {
if (data instanceof NUMBER) {
try {
data = ((NUMBER)data).bigDecimalValue();
}
catch (SQLException e) {
throw new RuntimeException("Couldn't convert value for column " + column.name(), e);
}
}
return super.convertNumeric(column, fieldDefn, data);
}
@Override
protected Object convertTimestampToEpochMicros(Column column, Field fieldDefn, Object data) {
if (data instanceof TIMESTAMP) {
try {
data = ((TIMESTAMP)data).timestampValue();
}
catch (SQLException e) {
throw new RuntimeException("Couldn't convert value for column " + column.name(), e);
}
}
return super.convertTimestampToEpochMicros(column, fieldDefn, data);
}
}

View File

@ -0,0 +1,84 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.time.Instant;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Struct;
import io.debezium.connector.AbstractSourceInfo;
public class SourceInfo extends AbstractSourceInfo {
public static final String SERVER_NAME_KEY = "name";
public static final String TXID_KEY = "txId";
public static final String TIMESTAMP_KEY = "ts_sec";
public static final String POSITION_KEY = "position";
public static final String SNAPSHOT_KEY = "snapshot";
public static final Schema SCHEMA = schemaBuilder()
.name("io.debezium.connector.oracle.Source")
.field(SERVER_NAME_KEY, Schema.STRING_SCHEMA)
.field(TIMESTAMP_KEY, Schema.OPTIONAL_INT64_SCHEMA)
.field(TXID_KEY, Schema.OPTIONAL_STRING_SCHEMA)
.field(POSITION_KEY, Schema.OPTIONAL_BYTES_SCHEMA)
.field(SNAPSHOT_KEY, Schema.OPTIONAL_BOOLEAN_SCHEMA)
.build();
private final String serverName;
private byte[] position;
private String transactionId;
private Instant sourceTime;
protected SourceInfo(String serverName) {
super(Module.version());
this.serverName = serverName;
}
@Override
protected Schema schema() {
return SCHEMA;
}
@Override
public Struct struct() {
return super.struct()
.put(SERVER_NAME_KEY, serverName)
.put(TIMESTAMP_KEY, sourceTime.toEpochMilli())
.put(TXID_KEY, transactionId)
.put(POSITION_KEY, position)
.put(SNAPSHOT_KEY, false);
}
public String getServerName() {
return serverName;
}
public byte[] getPosition() {
return position;
}
public void setPosition(byte[] position) {
this.position = position;
}
public String getTransactionId() {
return transactionId;
}
public void setTransactionId(String transactionId) {
this.transactionId = transactionId;
}
public Instant getSourceTime() {
return sourceTime;
}
public void setSourceTime(Instant sourceTime) {
this.sourceTime = sourceTime;
}
}

View File

@ -0,0 +1 @@
version=${project.version}

View File

@ -0,0 +1,132 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import static org.fest.assertions.Assertions.assertThat;
import java.math.BigDecimal;
import java.sql.SQLException;
import java.time.LocalDateTime;
import java.time.ZoneOffset;
import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import io.debezium.config.Configuration;
import io.debezium.connector.oracle.util.TestHelper;
import io.debezium.data.VerifyRecord;
import io.debezium.embedded.AbstractConnectorTest;
/**
* Integration test for the Debezium Oracle connector.
*
* @author Gunnar Morling
*/
public class OracleConnectorIT extends AbstractConnectorTest {
private static final long MICROS_PER_SECOND = TimeUnit.SECONDS.toMicros(1);
private static OracleConnection connection;
@BeforeClass
public static void beforeClass() throws SQLException {
connection = TestHelper.testConnection();
// TestHelper.dropAllSchemas();
}
@AfterClass
public static void closeConnection() throws SQLException {
if (connection != null) {
connection.close();
}
}
@Before
public void before() {
initializeConnectorTestFramework();
}
@Test
public void shouldReadChangeStreamForExistingTable() throws Exception {
TestHelper.dropTable(connection, "debezium.customer");
String ddl = "create table debezium.customer (" +
" id int not null, " +
" name varchar2(1000), " +
" score decimal(6, 2), " +
" registered timestamp, " +
" primary key (id)" +
")";
connection.execute(ddl);
connection.execute("GRANT SELECT ON debezium.customer to c##xstrmadmin");
Configuration config = TestHelper.defaultConfig().build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
Thread.sleep(1000);
connection.execute("INSERT INTO debezium.customer VALUES (1, 'Billie-Bob', 1234.56, TO_DATE('2018/02/22', 'yyyy-mm-dd'))");
connection.execute("COMMIT");
SourceRecords records = consumeRecordsByTopic(1);
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.ORCLPDB1.DEBEZIUM.CUSTOMER");
assertThat(testTableRecords).hasSize(1);
VerifyRecord.isValidInsert(testTableRecords.get(0));
Struct after = (Struct) ((Struct)testTableRecords.get(0).value()).get("after");
assertThat(after.get("ID")).isEqualTo(BigDecimal.valueOf(1));
assertThat(after.get("NAME")).isEqualTo("Billie-Bob");
assertThat(after.get("SCORE")).isEqualTo(BigDecimal.valueOf(1234.56));
assertThat(after.get("REGISTERED")).isEqualTo(LocalDateTime.of(2018, 2, 22, 0, 0, 0).toEpochSecond(ZoneOffset.UTC) * MICROS_PER_SECOND);
}
@Test
public void shouldReadChangeStreamForTableCreatedWhileStreaming() throws Exception {
TestHelper.dropTable(connection, "debezium.customer");
Configuration config = TestHelper.defaultConfig().build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
Thread.sleep(4000);
String ddl = "create table debezium.customer (" +
" id int not null, " +
" name varchar2(1000), " +
" score decimal(6, 2), " +
" registered timestamp, " +
" primary key (id)" +
")";
connection.execute(ddl);
connection.execute("GRANT SELECT ON debezium.customer to c##xstrmadmin");
connection.execute("INSERT INTO debezium.customer VALUES (2, 'Billie-Bob', 1234.56, TO_DATE('2018/02/22', 'yyyy-mm-dd'))");
connection.execute("COMMIT");
SourceRecords records = consumeRecordsByTopic(1);
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.ORCLPDB1.DEBEZIUM.CUSTOMER");
assertThat(testTableRecords).hasSize(1);
VerifyRecord.isValidInsert(testTableRecords.get(0));
Struct after = (Struct) ((Struct)testTableRecords.get(0).value()).get("after");
assertThat(after.get("ID")).isEqualTo(BigDecimal.valueOf(2));
assertThat(after.get("NAME")).isEqualTo("Billie-Bob");
assertThat(after.get("SCORE")).isEqualTo(BigDecimal.valueOf(1234.56));
assertThat(after.get("REGISTERED")).isEqualTo(LocalDateTime.of(2018, 2, 22, 0, 0, 0).toEpochSecond(ZoneOffset.UTC) * MICROS_PER_SECOND);
}
}

View File

@ -0,0 +1,79 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import static org.fest.assertions.Assertions.assertThat;
import java.sql.Types;
import org.antlr.v4.runtime.ANTLRInputStream;
import org.antlr.v4.runtime.CommonTokenStream;
import org.antlr.v4.runtime.RuleContext;
import org.antlr.v4.runtime.tree.ParseTree;
import org.junit.Test;
import io.debezium.connector.oracle.parser.PlSqlLexer;
import io.debezium.connector.oracle.parser.PlSqlParser;
import io.debezium.connector.oracle.parser.PlSqlParser.Unit_statementContext;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
public class OracleDdlParserTest {
@Test
public void shouldParseCreateTable() {
String ddl = "create table debezium.customer (" +
" id int not null, " +
" name varchar2(1000), " +
" score decimal(6, 2), " +
" registered date, " +
" primary key (id)" +
");";
OracleDdlParser parser = new OracleDdlParser();
Table table = parser.parseCreateTable(new TableId("ORCLPDB1", "DEBEZIUM", "CUSTOMER"), ddl);
Column id = table.columnWithName("ID");
assertThat(id.isOptional()).isFalse();
assertThat(id.jdbcType()).isEqualTo(Types.NUMERIC);
assertThat(id.typeName()).isEqualTo("NUMBER");
assertThat(table.columns()).hasSize(4);
assertThat(table.isPrimaryKeyColumn("ID"));
}
private void printAst(String ddl) {
PlSqlLexer lexer = new PlSqlLexer(new ANTLRInputStream(ddl));
CommonTokenStream tokens = new CommonTokenStream(lexer);
PlSqlParser parser = new PlSqlParser(tokens);
Unit_statementContext ast = parser.unit_statement();
new AstPrinter().print(ast);
}
private static class AstPrinter {
public void print(RuleContext ctx) {
explore(ctx, 0);
}
private void explore(RuleContext ctx, int indentation) {
String ruleName = PlSqlParser.ruleNames[ctx.getRuleIndex()];
for (int i=0;i<indentation;i++) {
System.out.print(" ");
}
System.out.println(ruleName);
for (int i=0;i<ctx.getChildCount();i++) {
ParseTree element = ctx.getChild(i);
if (element instanceof RuleContext) {
explore((RuleContext)element, indentation + 1);
}
}
}
}
}

View File

@ -0,0 +1,112 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.util;
import java.sql.SQLException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.config.Configuration;
import io.debezium.connector.oracle.OracleConnection;
import io.debezium.connector.oracle.OracleConnectionFactory;
import io.debezium.connector.oracle.OracleConnectorConfig;
import io.debezium.jdbc.JdbcConfiguration;
public class TestHelper {
private static final Logger LOGGER = LoggerFactory.getLogger(TestHelper.class);
private static JdbcConfiguration defaultJdbcConfig() {
return JdbcConfiguration.copy(Configuration.fromSystemProperties("database."))
.withDefault(JdbcConfiguration.HOSTNAME, "localhost")
.withDefault(JdbcConfiguration.PORT, 1521)
.withDefault(JdbcConfiguration.USER, "c##xstrmadmin")
.withDefault(JdbcConfiguration.PASSWORD, "xsa")
.withDefault(JdbcConfiguration.DATABASE, "ORCLCDB")
.build();
}
public static Configuration.Builder defaultConfig() {
JdbcConfiguration jdbcConfiguration = defaultJdbcConfig();
Configuration.Builder builder = Configuration.create();
jdbcConfiguration.forEach(
(field, value) -> builder.with(OracleConnectorConfig.DATABASE_CONFIG_PREFIX + field, value)
);
builder.with(OracleConnectorConfig.LOGICAL_NAME, "server1")
.with(OracleConnectorConfig.PDB_NAME, "ORCLPDB1")
.with(OracleConnectorConfig.XSTREAM_SERVER_NAME, "XOUT2");
return builder;
}
public static OracleConnection defaultConnection() {
Configuration config = defaultConfig().build();
Configuration jdbcConfig = config.subset("database.", true);
OracleConnection jdbcConnection = new OracleConnection(jdbcConfig, new OracleConnectionFactory());
String pdbName = new OracleConnectorConfig(config).getPdbName();
if (pdbName != null) {
jdbcConnection.setSessionToPdb(pdbName);
}
return jdbcConnection;
}
/**
* Returns a JDBC configuration for the test data schema and user (NOT the XStream user).
*/
private static JdbcConfiguration testJdbcConfig() {
return JdbcConfiguration.copy(Configuration.fromSystemProperties("database."))
.withDefault(JdbcConfiguration.HOSTNAME, "localhost")
.withDefault(JdbcConfiguration.PORT, 1521)
.withDefault(JdbcConfiguration.USER, "debezium")
.withDefault(JdbcConfiguration.PASSWORD, "dbz")
.withDefault(JdbcConfiguration.DATABASE, "ORCLPDB1")
.build();
}
private static Configuration.Builder testConfig() {
JdbcConfiguration jdbcConfiguration = testJdbcConfig();
Configuration.Builder builder = Configuration.create();
jdbcConfiguration.forEach(
(field, value) -> builder.with(OracleConnectorConfig.DATABASE_CONFIG_PREFIX + field, value)
);
return builder;
}
public static OracleConnection testConnection() {
Configuration config = testConfig().build();
Configuration jdbcConfig = config.subset("database.", true);
OracleConnection jdbcConnection = new OracleConnection(jdbcConfig, new OracleConnectionFactory());
String pdbName = new OracleConnectorConfig(config).getPdbName();
if (pdbName != null) {
jdbcConnection.setSessionToPdb(pdbName);
}
return jdbcConnection;
}
public static void dropTable(OracleConnection connection, String table) {
try {
connection.execute("drop table " + table);
}
catch (SQLException e) {
if (!e.getMessage().contains("table or view does not exist")) {
throw new RuntimeException(e);
}
}
}
}

View File

@ -0,0 +1,15 @@
# Direct log messages to stdout
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.Target=System.out
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %X{dbz.connectorType}|%X{dbz.connectorName}|%X{dbz.connectorContext} %m [%c]%n
# Root logger option
log4j.rootLogger=INFO, stdout
# Set up the default logging to be INFO level, then override specific units
log4j.logger.io.debezium=INFO
log4j.logger.io.debezium.embedded.EmbeddedEngine$EmbeddedConfig=WARN
#log4j.logger.io.debezium.embedded.EmbeddedEngine=DEBUG
log4j.logger.io.debezium.core=DEBUG
log4j.logger.io.debezium.connector.oracle=DEBUG

View File

@ -720,7 +720,7 @@ public String getPostgresPluginName() {
private final SnapshotMode snapshotMode;
protected PostgresConnectorConfig(Configuration config) {
super(config);
super(config, SERVER_NAME);
this.config = config;
String serverName = config.getString(PostgresConnectorConfig.SERVER_NAME);

View File

@ -66,12 +66,14 @@ public class CommonConnectorConfig {
private final int maxQueueSize;
private final int maxBatchSize;
private final Duration pollInterval;
private final String logicalName;
protected CommonConnectorConfig(Configuration config) {
protected CommonConnectorConfig(Configuration config, Field logicalNameField) {
this.emitTombstoneOnDelete = config.getBoolean(CommonConnectorConfig.TOMBSTONES_ON_DELETE);
this.maxQueueSize = config.getInteger(MAX_QUEUE_SIZE);
this.maxBatchSize = config.getInteger(MAX_BATCH_SIZE);
this.pollInterval = config.getDuration(POLL_INTERVAL_MS, ChronoUnit.MILLIS);
this.logicalName = config.getString(logicalNameField);
}
public boolean isEmitTombstoneOnDelete() {
@ -90,6 +92,10 @@ public Duration getPollInterval() {
return pollInterval;
}
public String getLogicalName() {
return logicalName;
}
private static int validateMaxQueueSize(Configuration config, Field field, Field.ValidationOutput problems) {
int maxQueueSize = config.getInteger(field);
int maxBatchSize = config.getInteger(MAX_BATCH_SIZE);

View File

@ -0,0 +1,84 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.connect.source.SourceConnector;
import io.debezium.pipeline.source.spi.ChangeEventSource;
import io.debezium.pipeline.source.spi.ChangeEventSourceFactory;
import io.debezium.pipeline.source.spi.SnapshotChangeEventSource;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.pipeline.spi.SnapshotResult.SnapshotResultStatus;
import io.debezium.util.Threads;
/**
* Coordinates one or more {@link ChangeEventSource}s and executes them in order.
*
* @author Gunnar Morling
*/
public class ChangeEventSourceCoordinator {
private final ErrorHandler errorHandler;
private final ChangeEventSourceFactory changeEventSourceFactory;
private final ExecutorService executor;
private final CountDownLatch stopped;
public ChangeEventSourceCoordinator(ErrorHandler errorHandler, Class<? extends SourceConnector> connectorType, String logicalName, ChangeEventSourceFactory changeEventSourceFactory) {
this.errorHandler = errorHandler;
this.changeEventSourceFactory = changeEventSourceFactory;
this.executor = Threads.newSingleThreadExecutor(connectorType, logicalName, "change-event-source-coordinator");
this.stopped = new CountDownLatch(1);
}
public void start() {
SnapshotChangeEventSource snapshotSource = changeEventSourceFactory.getSnapshotChangeEventSource();
CompletableFuture.supplyAsync(() -> snapshotSource.execute(new SnapshotContext()), executor)
.whenComplete((snapshotResult, throwable) -> {
// TODO
if (throwable != null) {
errorHandler.setProducerThrowable(throwable);
}
else if (snapshotResult.getStatus() == SnapshotResultStatus.COMPLETED) {
StreamingChangeEventSource streamingSource = changeEventSourceFactory.getStreamingChangeEventSource(snapshotResult.getOffset());
streamingSource.start();
try {
stopped.await();
streamingSource.stop();
}
catch (InterruptedException e) {
Thread.interrupted();
}
}
})
.exceptionally(throwable -> {
errorHandler.setProducerThrowable(throwable);
return null;
});
}
public void stop() throws InterruptedException {
stopped.countDown();
executor.shutdownNow();
executor.awaitTermination(60, TimeUnit.SECONDS);
}
private final class SnapshotContext implements SnapshotChangeEventSource.SnapshotContext {
@Override
public boolean isAborted() {
return stopped.getCount() == 0;
}
}
}

View File

@ -0,0 +1,26 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline;
import org.apache.kafka.connect.source.SourceRecord;
public class DataChangeEvent {
private final SourceRecord record;
public DataChangeEvent(SourceRecord record) {
this.record = record;
}
public SourceRecord getRecord() {
return record;
}
@Override
public String toString() {
return "DataChangeEvent [record=" + record + "]";
}
}

View File

@ -0,0 +1,54 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.connect.source.SourceConnector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.util.Threads;
public class ErrorHandler {
private static final Logger LOGGER = LoggerFactory.getLogger(ErrorHandler.class);
private final ChangeEventQueue<?> queue;
private final Runnable onThrowable;
private final AtomicReference<Throwable> producerThrowable;
private final ExecutorService executor;
public ErrorHandler(Class<? extends SourceConnector> connectorType, String logicalName, ChangeEventQueue<?> queue, Runnable onThrowable) {
this.queue = queue;
this.onThrowable = onThrowable;
this.executor = Threads.newSingleThreadExecutor(connectorType, logicalName, "error-handler");
this.producerThrowable = new AtomicReference<>();
}
public void setProducerThrowable(Throwable producerThrowable) {
LOGGER.error("Producer failure", producerThrowable);
boolean first = this.producerThrowable.compareAndSet(null, producerThrowable);
if (first) {
queue.producerFailure(producerThrowable);
executor.execute(() -> onThrowable.run());
}
}
public Throwable getProducerThrowable() {
return producerThrowable.get();
}
public void stop() throws InterruptedException {
executor.shutdownNow();
executor.awaitTermination(60, TimeUnit.SECONDS);
}
}

View File

@ -0,0 +1,150 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline;
import java.util.function.Supplier;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.data.Envelope.Operation;
import io.debezium.pipeline.spi.ChangeEventCreator;
import io.debezium.pipeline.spi.ChangeRecordEmitter;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.spi.SchemaChangeEventEmitter;
import io.debezium.relational.TableId;
import io.debezium.schema.DataCollectionId;
import io.debezium.schema.DataCollectionSchema;
import io.debezium.schema.DatabaseSchema;
import io.debezium.schema.SchemaChangeEvent;
import io.debezium.schema.TopicSelector;
/**
* Central dispatcher for data change and schema change events. The former will be routed to the change event queue, the
* latter will be routed to the {@link DatabaseSchema}. But based on the applying whitelist/blacklist configuration,
* events may be not be dispatched at all.
* <p>
* This router is also in charge of emitting heartbeat messages, exposing of metrics via JMX etc.
*
* @author Gunnar Morling
*/
public class EventDispatcher {
private final ErrorHandler errorHandler;
private final TopicSelector topicSelector;
private final DatabaseSchema schema;
private final ChangeEventQueue<Object> queue;
public EventDispatcher(ErrorHandler errorHandler, TopicSelector topicSelector, DatabaseSchema schema,
ChangeEventQueue<Object> queue) {
this.errorHandler = errorHandler;
this.topicSelector = topicSelector;
this.schema = schema;
this.queue = queue;
}
/**
* Dispatches one or more {@link DataChangeEvent}s. If the given data collection is included in the currently
* captured set of collections, the given emitter will be invoked, so it can emit one or more events (in the common
* case, one event will be emitted, but e.g. in case of PK updates, it may be a deletion and a creation event). The
* receiving coordinator creates {@link SourceRecord}s for all emitted events and passes them to the given
* {@link ChangeEventCreator} for converting them into data change events.
*/
public void dispatchDataChangeEvent(OffsetContext offsetContext, DataCollectionId dataCollectionId, Supplier<ChangeRecordEmitter> changeRecordEmitter, ChangeEventCreator changeEventCreator) throws InterruptedException {
// TODO Handle Heartbeat
// TODO Handle JMX
// TODO handle filtering based on table id
boolean tableIncluded = true;
if(!tableIncluded) {
return;
}
DataCollectionSchema dataCollectionSchema = schema.getDataCollectionSchema(dataCollectionId);
// TODO handle as per inconsistent schema info option
if(dataCollectionSchema == null) {
errorHandler.setProducerThrowable(new IllegalArgumentException("No metadata registered for captured table " + dataCollectionId));
return;
}
changeRecordEmitter.get().emitChangeRecords(
offsetContext,
dataCollectionSchema,
new ChangeRecordReceiver(dataCollectionId, changeEventCreator, dataCollectionSchema)
);
}
public void dispatchSchemaChangeEvent(TableId tableId, SchemaChangeEventEmitter schemaChangeEventEmitter) throws InterruptedException {
// TODO
boolean tableIncluded = true;
if(tableIncluded) {
schemaChangeEventEmitter.emitSchemaChangeEvent(tableId, new SchemaChangeEventReceiver());
}
}
private final class ChangeRecordReceiver implements ChangeRecordEmitter.Receiver {
private final DataCollectionId dataCollectionId;
private final ChangeEventCreator changeEventCreator;
private final DataCollectionSchema dataCollectionSchema;
private ChangeRecordReceiver(DataCollectionId dataCollectionId, ChangeEventCreator changeEventCreator,
DataCollectionSchema dataCollectionSchema) {
this.dataCollectionId = dataCollectionId;
this.changeEventCreator = changeEventCreator;
this.dataCollectionSchema = dataCollectionSchema;
}
@Override
public void changeRecord(Operation operation, Object key,
Struct value, OffsetContext offsetContext) throws InterruptedException {
if (key == null || value == null) {
// TODO raise exception?
return;
}
Schema keySchema = dataCollectionSchema.keySchema();
String topicName = topicSelector.topicNameFor(dataCollectionId);
SourceRecord record = new SourceRecord(offsetContext.getPartition(), offsetContext.getOffset(),
topicName, null, keySchema, key, dataCollectionSchema.getEnvelopeSchema().schema(), value);
queue.enqueue(changeEventCreator.createDataChangeEvent(record));
// TODO handle option
boolean emitTombstonesOnDelete = true;
if (emitTombstonesOnDelete && operation == Operation.DELETE) {
SourceRecord tombStone = record.newRecord(
record.topic(),
record.kafkaPartition(),
record.keySchema(),
record.key(),
dataCollectionSchema.getEnvelopeSchema().schema(),
null, // value
record.timestamp()
);
queue.enqueue(changeEventCreator.createDataChangeEvent(tombStone));
}
}
}
private final class SchemaChangeEventReceiver implements SchemaChangeEventEmitter.Receiver {
@Override
public void schemaChangeEvent(SchemaChangeEvent event) throws InterruptedException {
schema.applySchemaChange(event);
}
}
}

View File

@ -0,0 +1,10 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline.source.spi;
public interface ChangeEventSource {
}

View File

@ -0,0 +1,15 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline.source.spi;
import io.debezium.pipeline.spi.OffsetContext;
public interface ChangeEventSourceFactory {
SnapshotChangeEventSource getSnapshotChangeEventSource();
StreamingChangeEventSource getStreamingChangeEventSource(OffsetContext offsetContext);
}

View File

@ -0,0 +1,19 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline.source.spi;
import io.debezium.pipeline.spi.SnapshotResult;
public interface SnapshotChangeEventSource extends ChangeEventSource {
SnapshotResult execute(SnapshotContext context);
public interface SnapshotContext {
// TODO can we get rid of this in the favor of Thread.isInterrupted()?
boolean isAborted();
}
}

View File

@ -0,0 +1,13 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline.source.spi;
public interface StreamingChangeEventSource extends ChangeEventSource {
void start();
void stop() throws InterruptedException;
}

View File

@ -0,0 +1,16 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline.spi;
import org.apache.kafka.connect.source.SourceRecord;
import io.debezium.pipeline.DataChangeEvent;
@FunctionalInterface
public interface ChangeEventCreator {
DataChangeEvent createDataChangeEvent(SourceRecord sourceRecord);
}

View File

@ -0,0 +1,25 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline.spi;
import org.apache.kafka.connect.data.Struct;
import io.debezium.data.Envelope.Operation;
import io.debezium.schema.DataCollectionSchema;
/**
* Emits change one or more change records - specific to a given {@link DataCollectionSchema}.
*
* @author Gunnar Morling
*/
public interface ChangeRecordEmitter {
void emitChangeRecords(OffsetContext offsetContext, DataCollectionSchema schema, Receiver receiver) throws InterruptedException;
public interface Receiver {
void changeRecord(Operation operation, Object key, Struct value, OffsetContext offsetManager) throws InterruptedException;
}
}

View File

@ -0,0 +1,21 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline.spi;
import java.util.Map;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Struct;
public interface OffsetContext {
// load()
Map<String, ?> getPartition();
Map<String, ?> getOffset();
Schema getSourceInfoSchema();
Struct getSourceInfo();
}

View File

@ -0,0 +1,24 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline.spi;
import io.debezium.relational.TableId;
import io.debezium.schema.DataCollectionSchema;
import io.debezium.schema.SchemaChangeEvent;
/**
* Emits change one or more change records - specific to a given {@link DataCollectionSchema}.
*
* @author Gunnar Morling
*/
public interface SchemaChangeEventEmitter {
void emitSchemaChangeEvent(TableId tableId, Receiver receiver) throws InterruptedException;
public interface Receiver {
void schemaChangeEvent(SchemaChangeEvent event) throws InterruptedException;
}
}

View File

@ -0,0 +1,30 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.pipeline.spi;
public class SnapshotResult {
private final SnapshotResultStatus status;
private final OffsetContext offset;
public SnapshotResult(SnapshotResultStatus status, OffsetContext offset) {
this.status = status;
this.offset = offset;
}
public SnapshotResultStatus getStatus() {
return status;
}
public OffsetContext getOffset() {
return offset;
}
public static enum SnapshotResultStatus {
COMPLETED,
ABORTED;
}
}

View File

@ -0,0 +1,13 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.relational;
import io.debezium.schema.DatabaseSchema;
public interface RelationalDatabaseSchema extends DatabaseSchema {
Table getTable(TableId id);
}

View File

@ -6,6 +6,7 @@
package io.debezium.relational;
import io.debezium.annotation.Immutable;
import io.debezium.schema.DataCollectionId;
/**
* Unique identifier for a database table.
@ -13,7 +14,7 @@
* @author Randall Hauch
*/
@Immutable
public final class TableId implements Comparable<TableId> {
public final class TableId implements DataCollectionId, Comparable<TableId> {
/**
* Parse the supplied string, extracting up to the first 3 parts into a TableID.

View File

@ -14,6 +14,7 @@
import io.debezium.annotation.Immutable;
import io.debezium.data.Envelope;
import io.debezium.data.SchemaUtil;
import io.debezium.schema.DataCollectionSchema;
/**
* Defines the Kafka Connect {@link Schema} functionality associated with a given {@link Table table definition}, and which can
@ -46,7 +47,7 @@
* @see TableSchemaBuilder
*/
@Immutable
public class TableSchema {
public class TableSchema implements DataCollectionSchema {
private final Schema keySchema;
private final Envelope envelopeSchema;
@ -89,6 +90,7 @@ public Schema valueSchema() {
*
* @return the Schema describing the column's that make up the primary key; null if there is no primary key
*/
@Override
public Schema keySchema() {
return keySchema;
}
@ -99,6 +101,7 @@ public Schema keySchema() {
*
* @return the table's envelope schema
*/
@Override
public Envelope getEnvelopeSchema() {
return envelopeSchema;
}

View File

@ -0,0 +1,10 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.schema;
public interface DataCollectionId {
}

View File

@ -0,0 +1,16 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.schema;
import org.apache.kafka.connect.data.Schema;
import io.debezium.data.Envelope;
public interface DataCollectionSchema {
Schema keySchema();
Envelope getEnvelopeSchema();
}

View File

@ -0,0 +1,13 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.schema;
public interface DatabaseSchema {
void applySchemaChange(SchemaChangeEvent schemaChange);
DataCollectionSchema getDataCollectionSchema(DataCollectionId id);
}

View File

@ -0,0 +1,46 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.schema;
import java.util.Objects;
import io.debezium.relational.Table;
public class SchemaChangeEvent {
private final String ddl;
private final Table table;
private final SchemaChangeEventType type;
public SchemaChangeEvent(String ddl, Table table, SchemaChangeEventType type) {
this.ddl = Objects.requireNonNull(ddl, "ddl must not be null");
this.table = Objects.requireNonNull(table, "table must not be null");
this.type = Objects.requireNonNull(type, "type must not be null");
}
public String getDdl() {
return ddl;
}
public Table getTable() {
return table;
}
public SchemaChangeEventType getType() {
return type;
}
@Override
public String toString() {
return "SchemaChangeEvent [ddl=" + ddl + ", table=" + table + ", type=" + type + "]";
}
public static enum SchemaChangeEventType {
CREATE,
ALTER,
DROP;
}
}

View File

@ -0,0 +1,11 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.schema;
public interface TopicSelector {
String topicNameFor(DataCollectionId id);
}

View File

@ -69,20 +69,20 @@
* {@link #start(Class, Configuration) starts the connector} using your connector's custom configuration.
* Then, your test methods can call {@link #consumeRecords(int, Consumer)} to consume the specified number
* of records (the supplied function gives you a chance to do something with the record).
*
*
* @author Randall Hauch
*/
public abstract class AbstractConnectorTest implements Testing {
@Rule
public TestRule skipTestRule = new SkipTestRule();
protected static final Path OFFSET_STORE_PATH = Testing.Files.createTestingPath("file-connector-offsets.txt").toAbsolutePath();
private ExecutorService executor;
protected EmbeddedEngine engine;
private BlockingQueue<SourceRecord> consumedLines;
protected long pollTimeoutInMs = TimeUnit.SECONDS.toMillis(5);
protected long pollTimeoutInMs = TimeUnit.SECONDS.toMillis(60);
protected final Logger logger = LoggerFactory.getLogger(getClass());
private CountDownLatch latch;
private JsonConverter keyJsonConverter = new JsonConverter();
@ -120,7 +120,7 @@ public final void stopConnector() {
/**
* Stop the connector, and return whether the connector was successfully stopped.
*
*
* @param callback the function that should be called with whether the connector was successfully stopped; may be null
*/
public void stopConnector(BooleanConsumer callback) {
@ -166,10 +166,10 @@ public void stopConnector(BooleanConsumer callback) {
* Get the maximum number of messages that can be obtained from the connector and held in-memory before they are
* consumed by test methods using {@link #consumeRecord()}, {@link #consumeRecords(int)}, or
* {@link #consumeRecords(int, Consumer)}.
*
*
* <p>
* By default this method return {@code 100}.
*
*
* @return the maximum number of records that can be enqueued
*/
protected int getMaximumEnqueuedRecordCount() {
@ -179,7 +179,7 @@ protected int getMaximumEnqueuedRecordCount() {
/**
* Create a {@link CompletionCallback} that logs when the engine fails to start the connector or when the connector
* stops running after completing successfully or due to an error
*
*
* @return the logging {@link CompletionCallback}
*/
protected CompletionCallback loggingCompletion() {
@ -195,7 +195,7 @@ protected CompletionCallback loggingCompletion() {
/**
* Start the connector using the supplied connector configuration, where upon completion the status of the connector is
* logged.
*
*
* @param connectorClass the connector class; may not be null
* @param connectorConfig the configuration for the connector; may not be null
*/
@ -206,7 +206,7 @@ protected void start(Class<? extends SourceConnector> connectorClass, Configurat
/**
* Start the connector using the supplied connector configuration, where upon completion the status of the connector is
* logged. The connector will stop immediately when the supplied predicate returns true.
*
*
* @param connectorClass the connector class; may not be null
* @param connectorConfig the configuration for the connector; may not be null
* @param isStopRecord the function that will be called to determine if the connector should be stopped before processing
@ -219,7 +219,7 @@ protected void start(Class<? extends SourceConnector> connectorClass, Configurat
/**
* Start the connector using the supplied connector configuration.
*
*
* @param connectorClass the connector class; may not be null
* @param connectorConfig the configuration for the connector; may not be null
* @param callback the function that will be called when the engine fails to start the connector or when the connector
@ -232,7 +232,7 @@ protected void start(Class<? extends SourceConnector> connectorClass, Configurat
/**
* Start the connector using the supplied connector configuration.
*
*
* @param connectorClass the connector class; may not be null
* @param connectorConfig the configuration for the connector; may not be null
* @param isStopRecord the function that will be called to determine if the connector should be stopped before processing
@ -260,15 +260,15 @@ protected void start(Class<? extends SourceConnector> connectorClass, Configurat
}
Testing.debug("Stopped connector");
};
ConnectorCallback connectorCallback = new ConnectorCallback() {
@Override
public void taskStarted() {
// if this is called, it means a task has been started successfully so we can continue
latch.countDown();
}
};
};
// Create the connector ...
engine = EmbeddedEngine.create()
.using(config)
@ -310,7 +310,7 @@ public void taskStarted() {
/**
* Set the maximum amount of time that the {@link #consumeRecord()}, {@link #consumeRecords(int)}, and
* {@link #consumeRecords(int, Consumer)} methods block while waiting for each record before returning <code>null</code>.
*
*
* @param timeout the timeout; must be positive
* @param unit the time unit; may not be null
*/
@ -321,7 +321,7 @@ protected void setConsumeTimeout(long timeout, TimeUnit unit) {
/**
* Consume a single record from the connector.
*
*
* @return the next record that was returned from the connector, or null if no such record has been produced by the connector
* @throws InterruptedException if the thread was interrupted while waiting for a record to be returned
*/
@ -332,7 +332,7 @@ protected SourceRecord consumeRecord() throws InterruptedException {
/**
* Try to consume the specified number of records from the connector, and return the actual number of records that were
* consumed. Use this method when your test does not care what the records might contain.
*
*
* @param numberOfRecords the number of records that should be consumed
* @return the actual number of records that were consumed
* @throws InterruptedException if the thread was interrupted while waiting for a record to be returned
@ -344,7 +344,7 @@ protected int consumeRecords(int numberOfRecords) throws InterruptedException {
/**
* Try to consume the specified number of records from the connector, calling the given function for each, and return the
* actual number of records that were consumed.
*
*
* @param numberOfRecords the number of records that should be consumed
* @param recordConsumer the function that should be called with each consumed record
* @return the actual number of records that were consumed
@ -378,7 +378,7 @@ protected int consumeRecords(int numberOfRecords, Consumer<SourceRecord> recordC
/**
* Try to consume and capture exactly the specified number of records from the connector.
*
*
* @param numRecords the number of records that should be consumed
* @return the collector into which the records were captured; never null
* @throws InterruptedException if the thread was interrupted while waiting for a record to be returned
@ -414,7 +414,7 @@ protected String getAffectedDatabase(SourceRecord record) {
/**
* Get the DDL events for the named database.
*
*
* @param dbName the name of the database; may not be null
* @return the DDL-related events; never null but possibly empty
*/
@ -424,7 +424,7 @@ public List<SourceRecord> ddlRecordsForDatabase(String dbName) {
/**
* Get the names of the databases that were affected by the DDL statements.
*
*
* @return the set of database names; never null but possibly empty
*/
public Set<String> databaseNames() {
@ -433,7 +433,7 @@ public Set<String> databaseNames() {
/**
* Get the records on the given topic.
*
*
* @param topicName the name of the topic.
* @return the records for the topic; possibly null if there were no records produced on the topic
*/
@ -443,7 +443,7 @@ public List<SourceRecord> recordsForTopic(String topicName) {
/**
* Get the set of topics for which records were received.
*
*
* @return the names of the topics; never null
*/
public Set<String> topics() {
@ -474,7 +474,7 @@ public void print() {
/**
* Try to consume all of the messages that have already been returned by the connector.
*
*
* @param recordConsumer the function that should be called with each consumed record
* @return the number of records that were consumed
*/
@ -489,7 +489,7 @@ protected int consumeAvailableRecords(Consumer<SourceRecord> recordConsumer) {
/**
* Wait for a maximum amount of time until the first record is available.
*
*
* @param timeout the maximum amount of time to wait; must not be negative
* @param unit the time unit for {@code timeout}
* @return {@code true} if records are available, or {@code false} if the timeout occurred and no records are available
@ -556,22 +556,22 @@ protected void assertTombstone(SourceRecord record, String pkField, int pk) {
protected void assertTombstone(SourceRecord record) {
VerifyRecord.isValidTombstone(record);
}
protected void assertOffset(SourceRecord record, Map<String,?> expectedOffset) {
Map<String,?> offset = record.sourceOffset();
assertThat(offset).isEqualTo(expectedOffset);
}
protected void assertOffset(SourceRecord record, String offsetField, Object expectedValue) {
Map<String,?> offset = record.sourceOffset();
Object value = offset.get(offsetField);
assertSameValue(value,expectedValue);
}
protected void assertValueField(SourceRecord record, String fieldPath, Object expectedValue) {
VerifyRecord.assertValueField(record, fieldPath, expectedValue);
}
private void assertSameValue(Object actual, Object expected) {
VerifyRecord.assertSameValue(actual, expected);
}
@ -579,7 +579,7 @@ private void assertSameValue(Object actual, Object expected) {
/**
* Assert that the supplied {@link Struct} is {@link Struct#validate() valid} and its {@link Struct#schema() schema}
* matches that of the supplied {@code schema}.
*
*
* @param value the value with a schema; may not be null
*/
protected void assertSchemaMatchesStruct(SchemaAndValue value) {
@ -589,7 +589,7 @@ protected void assertSchemaMatchesStruct(SchemaAndValue value) {
/**
* Assert that the supplied {@link Struct} is {@link Struct#validate() valid} and its {@link Struct#schema() schema}
* matches that of the supplied {@code schema}.
*
*
* @param struct the {@link Struct} to validate; may not be null
* @param schema the expected schema of the {@link Struct}; may not be null
*/
@ -607,7 +607,7 @@ protected void assertEngineIsRunning() {
/**
* Validate that a {@link SourceRecord}'s key and value can each be converted to a byte[] and then back to an equivalent
* {@link SourceRecord}.
*
*
* @param record the record to validate; may not be null
*/
protected void validate(SourceRecord record) {
@ -656,7 +656,7 @@ protected ConfigValue configValue(Config config, String fieldName) {
/**
* Utility to read the last committed offset for the specified partition.
*
*
* @param config the configuration of the engine used to persist the offsets
* @param partition the partition
* @return the map of partitions to offsets; never null but possibly empty
@ -667,7 +667,7 @@ protected <T> Map<String, Object> readLastCommittedOffset(Configuration config,
/**
* Utility to read the last committed offsets for the specified partitions.
*
*
* @param config the configuration of the engine used to persist the offsets
* @param partitions the partitions
* @return the map of partitions to offsets; never null but possibly empty

View File

@ -715,5 +715,11 @@
<modules.argline>--add-modules java.xml.bind</modules.argline>
</properties>
</profile>
<profile>
<id>oracle</id>
<modules>
<module>debezium-connector-oracle</module>
</modules>
</profile>
</profiles>
</project>