type: languages title: “Beam SQL extension: CREATE EXTERNAL TABLE Statement” aliases:
Beam SQL's CREATE EXTERNAL TABLE
statement registers a virtual table that maps to an external storage system. 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. 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 with one of the following values:bigquery
bigtable
pubsub
kafka
text
location
: The I/O specific location of the underlying table, specified as a String Literal. 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. 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]' TBLPROPERTIES '{"method": "DIRECT_READ"}'
LOCATION
: Location of the table in the BigQuery CLI format.PROJECT_ID
: ID of the Google Cloud Project.DATASET
: BigQuery Dataset ID.TABLE
: BigQuery Table ID within the Dataset.TBLPROPERTIES
:method
: Optional. Read method to use. Following options are available:DIRECT_READ
: Use the BigQuery Storage API.EXPORT
: Export data to Google Cloud Storage in Avro format and read data files from that location.DIRECT_READ
for Beam 2.21+ (older versions use EXPORT
).Beam SQL supports reading columns with simple types (simpleType
) and arrays of simple types (ARRAY<simpleType>
).
When reading using EXPORT
method the following pipeline options should be set:
project
: ID of the Google Cloud Project.tempLocation
: Bucket to store intermediate data in. Ex: gs://temp-storage/temp
.When reading using DIRECT_READ
method, an optimizer will attempt to perform project and predicate push-down, potentially reducing the time requited to read the data from BigQuery.
More information about the BigQuery Storage API can be found here.
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 ( key VARCHAR NOT NULL, family ROW<qualifier cells [, qualifier cells ]* > [, family ROW< qualifier cells [, qualifier cells ]* > ]* ) TYPE bigtable LOCATION 'googleapis.com/bigtable/projects/[PROJECT_ID]/instances/[INSTANCE_ID]/tables/[TABLE]'
key
: key of the Bigtable rowfamily
: name of the column familyqualifier
: the column qualifiercells
: Either of each value:TYPE
ARRAY<SIMPLE_TYPE>
LOCATION
:PROJECT_ID
: ID of the Google Cloud Project.INSTANCE_ID
: Bigtable instance ID.TABLE
: Bigtable Table ID.TYPE
: SIMPLE_TYPE
or CELL_ROW
CELL_ROW
: ROW<val SIMPLE_TYPE [, timestampMicros BIGINT [NOT NULL]] [, labels ARRAY<VARCHAR> [NOT NULL]]
SIMPLE_TYPE
: on of the following:BINARY
VARCHAR
BIGINT
INTEGER
SMALLINT
TINYINT
DOUBLE
FLOAT
BOOLEAN
TIMESTAMP
An alternative syntax with a flat schema:
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName ( key VARCHAR NOT NULL, qualifier SIMPLE_TYPE [, qualifier SIMPLE_TYPE ]* ) TYPE bigtable LOCATION 'googleapis.com/bigtable/projects/[PROJECT_ID]/instances/[INSTANCE_ID]/tables/[TABLE]' TBLPROPERTIES '{ "columnsMapping": "family:qualifier[,family:qualifier]*" }'
key
: key of the Bigtable rowfamily
: name of the column familyqualifier
: the column qualifierLOCATION
:PROJECT_ID
: ID of the Google Cloud Project.INSTANCE_ID
: Bigtable instance ID.TABLE
: Bigtable Table ID.TBLPROPERTIES
: JSON object containing columnsMapping key with comma-separated key-value pairs separated by a colonSIMPLE_TYPE
: the same as in the previous syntaxBeam SQL supports reading rows with mandatory key
field, at least one family
with at least one qualifier
. Cells are represented as simple types (SIMPLE_TYPE
) or ROW type with a mandatory val
field, optional timestampMicros
and optional labels
. Both read the latest cell in the column. Cells specified as Arrays of simple types (ARRAY<simpleType>
) allow to read all the column's values.
For flat schema only SIMPLE_TYPE
values are allowed. Every field except for key
must correspond to the key-values pairs specified in columnsMapping
.
Not all existing column families and qualifiers have to be provided to the schema.
Filters are only allowed by key
field with single LIKE
statement with RE2 Syntax regex, e.g. SELECT * FROM table WHERE key LIKE '^key[012]{1}'
Supported for flat schema only.
CREATE EXTERNAL TABLE beamTable( key VARCHAR NOT NULL, beamFamily ROW< boolLatest BOOLEAN NOT NULL, longLatestWithTs ROW< val BIGINT NOT NULL, timestampMicros BIGINT NOT NULL > NOT NULL, allStrings ARRAY<VARCHAR> NOT NULL, doubleLatestWithTsAndLabels ROW< val DOUBLE NOT NULL, timestampMicros BIGINT NOT NULL, labels ARRAY<VARCHAR> NOT NULL > NOT NULL, binaryLatestWithLabels ROW< val BINARY NOT NULL, labels ARRAY<VARCHAR> NOT NULL > NOT NULL > NOT NULL ) TYPE bigtable LOCATION 'googleapis.com/bigtable/projects/beam/instances/beamInstance/tables/beamTable'
Flat schema example:
CREATE EXTERNAL TABLE flatTable( key VARCHAR NOT NULL, boolColumn BOOLEAN NOT NULL, longColumn BIGINT NOT NULL, stringColumn VARCHAR NOT NULL, doubleColumn DOUBLE NOT NULL, binaryColumn BINARY NOT NULL ) TYPE bigtable LOCATION 'googleapis.com/bigtable/projects/beam/instances/beamInstance/tables/flatTable' TBLPROPERTIES '{ "columnsMapping": "f:boolColumn,f:longColumn,f:stringColumn,f2:doubleColumn,f2:binaryColumn" }'
Write example:
INSERT INTO writeTable(key, boolColumn, longColumn, stringColumn, doubleColumn) VALUES ('key', TRUE, 10, 'stringValue', 5.5)
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName( event_timestamp TIMESTAMP, attributes [MAP<VARCHAR, VARCHAR>, ARRAY<ROW<VARCHAR key, VARCHAR value>>], payload [BYTES, ROW<tableElement [, tableElement ]*>] ) TYPE pubsub LOCATION 'projects/[PROJECT]/topics/[TOPIC]'
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName(tableElement [, tableElement ]*) TYPE pubsub LOCATION 'projects/[PROJECT]/topics/[TOPIC]'
In nested mode, the following fields hold topic metadata. The presence of the attributes
field triggers nested mode usage.
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 payload of the Pub/Sub message. 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.format
: Optional. Allows you to specify the Pubsub payload format.PubsubIO supports reading from topics by creating a new subscription.
PubsubIO supports writing to topics.
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.
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'
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName( publish_timestamp DATETIME, event_timestamp DATETIME, message_key BYTES, attributes ARRAY<ROW<key VARCHAR, `values` ARRAY<VARBINARY>>>, payload [BYTES, ROW<tableElement [, tableElement ]*>] ) TYPE pubsublite // For writing LOCATION 'projects/[PROJECT]/locations/[GCP-LOCATION]/topics/[TOPIC]' // For reading LOCATION 'projects/[PROJECT]/locations/[GCP-LOCATION]/subscriptions/[SUBSCRIPTION]'
LOCATION
:PROJECT
: ID of the Google Cloud ProjectTOPIC
: The Pub/Sub Lite topic name.SUBSCRIPTION
: The Pub/Sub Lite subscription name.GCP-LOCATION
: The location for this Pub/Sub Lite topic os subscription.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
: Optional, supports Generic DLQ Handlingformat
: Optional. Allows you to specify the payload format.PubsubLiteIO supports reading from subscriptions.
PubsubLiteIO supports writing to topics.
CREATE EXTERNAL TABLE locations (event_timestamp TIMESTAMP, attributes ARRAY<ROW<key VARCHAR, `values` ARRAY<VARBINARY>>>, payload ROW<id INTEGER, location VARCHAR>) TYPE pubsublite LOCATION 'projects/testing-integration/locations/us-central1-a/topics/user-location'
KafkaIO is experimental in Beam SQL.
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*) TYPE kafka LOCATION 'my.company.url.com:2181/topic1' TBLPROPERTIES '{ "bootstrap_servers": ["localhost:9092", "PLAINTEXT://192.168.1.200:2181"], "topics": ["topic2", "topic3"], "format": "json" }'
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName ( event_timestamp DATETIME, message_key BYTES, headers ARRAY<ROW<key VARCHAR, `values` ARRAY<VARBINARY>>>, payload [BYTES, ROW<tableElement [, tableElement ]*>] ) TYPE kafka LOCATION 'my.company.url.com:2181/topic1' TBLPROPERTIES '{ "bootstrap_servers": ["localhost:9092", "PLAINTEXT://192.168.1.200:2181"], "topics": ["topic2", "topic3"], "format": "json" }'
The presence of the headers
field triggers nested mode usage.
LOCATION
: A url with the initial bootstrap broker to use and the initial topic name provided as the path.TBLPROPERTIES
:bootstrap_servers
: Optional. Allows you to specify additional bootstrap servers, which are used in addition to the one in LOCATION
.topics
: Optional. Allows you to specify additional topics, which are used in addition to the one in LOCATION
.format
: Optional. Allows you to specify the Kafka values format. Possible values are {csv
, avro
, json
, proto
, thrift
}. Defaults to csv
in flattened mode or json
in nested mode. csv
does not support nested mode.Read Mode supports reading from a topic.
Write Mode supports writing to a topic.
For CSV only simple types are supported.
CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*) TYPE mongodb LOCATION 'mongodb://[HOST]:[PORT]/[DATABASE]/[COLLECTION]'
LOCATION
: Location of the collection.HOST
: Location of the MongoDB server. Can be localhost or an ip address. When authentication is required username and password can be specified as follows: username:password@localhost
.PORT
: Port on which MongoDB server is listening.DATABASE
: Database to connect to.COLLECTION
: Collection within the database.Read Mode supports reading from a collection.
Write Mode supports writing to a collection.
Only simple types are supported. MongoDB documents are mapped to Beam SQL types via JsonToRow
transform.
CREATE EXTERNAL TABLE users (id INTEGER, username VARCHAR) TYPE mongodb LOCATION 'mongodb://localhost:27017/apache/users'
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:{{< table class=“table-bordered” >}} | 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 >}}
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'
Certain data sources and sinks support generic payload handling. This handling parses a byte array payload field into a table schema. The following schemas are supported by this handling. All require at least setting "format": "<type>"
, and may require other properties.
avro
: Avrojson
: JSON Objectsproto
: Protocol BuffersprotoClass
: Required. The proto class name to use. Must be built into the deployed JAR.protoClass
.thrift
: ThriftthriftClass
.thriftClass
: Required. Allows you to specify full thrift java class name. Must be built into the deployed JAR.thriftProtocolFactoryClass
: Required. Allows you to specify full class name of the TProtocolFactory
to use for thrift serialization. Must be built into the deployed JAR.TProtocolFactory
used for thrift serialization must match the provided thriftProtocolFactoryClass
.Sources and sinks which support generic DLQ handling specify a parameter with the format "<dlqParamName>": "[DLQ_KIND]:[DLQ_ID]"
. The following types of DLQ handling are supported:
bigquery
: BigQuerypubsub
: Pub/Sub Topicpubsublite
: Pub/Sub Lite Topic