This is an automated email from the ASF dual-hosted git repository.
ahmedabu98 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new 56a41422765 SQL DDL documentation (#37539)
56a41422765 is described below
commit 56a41422765190e9d675a8d15963befced83468b
Author: Ahmed Abualsaud <[email protected]>
AuthorDate: Thu Jun 11 18:38:37 2026 -0400
SQL DDL documentation (#37539)
* create catalog and iceberg table details
* add sql ddl website documentation
* add links
* spotless
* address comments
* fix links
* extra description for databases and tables
* make prettier
* cleanup
* address comments
* address comments
* fix tab switch issue
* cleanup
* address comments
* address comments
* address comments
* fix link
* cleanup
---------
Co-authored-by: Talat Uyarer <[email protected]>
---
.../meta/provider/iceberg/IcebergMetastore.java | 13 +-
.../sql/meta/provider/iceberg/IcebergTable.java | 19 +-
.../meta/provider/iceberg/PubsubToIcebergIT.java | 4 +-
.../beam/sdk/io/iceberg/IcebergCatalogConfig.java | 10 +-
.../site/content/en/documentation/dsls/sql/ddl.md | 344 +++++++++++++++++++++
.../content/en/documentation/dsls/sql/overview.md | 2 +
.../en/documentation/io/built-in/iceberg.md | 2 +-
.../layouts/partials/section-menu/en/sdks.html | 1 +
website/www/site/layouts/shortcodes/tab.html | 22 ++
9 files changed, 405 insertions(+), 12 deletions(-)
diff --git
a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java
b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java
index 678c76153c5..5604b7c0837 100644
---
a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java
+++
b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java
@@ -23,6 +23,7 @@ import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Pr
import com.fasterxml.jackson.core.type.TypeReference;
import java.util.HashMap;
import java.util.Map;
+import java.util.stream.Collectors;
import org.apache.beam.sdk.extensions.sql.TableUtils;
import org.apache.beam.sdk.extensions.sql.impl.TableName;
import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
@@ -60,12 +61,14 @@ public class IcebergMetastore extends InMemoryMetaStore {
getProvider(table.getType()).createTable(table);
} else {
String identifier = getIdentifier(table);
+ Map<String, String> props =
+ TableUtils.getObjectMapper()
+ .convertValue(table.getProperties(), new
TypeReference<Map<String, String>>() {})
+ .entrySet().stream()
+ .filter(p -> !p.getKey().startsWith("beam."))
+ .collect(Collectors.toMap(Map.Entry::getKey,
Map.Entry::getValue));
try {
- Map<String, String> properties =
- TableUtils.getObjectMapper()
- .convertValue(table.getProperties(), new
TypeReference<Map<String, String>>() {});
- catalogConfig.createTable(
- identifier, table.getSchema(), table.getPartitionFields(),
properties);
+ catalogConfig.createTable(identifier, table.getSchema(),
table.getPartitionFields(), props);
} catch (TableAlreadyExistsException e) {
LOG.info(
"Iceberg table '{}' already exists at location '{}'.",
table.getName(), identifier);
diff --git
a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java
b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java
index b68aa34a177..2cdcaa9b63a 100644
---
a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java
+++
b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java
@@ -19,6 +19,7 @@ package
org.apache.beam.sdk.extensions.sql.meta.provider.iceberg;
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
import java.util.HashMap;
import java.util.List;
@@ -56,6 +57,8 @@ class IcebergTable extends SchemaBaseBeamTable {
@VisibleForTesting static final String CATALOG_PROPERTIES_FIELD =
"catalog_properties";
@VisibleForTesting static final String HADOOP_CONFIG_PROPERTIES_FIELD =
"config_properties";
@VisibleForTesting static final String CATALOG_NAME_FIELD = "catalog_name";
+ static final String BEAM_WRITE_PROPERTY = "beam.write.";
+ static final String BEAM_READ_PROPERTY = "beam.read.";
@VisibleForTesting
static final String TRIGGERING_FREQUENCY_FIELD =
"triggering_frequency_seconds";
@@ -71,9 +74,21 @@ class IcebergTable extends SchemaBaseBeamTable {
this.tableIdentifier = tableIdentifier;
this.catalogConfig = catalogConfig;
ObjectNode properties = table.getProperties();
- if (properties.has(TRIGGERING_FREQUENCY_FIELD)) {
- this.triggeringFrequency =
properties.get(TRIGGERING_FREQUENCY_FIELD).asInt();
+ for (Map.Entry<String, JsonNode> property : properties.properties()) {
+ String name = property.getKey().toLowerCase();
+ if (name.startsWith(BEAM_WRITE_PROPERTY)) {
+ String prop = name.substring(BEAM_WRITE_PROPERTY.length());
+ if (prop.equalsIgnoreCase(TRIGGERING_FREQUENCY_FIELD)) {
+ this.triggeringFrequency = property.getValue().asInt();
+ } else {
+ throw new IllegalArgumentException("Unknown Beam write property: " +
name);
+ }
+ } else if (name.startsWith(BEAM_READ_PROPERTY)) {
+ // none supported yet
+ throw new IllegalArgumentException("Unknown Beam read property: " +
name);
+ }
}
+
this.partitionFields = table.getPartitionFields();
}
diff --git
a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java
b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java
index 900fdae743a..8b250af2754 100644
---
a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java
+++
b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java
@@ -156,7 +156,7 @@ public class PubsubToIcebergIT implements Serializable {
+ ") \n"
+ "TYPE 'iceberg' \n"
+ "PARTITIONED BY('id', 'truncate(name, 3)') \n"
- + "TBLPROPERTIES '{ \"triggering_frequency_seconds\" : 10 }'";
+ + "TBLPROPERTIES '{ \"beam.write.triggering_frequency_seconds\" :
10 }'";
String insertStatement =
format("INSERT INTO %s \n", tableIdentifier)
+ "SELECT \n"
@@ -211,7 +211,7 @@ public class PubsubToIcebergIT implements Serializable {
+ " name VARCHAR \n "
+ ") \n"
+ "TYPE 'iceberg' \n"
- + "TBLPROPERTIES '{ \"triggering_frequency_seconds\" : 10 }'";
+ + "TBLPROPERTIES '{ \"beam.write.triggering_frequency_seconds\" :
10 }'";
String insertStatement =
format("INSERT INTO %s \n", tableIdentifier)
+ "SELECT \n"
diff --git
a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java
b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java
index e01b174decb..6fb0a3480a0 100644
---
a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java
+++
b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java
@@ -154,7 +154,7 @@ public abstract class IcebergCatalogConfig implements
Serializable {
String tableIdentifier,
Schema tableSchema,
@Nullable List<String> partitionFields,
- Map<String, String> properties) {
+ @Nullable Map<String, String> properties) {
TableIdentifier icebergIdentifier = TableIdentifier.parse(tableIdentifier);
org.apache.iceberg.Schema icebergSchema =
IcebergUtils.beamSchemaToIcebergSchema(tableSchema);
PartitionSpec icebergSpec =
PartitionUtils.toPartitionSpec(partitionFields, tableSchema);
@@ -164,7 +164,13 @@ public abstract class IcebergCatalogConfig implements
Serializable {
icebergIdentifier,
icebergSchema,
icebergSpec);
- catalog().createTable(icebergIdentifier, icebergSchema, icebergSpec,
properties);
+ Catalog.TableBuilder builder =
+ catalog().buildTable(icebergIdentifier,
icebergSchema).withPartitionSpec(icebergSpec);
+ if (properties != null) {
+ builder = builder.withProperties(properties);
+ }
+ builder.create();
+
LOG.info("Successfully created table '{}'.", icebergIdentifier);
} catch (AlreadyExistsException e) {
throw new TableAlreadyExistsException(e);
diff --git a/website/www/site/content/en/documentation/dsls/sql/ddl.md
b/website/www/site/content/en/documentation/dsls/sql/ddl.md
new file mode 100644
index 00000000000..11d588a4c35
--- /dev/null
+++ b/website/www/site/content/en/documentation/dsls/sql/ddl.md
@@ -0,0 +1,344 @@
+---
+type: languages
+title: "Beam SQL DDL"
+---
+<!--
+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.
+-->
+
+# Beam SQL DDL
+
+Beam SQL Data Definition Language (DDL) provides a standard three-level
hierarchy to manage metadata across external data sources,
+making it easy to explore available data structures and query data across
different systems.
+1. **Catalog**: The top-level container representing an external metadata
provider. For example, a Hive Metastore, AWS Glue, or a Lakehouse (formerly
BigLake) Catalog.
+2. **Database**: A logical grouping within a Catalog. This typically maps to a
"Schema" in traditional RDBMS or a "Namespace" in systems like Apache Iceberg.
+3. **Table**: The leaf node containing the schema definition and the
underlying data.
+
+Beam can resolve multiple Catalogs simultaneously. This structure enables
Federated Querying, meaning
+you can execute complex pipelines that bridge disparate environments within a
single SQL statement.
+For example, you could jointly query a production BigQuery table and a
development Iceberg dataset in Cloud Storage.
+
+By using fully qualified names (for example, `catalog.database.table`), you
can perform cross-catalog joins or
+migrate data between cloud providers without manual schema mapping or
intermediate storage.
+
+Below are details about metadata management at each level:
+
+## Catalogs
+The Catalog is the entry point for external metadata. When you initialize Beam
SQL, you start off with a `default` Catalog that contains a `default` Database.
+You can register new Catalogs, switch between them, and modify their
configurations.
+
+{{< tab CREATE >}}
+<p>Registers a new Catalog instance</p>
+<p><i><strong>Note</strong>: Creating a Catalog does not automatically switch
to it. Remember
+to run <code>USE CATALOG</code> afterwards to set it.</i></p>
+
+{{< highlight >}}
+CREATE CATALOG [ IF NOT EXISTS ] catalog_name
+TYPE 'type_name'
+[ PROPERTIES ( 'key' = 'value' [, ...] ) ]
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: Creating a Hadoop Catalog (Local
Storage)</i></p>
+{{< highlight >}}
+CREATE CATALOG local_catalog
+TYPE iceberg
+PROPERTIES (
+ 'type' = 'hadoop',
+ 'warehouse' = 'file:///tmp/iceberg-warehouse'
+)
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: Registering a Lakehouse Catalog (GCS)</i></p>
+{{< highlight >}}
+CREATE CATALOG prod_iceberg
+TYPE iceberg
+PROPERTIES (
+ 'type' = 'rest',
+ 'uri' = 'https://biglake.googleapis.com/iceberg/v1/restcatalog',
+ 'warehouse' = 'gs://my-company-bucket/warehouse',
+ 'header.x-goog-user-project' = 'my_prod_project',
+ 'rest.auth.type' = 'google',
+ 'io-impl' = 'org.apache.iceberg.gcp.gcs.GCSFileIO',
+ 'header.X-Iceberg-Access-Delegation' = 'vended-credentials'
+);
+{{< /highlight >}}
+{{< /tab >}}
+{{< tab USE >}}
+<p>Sets the active Catalog for the current session. This simplifies queries by
allowing you
+to reference Databases directly without their fully-qualified names. For
example, you can use <code>my_db</code> instead of
<code>my_catalog.my_db</code>.</p>
+
+<p><i><strong>Tip:</strong> run <code>SHOW CURRENT CATALOG</code> to view the
currently active Catalog.</i></p>
+<p><i><strong>Note:</strong> All subsequent DATABASE and TABLE commands will
be executed under this Catalog, unless fully qualified.</i></p>
+
+{{< highlight >}}
+USE CATALOG prod_iceberg;
+{{< /highlight >}}
+{{< /tab >}}
+{{< tab ALTER >}}
+Modifies the properties of a registered Catalog.
+{{< highlight >}}
+ALTER CATALOG catalog_name
+[ SET ( 'key' = 'val', ... ) ]
+[ RESET ( 'key', ... ) ]
+{{< /highlight >}}
+<ol>
+ <li><strong>SET:</strong> Adds new properties or updates existing ones.</li>
+ <li><strong>RESET / UNSET:</strong> Removes properties.</li>
+</ol>
+<br>
+{{< /tab >}}
+{{< tab SHOW >}}
+<p>Can be used to either:</p>
+<ol>
+ <li>List Catalogs registered in this Beam SQL session.</li>
+ <li>View the currently active Catalog.</li>
+</ol>
+
+{{< highlight >}}
+SHOW CATALOGS [ LIKE regex_pattern ]
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: List all Catalogs</i></p>
+{{< highlight >}}
+SHOW CATALOGS;
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: List Catalogs matching a pattern</i></p>
+{{< highlight >}}
+SHOW CATALOGS LIKE 'prod_%';
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: Verify which Catalog is currently
active</i></p>
+{{< highlight >}}
+SHOW CURRENT CATALOG;
+{{< /highlight >}}
+{{< /tab >}}
+{{< tab DROP >}}
+<p>Unregisters a Catalog from the current Beam SQL session. This does not
destroy external data.</p>
+
+{{< highlight >}}
+DROP CATALOG [ IF EXISTS ] catalog_name;
+{{< /highlight >}}
+{{< /tab >}}
+
+## Databases
+A Database lives inside a Catalog and may contain a number of Tables.
+
+{{< tab CREATE >}}
+<p>Creates a new Database within the current Catalog (default), or the
specified Catalog.</p>
+<p><i><strong>Note</strong>: Creating a Database does not automatically switch
to it. Remember
+to run <code>USE DATABASE</code> afterwards to set it.</i></p>
+
+{{< highlight >}}
+CREATE DATABASE [ IF NOT EXISTS ] [ catalog_name. ]database_name;
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: Create a Database in the current active
Catalog</i></p>
+{{< highlight >}}
+USE CATALOG my_catalog;
+CREATE DATABASE sales_data;
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: Create a Database in a specified registered
Catalog</i></p>
+{{< highlight >}}
+CREATE DATABASE other_catalog.sales_data;
+{{< /highlight >}}
+{{< /tab >}}
+{{< tab USE >}}
+<p>Sets the active Database for the current session. This simplifies queries
by allowing you
+to reference Databases directly without their fully-qualified names (for
example, <code>my_db</code> instead of <code>my_catalog.my_db</code>)</p>
+
+<p><i><strong>Note:</strong> All subsequent TABLE commands will be executed
under this Database, unless fully qualified.</i></p>
+
+{{< highlight >}}
+USE DATABASE sales_data;
+{{< /highlight >}}
+
+<p><i>Switch to a Database in a specified Catalog.</i></p>
+<p><i><strong>Note</strong>: this also switches the default to that
Catalog</i></p>
+{{< highlight >}}
+USE DATABASE other_catalog.sales_data;
+{{< /highlight >}}
+{{< /tab >}}
+{{< tab SHOW >}}
+<p>Can be used to either:</p>
+<ol>
+ <li>List Databases within the currently active Catalog, or a specified
Catalog.</li>
+ <li>View the currently active Database.</li>
+</ol>
+
+{{< highlight >}}
+SHOW DATABASES [ ( FROM | IN )? catalog_name ] [LIKE regex_pattern ]
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: List Databases in the currently active
Catalog</i></p>
+{{< highlight >}}
+SHOW DATABASES;
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: List Databases in a specified Catalog</i></p>
+{{< highlight >}}
+SHOW DATABASES IN my_catalog;
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: List Databases matching a pattern</i></p>
+{{< highlight >}}
+SHOW DATABASES IN my_catalog LIKE '%geo%';
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: Verify which Database is currently
active</i></p>
+{{< highlight >}}
+SHOW CURRENT DATABASE;
+{{< /highlight >}}
+{{< /tab >}}
+{{< tab DROP >}}
+<p>Unregisters a Database from the current session. For some connectors, this
+will also <strong>delete</strong> the Database from the external data
source.</p>
+
+{{< highlight >}}
+DROP DATABASE [ IF EXISTS ] database_name [ RESTRICT | CASCADE ];
+{{< /highlight >}}
+
+<ol>
+ <li><strong>RESTRICT</strong> (Default): Fails if the Database is not
empty.</li>
+ <li><strong>CASCADE:</strong> Drops the Database and all tables contained
within it. <strong>Use with caution</strong>.</li>
+</ol>
+<br>
+{{< /tab >}}
+
+## Tables
+The actual entity containing data, and is described by a schema. Some
+data sources also let you apply a partition spec or attach table-specific
properties.
+
+{{< tab CREATE >}}
+<p>Creates a new Table within the current Catalog and Database (default), or
the Catalog and Database you specify.</p>
+
+{{< highlight >}}
+CREATE EXTERNAL TABLE [ IF NOT EXISTS ] [ catalog. ][ db. ]table_name (
+ col_name col_type [ NOT NULL ] [ COMMENT 'col_comment' ],
+ ...
+ )
+ TYPE 'type_name'
+ [ PARTITIONED BY ( 'partition_field' [, ... ] ) ]
+ [ COMMENT 'table_comment' ]
+ [ LOCATION 'location_uri' ]
+ [ TBLPROPERTIES 'properties_json_string' ];
+{{< /highlight >}}
+<ul>
+ <li><strong>TYPE:</strong> the table type (for example,
<code>'iceberg'</code>, <code>'text'</code>, <code>'kafka'</code>).</li>
+ <li><strong>PARTITIONED BY:</strong> an ordered list of fields describing
the partition spec.</li>
+ <li><strong>LOCATION:</strong> explicitly sets the location of the table.
This overrides the inferred <code>catalog.db.table_name</code> location.</li>
+ <li><strong>TBLPROPERTIES:</strong> configuration properties used when
creating the table or setting up its IO connection.</li>
+</ul>
+<br>
+
+<p><i><strong>Example</strong>: Creating an Iceberg Table</i></p>
+{{< highlight >}}
+CREATE EXTERNAL TABLE prod_iceberg.sales_data.orders (
+ order_id BIGINT NOT NULL COMMENT 'Unique order identifier',
+ amount DECIMAL(10, 2),
+ order_date TIMESTAMP,
+ region_id VARCHAR
+)
+TYPE 'iceberg'
+PARTITIONED BY ( 'region_id', 'day(order_date)' )
+COMMENT 'Daily sales transactions'
+TBLPROPERTIES '{
+ "write.format.default": "parquet",
+ "read.split.target-size": 268435456",
+ "beam.write.triggering_frequency_seconds": 60"
+}';
+{{< /highlight >}}
+
+<ul>
+ <li>This creates an Iceberg table named <code>orders</code> under the
namespace <code>sales_data</code>, within the <code>prod_iceberg</code>
catalog.</li>
+ <li>The table is partitioned by <code>region_id</code>, then by the day
value of <code>order_date</code> (using Iceberg's <a
href="https://iceberg.apache.org/docs/latest/partitioning/#icebergs-hidden-partitioning">hidden
partitioning</a>).</li>
+ <li>The table is created with the appropriate properties
<code>"write.format.default"</code> and <code>"read.split.target-size"</code>.
The Beam property <code>"beam.write.triggering_frequency_seconds"</code>
configures the Iceberg sink.</li>
+ <li>Beam sink and source configuration properties are prefixed with
<code>"beam.write."</code> and <code>"beam.read."</code>, respectively.</li>
+</ul>
+{{< /tab >}}
+{{< tab ALTER >}}
+Modifies an existing Table's properties and evolves its partition and schema.
+{{< highlight >}}
+ALTER TABLE table_name
+ [ ADD COLUMNS ( col_def, ... ) ]
+ [ DROP COLUMNS ( col_name, ... ) ]
+ [ ADD PARTITIONS ( partition_field, ... ) ]
+ [ DROP PARTITIONS ( partition_field, ... ) ]
+ [ SET ( 'key' = 'val', ... ) ]
+ [ ( RESET | UNSET ) ( 'key', ... ) ];
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: Add or remove columns</i></p>
+{{< highlight >}}
+-- Add columns
+ALTER TABLE orders ADD COLUMNS (
+ customer_email VARCHAR,
+ shipping_region VARCHAR
+);
+
+-- Drop columns
+ALTER TABLE orders DROP COLUMNS ( customer_email );
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: Modify partition spec</i></p>
+{{< highlight >}}
+-- Add a partition field
+ALTER TABLE orders ADD PARTITIONS ( 'year(order_date)' );
+
+-- Remove a partition field
+ALTER TABLE orders DROP PARTITIONS ( 'region_id' );
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: Modify table properties</i></p>
+{{< highlight >}}
+ALTER TABLE orders SET (
+ 'write.format.default' = 'orc',
+ 'write.metadata.metrics.default' = 'full' );
+
+ALTER TABLE orders RESET ( 'write.target-file-size-bytes' );
+{{< /highlight >}}
+
+{{< /tab >}}
+{{< tab SHOW >}}
+<p>Lists tables under the currently active database, or a database you
specify.</p>
+
+{{< highlight >}}
+SHOW TABLES [ ( FROM | IN )? [ catalog_name '.' ] database_name ] [ LIKE
regex_pattern ]
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: List tables in the currently active database
and catalog</i></p>
+{{< highlight >}}
+SHOW TABLES;
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: List tables in a specified database</i></p>
+{{< highlight >}}
+SHOW TABLES IN my_db;
+SHOW TABLES IN my_catalog.my_db;
+{{< /highlight >}}
+
+<p><i><strong>Example</strong>: List tables matching a pattern</i></p>
+{{< highlight >}}
+SHOW TABLES IN my_db LIKE '%orders%';
+{{< /highlight >}}
+
+{{< /tab >}}
+{{< tab DROP >}}
+<p>Unregisters a table from the current session. For supported connectors, this
+will also <strong>delete</strong> the table from the external data source.</p>
+
+{{< highlight >}}
+DROP TABLE [ IF EXISTS ] table_name;
+{{< /highlight >}}
+{{< /tab >}}
diff --git a/website/www/site/content/en/documentation/dsls/sql/overview.md
b/website/www/site/content/en/documentation/dsls/sql/overview.md
index 7aa9d7bab29..12bcec1a077 100644
--- a/website/www/site/content/en/documentation/dsls/sql/overview.md
+++ b/website/www/site/content/en/documentation/dsls/sql/overview.md
@@ -23,6 +23,8 @@ bounded and unbounded `PCollections` with SQL statements.
Your SQL query
is translated to a `PTransform`, an encapsulated segment of a Beam pipeline.
You can freely mix SQL `PTransforms` and other `PTransforms` in your pipeline.
+Beam SQL extends [DDL commands](/documentation/dsls/sql/ddl) for supported
catalogs.
+
Beam SQL uses Calcite SQL based on [Apache
Calcite](https://calcite.apache.org),
a dialect widespread in big data processing.
diff --git a/website/www/site/content/en/documentation/io/built-in/iceberg.md
b/website/www/site/content/en/documentation/io/built-in/iceberg.md
index 732d8308c0c..3f3b15b5786 100644
--- a/website/www/site/content/en/documentation/io/built-in/iceberg.md
+++ b/website/www/site/content/en/documentation/io/built-in/iceberg.md
@@ -102,7 +102,7 @@ When you've met those prerequisites, start by setting up
your catalog:
{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py"
biglake_public_catalog_props >}}
{{< /highlight >}}
{{< highlight yaml >}}
-catalog_props: &catalog_props
+catalog_props: &biglake_catalog_props
type: "rest"
uri: "https://biglake.googleapis.com/iceberg/v1/restcatalog"
warehouse: "gs://biglake-public-nyc-taxi-iceberg"
diff --git a/website/www/site/layouts/partials/section-menu/en/sdks.html
b/website/www/site/layouts/partials/section-menu/en/sdks.html
index 45fc937ac1f..633d14c2328 100644
--- a/website/www/site/layouts/partials/section-menu/en/sdks.html
+++ b/website/www/site/layouts/partials/section-menu/en/sdks.html
@@ -112,6 +112,7 @@
<li><a href="/documentation/dsls/sql/overview/">Overview</a></li>
<li><a href="/documentation/dsls/sql/walkthrough/">Walkthrough</a></li>
<li><a href="/documentation/dsls/sql/shell/">Shell</a></li>
+ <li><a href="/documentation/dsls/sql/ddl/">DDL</a></li>
<li class="section-nav-item--collapsible">
<span class="section-nav-list-title">Apache Calcite dialect</span>
diff --git a/website/www/site/layouts/shortcodes/tab.html
b/website/www/site/layouts/shortcodes/tab.html
index a5d6ecd607a..4f329832c67 100644
--- a/website/www/site/layouts/shortcodes/tab.html
+++ b/website/www/site/layouts/shortcodes/tab.html
@@ -10,6 +10,28 @@ See the License for the specific language governing
permissions and
limitations under the License. See accompanying LICENSE file.
*/}}
+{{ if eq .Ordinal 0 }}
+<script>
+ window.addEventListener('click', e => {
+ if (e.target.matches('.tab-switcher li')) {
+ e.stopPropagation();
+
+ e.target.closest('ul').querySelectorAll('li').forEach(li =>
li.classList.remove('active'));
+ e.target.classList.add('active');
+
+ let val = e.target.getAttribute('data-value');
+ let box = e.target.closest('.tab-switcher').nextElementSibling;
+
+ while (box && box.className.startsWith('tab-')) {
+ box.style.display = box.classList.contains('tab-' + val) ? 'block'
: 'none';
+ box = box.nextElementSibling;
+ }
+ }
+ }, true);
+</script>
+{{ end }}
+
+{{/* --- 2. Your Normal Content Box --- */}}
{{ $content := (trim .Inner "\n\r") | htmlUnescape | safeHTML }}
{{ $ctx := . }}
{{ $language := .Get 0 }}