kennknowles commented on code in PR #36509:
URL: https://github.com/apache/beam/pull/36509#discussion_r2600510966
##########
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java:
##########
@@ -202,6 +208,9 @@ public CatalogSchema getCurrentCatalogSchema() {
if (name == null) {
return null;
}
+ if (name.equals(BeamSystemSchema.BEAMSYSTEM)) {
+ return beamSystemSchema;
Review Comment:
This seems fine, since it is magic schema. But to make it show up in the
data structures and/or in a debugger could you insert it into the map of
subschemas and access it the same way as any other subschema?
##########
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java:
##########
@@ -84,7 +84,10 @@ public void execute(CalcitePrepare.Context context) {
List<String> components =
Lists.newArrayList(Splitter.on(".").split(databaseName.toString()));
TableName pathOverride = TableName.create(components, "");
- CatalogSchema catalogSchema = ((CatalogManagerSchema)
schema).getCatalogSchema(pathOverride);
+ CatalogSchema catalogSchema =
+ pathOverride.catalog() != null
+ ? ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride)
Review Comment:
ditto
##########
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java:
##########
@@ -53,7 +53,10 @@ public void execute(CalcitePrepare.Context context) {
BeamCalciteSchema beamCalciteSchema;
if (schema instanceof CatalogManagerSchema) {
- CatalogSchema catalogSchema = ((CatalogManagerSchema)
schema).getCatalogSchema(pathOverride);
+ CatalogSchema catalogSchema =
+ pathOverride.catalog() != null
Review Comment:
ditto - and I do think doing it in the parse layer is the right way to go.
But we should probably be able to make a common "catalog reference" and "table
reference" construct.
##########
sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl:
##########
@@ -456,6 +505,24 @@ SqlDrop SqlDropTable(Span s, boolean replace) :
}
}
+/**
+ * SHOW TABLES [ ( FROM | IN )? [ catalog_name '.' ] database_name ] [ LIKE
regex_pattern ]
Review Comment:
Well.. we've have tables named like `bigquery.my_dataset.my_tablename` and
`pubsub.my_topic` for a long time. Ideally we can merge these two things
soonish since they have identical syntax and meaning.
##########
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java:
##########
@@ -159,7 +159,10 @@ public void execute(CalcitePrepare.Context context) {
CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema)
schema;
catalogManagerSchema.maybeRegisterProvider(pathOverride,
SqlDdlNodes.getString(type));
- CatalogSchema catalogSchema =
catalogManagerSchema.getCatalogSchema(pathOverride);
+ CatalogSchema catalogSchema =
+ pathOverride.catalog() != null
Review Comment:
See - here you are resolving it in the parser, unlike my other comment.
Might want to make a hard call whether it is a syntactic sugar or a thing in
the semantic layer. That'll make future dev work easier, to know that "at stage
X of analysis it is resolved and I don't need to check for this"
##########
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java:
##########
@@ -142,7 +145,8 @@ public void dropCatalog(SqlIdentifier identifier, boolean
ifExists) {
// will attempt to do so.
public void maybeRegisterProvider(TableName path, String type) {
type = type.toLowerCase();
- CatalogSchema catalogSchema = getCatalogSchema(path);
+ CatalogSchema catalogSchema =
+ path.catalog() != null ? getCatalogSchema(path) :
getCurrentCatalogSchema();
Review Comment:
FWIW you could also resolve this earlier in the parsing/analysis phases so
that at this level of implementation it is dead simple with no conditionals.
##########
sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl:
##########
@@ -331,6 +372,98 @@ SqlDrop SqlDropDatabase(Span s, boolean replace) :
}
}
+/**
+ * SHOW DATABASES [ ( FROM | IN )? catalog_name ] [LIKE regex_pattern ]
+ */
+SqlCall SqlShowDatabases(Span s) :
+{
+ SqlIdentifier catalogName = null;
+ SqlNode regex = null;
+}
+{
+ <SHOW> <DATABASES> { s.add(this); }
+ [ ( <FROM> | <IN> ) catalogName = SimpleIdentifier() ]
+ [ <LIKE> regex = StringLiteral() ]
+ {
+ List<String> path = new ArrayList<String>();
+ path.add("beamsystem");
+ path.add("databases");
+ SqlNodeList selectList =
SqlNodeList.of(SqlIdentifier.star(s.end(this)));
+ SqlNode where = null;
+ if (regex != null) {
+ SqlIdentifier nameIdentifier = new SqlIdentifier("NAME",
s.end(this));
+ where = SqlStdOperatorTable.LIKE.createCall(
+ s.end(this),
+ nameIdentifier, regex);
+ }
+ if (catalogName != null) {
+ path.add(catalogName.getSimple());
+ } else {
+ path.add("__current_catalog__");
+ }
+ SqlIdentifier from = new SqlIdentifier(path, s.end(this));
+
+ return new SqlSelect(
+ s.end(this),
+ null,
+ selectList,
+ from,
+ where,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null);
+ }
+}
+
+/**
+ * SHOW CURRENT ( CATALOG | DATABASE )
+ */
+SqlCall SqlShowCurrent(Span s) :
+{
+}
+{
+ <SHOW> <CURRENT> { s.add(this); }
+ {
+ List<String> path = new ArrayList<String>();
+ path.add("beamsystem");
Review Comment:
since this is a weird magic string, maybe also `__beamsystem__`? But also
yea we could leave as `beamsystem` and that could eventually be something we
let users query for more flexible processing of the contents. I know most
RDBMSes do that kind of thing, yea?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]