layout: section title: “Beam SQL: CREATE EXTERNAL TABLE Statement” section_menu: section-menu/sdks.html permalink: /documentation/dsls/sql/create-external-table/ redirect_from:
Beam SQL's CREATE EXTERNAL TABLE
statement registers a virtual table that maps to an [external storage system]({{ site.baseurl }}/documentation/io/built-in/). For some storage systems, CREATE EXTERNAL TABLE
does not create a physical table until a write occurs. After the physical table exists, you can access the table with the SELECT
, JOIN
, and INSERT INTO
statements.
The CREATE EXTERNAL TABLE
statement includes a schema and extended clauses.
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*) TYPE type [LOCATION location] [TBLPROPERTIES tblProperties] simpleType: TINYINT | SMALLINT | INTEGER | BIGINT | FLOAT | DOUBLE | DECIMAL | BOOLEAN | DATE | TIME | TIMESTAMP | CHAR | VARCHAR fieldType: simpleType | MAP<simpleType, fieldType> | ARRAY<fieldType> | ROW<tableElement [, tableElement ]*> tableElement: columnName fieldType [ NOT NULL ]
IF NOT EXISTS
: Optional. If the table is already registered, Beam SQL ignores the statement instead of returning an error.tableName
: The case sensitive name of the table to create and register, specified as an [Identifier]({{ site.baseurl }}/documentation/dsls/sql/lexical/#identifiers). The table name does not need to match the name in the underlying data storage system.tableElement
: columnName
fieldType
[ NOT NULL ]
columnName
: The case sensitive name of the column, specified as a backtick_quoted_expression.fieldType
: The field's type, specified as one of the following types:simpleType
: TINYINT
, SMALLINT
, INTEGER
, BIGINT
, FLOAT
, DOUBLE
, DECIMAL
, BOOLEAN
, DATE
, TIME
, TIMESTAMP
, CHAR
, VARCHAR
MAP<simpleType, fieldType>
ARRAY<fieldType>
ROW<tableElement [, tableElement ]*>
NOT NULL
: Optional. Indicates that the column is not nullable.type
: The I/O transform that backs the virtual table, specified as an [Identifier]({{ site.baseurl }}/documentation/dsls/sql/lexical/#identifiers) with one of the following values:bigquery
pubsub
kafka
text
location
: The I/O specific location of the underlying table, specified as a [String Literal]({{ site.baseurl }}/documentation/dsls/sql/lexical/#string-literals). See the I/O specific sections for location
format requirements.tblProperties
: The I/O specific quoted key value JSON object with extra configuration, specified as a [String Literal]({{ site.baseurl }}/documentation/dsls/sql/lexical/#string-literals). See the I/O specific sections for tblProperties
format requirements.CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*) TYPE bigquery LOCATION '[PROJECT_ID]:[DATASET].[TABLE]'
LOCATION:
Location of the table in the BigQuery CLI format.PROJECT_ID
: ID of the Google Cloud ProjectDATASET
: BigQuery Dataset IDTABLE
: BigQuery Table ID within the DatasetBeam SQL supports reading columns with simple types (simpleType
) and arrays of simple types (ARRAY<simpleType>
).
if the table does not exist, Beam creates the table specified in location when the first record is written. If the table does exist, the specified columns must match the existing table.
Schema-related errors will cause the pipeline to crash. The Map type is not supported. Beam SQL types map to BigQuery Standard SQL types as follows:
CREATE EXTERNAL TABLE users (id INTEGER, username VARCHAR) TYPE bigquery LOCATION 'testing-integration:apache.users'
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName ( event_timestamp TIMESTAMP, attributes MAP<VARCHAR, VARCHAR>, payload ROW<tableElement [, tableElement ]*> ) TYPE pubsub LOCATION 'projects/[PROJECT]/topics/[TOPIC]' TBLPROPERTIES '{"timestampAttributeKey": "key", "deadLetterQueue": "projects/[PROJECT]/topics/[TOPIC]"}'
event_timestamp
: The event timestamp associated with the Pub/Sub message by PubsubIO. It can be one of the following:timestampAttributeKey
field of the tblProperties
blob. The value of the attribute should conform to the requirements of PubsubIO, which is either millis since Unix epoch or RFC 339 date string.attributes
: The user-provided attributes map from the Pub/Sub message;payload
: The schema of the JSON payload of the Pub/Sub message. No other payload formats are currently supported by Beam SQL. If a record can't be unmarshalled, the record is written to the topic specified in the deadLeaderQueue
field of the tblProperties
blob. If no dead-letter queue is specified in this case, an exception is thrown and the pipeline will crash.LOCATION
:PROJECT
: ID of the Google Cloud ProjectTOPIC
: The Pub/Sub topic name. A subscription will be created automatically, but the subscription is not cleaned up automatically. Specifying an existing subscription is not supported.TBLPROPERTIES
:timestampAttributeKey
: Optional. The key which contains the event timestamp associated with the Pub/Sub message. If not specified, the message publish timestamp is used as an event timestamp for windowing/watermarking.deadLetterQueue
: The topic into which messages are written if the payload was not parsed. If not specified, an exception is thrown for parsing failures.PubsubIO is currently limited to read access only.
Not supported. PubSubIO is currently limited to read access only in Beam SQL.
Pub/Sub messages have metadata associated with them, and you can reference this metadata in your queries. For each message, Pub/Sub exposes its publish time and a map of user-provided attributes in addition to the payload (unstructured in the general case). This information must be preserved and accessible from the SQL statements. Currently, this means that PubsubIO tables require you to declare a special set of columns, as shown below.
payload
field.CREATE EXTERNAL TABLE locations (event_timestamp TIMESTAMP, attributes MAP<VARCHAR, VARCHAR>, payload ROW<id INTEGER, location VARCHAR>) TYPE pubsub LOCATION 'projects/testing-integration/topics/user-location'
KafkaIO is experimental in Beam SQL.
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*) TYPE kafka LOCATION 'kafka://localhost:2181/brokers' TBLPROPERTIES '{"bootstrap.servers":"localhost:9092", "topics": ["topic1", "topic2"]}'
LOCATION
: The Kafka topic URL.TBLPROPERTIES
:bootstrap.servers
: Optional. Allows you to specify the bootstrap server.topics
: Optional. Allows you to specify specific topics.Read Mode supports reading from a topic.
Write Mode supports writing to a topic.
Only simple types are supported.
TextIO is experimental in Beam SQL. Read Mode and Write Mode do not currently access the same underlying data.
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*) TYPE text LOCATION '/home/admin/orders' TBLPROPERTIES '{"format: "Excel"}'
LOCATION
: The path to the file for Read Mode. The prefix for Write Mode.TBLPROPERTIES
:format
: Optional. Allows you to specify the CSV Format, which controls the field delimeter, quote character, record separator, and other properties. See the following table:Value for format | Field delimiter | Quote | Record separator | Ignore empty lines? | Allow missing column names? |
---|---|---|---|---|---|
default | , | " | \r\n | Yes | No |
rfc4180 | , | " | \r\n | No | No |
excel | , | " | \r\n | No | Yes |
tdf | \t | " | \r\n | Yes | No |
mysql | \t | none | \n | No | No |
{:.table-bordered} |
Read Mode supports reading from a file.
Write Mode supports writing to a set of files. TextIO creates file on writes.
Only simple types are supported.
CREATE EXTERNAL TABLE orders (id INTEGER, price INTEGER) TYPE text LOCATION '/home/admin/orders'