Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion .github/trigger_files/beam_PostCommit_SQL.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run ",
"modification": 3
"modification": 2
}
1 change: 1 addition & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,7 @@
* Milvus enrichment handler added (Python) ([#35216](https://github.com/apache/beam/pull/35216)).
Beam now supports Milvus enrichment handler capabilities for vector, keyword,
and hybrid search operations.
* [Beam SQL] Add support for DATABASEs, with an implementation for Iceberg ([]())

## Breaking Changes

Expand Down
6 changes: 6 additions & 0 deletions sdks/java/extensions/sql/src/main/codegen/config.fmpp
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,14 @@ data: {
"org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlDrop"
"org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName"
"org.apache.beam.sdk.extensions.sql.impl.parser.SqlCreateCatalog"
"org.apache.beam.sdk.extensions.sql.impl.parser.SqlCreateDatabase"
"org.apache.beam.sdk.extensions.sql.impl.parser.SqlCreateExternalTable"
"org.apache.beam.sdk.extensions.sql.impl.parser.SqlCreateFunction"
"org.apache.beam.sdk.extensions.sql.impl.parser.SqlDropCatalog"
"org.apache.beam.sdk.extensions.sql.impl.parser.SqlDropDatabase"
"org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes"
"org.apache.beam.sdk.extensions.sql.impl.parser.SqlUseCatalog"
"org.apache.beam.sdk.extensions.sql.impl.parser.SqlUseDatabase"
"org.apache.beam.sdk.extensions.sql.impl.parser.SqlSetOptionBeam"
"org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils"
"org.apache.beam.sdk.schemas.Schema"
Expand Down Expand Up @@ -395,6 +398,7 @@ data: {
# Example: SqlShowDatabases(), SqlShowTables().
statementParserMethods: [
"SqlUseCatalog(Span.of(), null)"
"SqlUseDatabase(Span.of(), null)"
"SqlSetOptionBeam(Span.of(), null)"
]

Expand Down Expand Up @@ -427,6 +431,7 @@ data: {
# Each must accept arguments "(SqlParserPos pos, boolean replace)".
createStatementParserMethods: [
"SqlCreateCatalog"
"SqlCreateDatabase"
"SqlCreateExternalTable"
"SqlCreateFunction"
"SqlCreateTableNotSupportedMessage"
Expand All @@ -436,6 +441,7 @@ data: {
# Each must accept arguments "(SqlParserPos pos)".
dropStatementParserMethods: [
"SqlDropTable"
"SqlDropDatabase"
"SqlDropCatalog"
]

Expand Down
89 changes: 89 additions & 0 deletions sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,15 @@ boolean IfExistsOpt() :
{ return false; }
}

boolean CascadeOpt() :
{
}
{
<CASCADE> { return true; }
|
{ return false; }
}

SqlNodeList Options() :
{
final Span s;
Expand Down Expand Up @@ -255,6 +264,86 @@ SqlDrop SqlDropCatalog(Span s, boolean replace) :
}
}

/**
* CREATE DATABASE ( IF NOT EXISTS )? database_name
*/
SqlCreate SqlCreateDatabase(Span s, boolean replace) :
{
final boolean ifNotExists;
final SqlNode databaseName;
}
{
<DATABASE> {
s.add(this);
}

ifNotExists = IfNotExistsOpt()
(
databaseName = StringLiteral()
|
databaseName = SimpleIdentifier()
)

{
return new SqlCreateDatabase(
s.end(this),
replace,
ifNotExists,
databaseName);
}
}

/**
* USE DATABASE database_name
*/
SqlCall SqlUseDatabase(Span s, String scope) :
{
final SqlNode databaseName;
}
{
<USE> {
s.add(this);
}
<DATABASE>
(
databaseName = StringLiteral()
|
databaseName = SimpleIdentifier()
)
{
return new SqlUseDatabase(
s.end(this),
scope,
databaseName);
}
}

/**
* DROP DATABASE [ IF EXISTS ] database_name [ RESTRICT | CASCADE ]
*/
SqlDrop SqlDropDatabase(Span s, boolean replace) :
{
final boolean ifExists;
final SqlNode databaseName;
final boolean cascade;
}
{
<DATABASE>
ifExists = IfExistsOpt()
(
databaseName = StringLiteral()
|
databaseName = SimpleIdentifier()
)

cascade = CascadeOpt()

{
return new SqlDropDatabase(s.end(this), ifExists, databaseName, cascade);
}
}


SqlNodeList PartitionFieldList() :
{
final List<SqlNode> list = new ArrayList<SqlNode>();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.extensions.sql.impl.parser;

import static java.lang.String.format;
import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE;

import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema;
import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog;
import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCreate;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class SqlCreateDatabase extends SqlCreate implements BeamSqlParser.ExecutableStatement {
private static final Logger LOG = LoggerFactory.getLogger(SqlCreateDatabase.class);
private final SqlIdentifier databaseName;
private static final SqlOperator OPERATOR =
new SqlSpecialOperator("CREATE DATABASE", SqlKind.OTHER_DDL);

public SqlCreateDatabase(
SqlParserPos pos, boolean replace, boolean ifNotExists, SqlNode databaseName) {
super(OPERATOR, pos, replace, ifNotExists);
this.databaseName = SqlDdlNodes.getIdentifier(databaseName, pos);
}

@Override
public List<SqlNode> getOperandList() {
ImmutableList.Builder<SqlNode> operands = ImmutableList.builder();
operands.add(databaseName);
return operands.build();
}

@Override
public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
writer.keyword("CREATE");
if (getReplace()) {
writer.keyword("OR REPLACE");
}
writer.keyword("DATABASE");
if (ifNotExists) {
writer.keyword("IF NOT EXISTS");
}
databaseName.unparse(writer, leftPrec, rightPrec);
}

@Override
public void execute(CalcitePrepare.Context context) {
final Pair<CalciteSchema, String> pair = SqlDdlNodes.schema(context, true, databaseName);
Schema schema = pair.left.schema;
String name = pair.right;

if (!(schema instanceof BeamCalciteSchema)) {
throw SqlUtil.newContextException(
databaseName.getParserPosition(),
RESOURCE.internal("Schema is not of instance BeamCalciteSchema"));
}

@Nullable CatalogManager catalogManager = ((BeamCalciteSchema) schema).getCatalogManager();
if (catalogManager == null) {
throw SqlUtil.newContextException(
databaseName.getParserPosition(),
RESOURCE.internal(
format(
"Unexpected 'CREATE DATABASE' call using Schema '%s' that is not a Catalog.",
name)));
}

// Attempt to create the database.
Catalog catalog = catalogManager.currentCatalog();
try {
LOG.info("Creating database '{}'", name);
boolean created = catalog.createDatabase(name);

if (created) {
LOG.info("Successfully created database '{}'", name);
} else if (ifNotExists) {
LOG.info("Database '{}' already exists.", name);
} else {
throw SqlUtil.newContextException(
databaseName.getParserPosition(),
RESOURCE.internal(format("Database '%s' already exists.", name)));
}
} catch (Exception e) {
throw SqlUtil.newContextException(
databaseName.getParserPosition(),
RESOURCE.internal(
format("Encountered an error when creating database '%s': %s", name, e)));
}
}
}
Loading
Loading