Skip to content

Commit

Permalink
rebase on enhanced create catalog
Browse files Browse the repository at this point in the history
  • Loading branch information
liyubin117 committed Jun 14, 2024
1 parent ec1f793 commit b6aea5a
Show file tree
Hide file tree
Showing 3 changed files with 16 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ SqlAlterCatalog SqlAlterCatalog() :
SqlParserPos startPos;
SqlIdentifier catalogName;
SqlNodeList propertyList = SqlNodeList.EMPTY;
SqlCharStringLiteral comment = null;
SqlNode comment = null;
}
{
<ALTER> <CATALOG> { startPos = getPos(); }
Expand All @@ -188,13 +188,12 @@ SqlAlterCatalog SqlAlterCatalog() :
propertyList);
}
|
<COMMENT> <QUOTED_STRING>
<COMMENT>
comment = StringLiteral()
{
String p = SqlParserUtil.parseString(token.image);
comment = SqlLiteral.createCharString(p, getPos());
return new SqlAlterCatalogComment(startPos.plus(getPos()),
catalogName,
comment);
catalogName,
comment);
}
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,11 @@

package org.apache.flink.sql.parser.ddl;

import org.apache.calcite.sql.SqlCharStringLiteral;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlWriter;
import org.apache.calcite.sql.parser.SqlParserPos;
import org.apache.calcite.util.ImmutableNullableList;
import org.apache.calcite.util.NlsString;

import java.util.List;

Expand All @@ -33,10 +31,10 @@
/** ALTER CATALOG catalog_name COMMENT 'comment'. */
public class SqlAlterCatalogComment extends SqlAlterCatalog {

private final SqlCharStringLiteral comment;
private final SqlNode comment;

public SqlAlterCatalogComment(
SqlParserPos position, SqlIdentifier catalogName, SqlCharStringLiteral comment) {
SqlParserPos position, SqlIdentifier catalogName, SqlNode comment) {
super(position, catalogName);
this.comment = requireNonNull(comment, "comment cannot be null");
}
Expand All @@ -46,8 +44,8 @@ public List<SqlNode> getOperandList() {
return ImmutableNullableList.of(catalogName, comment);
}

public String getComment() {
return comment.getValueAs(NlsString.class).getValue();
public SqlNode getComment() {
return comment;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,19 @@
import org.apache.flink.table.operations.Operation;
import org.apache.flink.table.operations.ddl.AlterCatalogCommentOperation;

import org.apache.calcite.sql.SqlCharStringLiteral;
import org.apache.calcite.util.NlsString;

/** A converter for {@link SqlAlterCatalogComment}. */
public class SqlAlterCatalogCommentConverter implements SqlNodeConverter<SqlAlterCatalogComment> {

@Override
public Operation convertSqlNode(
SqlAlterCatalogComment sqlAlterCatalogComment, ConvertContext context) {
return new AlterCatalogCommentOperation(
sqlAlterCatalogComment.catalogName(), sqlAlterCatalogComment.getComment());
sqlAlterCatalogComment.catalogName(),
((SqlCharStringLiteral) sqlAlterCatalogComment.getComment())
.getValueAs(NlsString.class)
.getValue());
}
}

0 comments on commit b6aea5a

Please sign in to comment.