Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

Discussion threadhttps://lists.apache.org/thread/54fyd27m8on1cf3hn6dz564zqmkobjyd
Vote threadhttps://lists.apache.org/thread/bc0qcmg73t4q7do3k657rqcdx1vyjqzy
JIRA

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyFLINK-31256

Release

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

...

But these features are not very complete compared with other popular engines such as spark, hive, presto and commercial engines such as snowflake.

For example, many popular engines support show operation with filtering except flink, and support describe other objectmany objects(flink only support supports describe table).

show

Support or Not

Support filter or Not

show tables

Yes

Yes

show columns

Yes

Yes

show catalogs

Yes

No

show databases

Yes

No

show functions

Yes

No

show views

Yes

No

show modules

Yes

No

show jars

Yes

No

show jobs

Yes

No

show connectors

No

No

We can see current flink many sql statements only support showing with full datas, without 'FROM/IN' or 'LIKE' filter clause.

...

describe

...

Support or Not

...

note

...

describe catalog

...

No

...

has sqlNode but not support in table api

...

describe database

...

No

...

has sqlNode but not support in table api

And these improved features are very useful for users and developers.

Comparison with other popular engines


Because each engine has its own personalized auxiliary sql statements features, here are some common operations listed as much as possible, and compare what other unrealized abilities of flink.

We focus on standard sql, but also consider absorbing the syntax of some mature engines to let users know that this is not standard, just for better use (e.g. ILIKE).


show catalogs

Engine

Support or Not, If support show the syntax

Flink

SHOW CATALOGS

Spark

No

Hive

No

Presto

SHOW CATALOGS [ LIKE pattern ]

MySQL

No

SnowFlake

No

 

show databases

Engine

Support or Not, If support show the syntax

Flink

SHOW DATABASES

Spark

SHOW { DATABASES | SCHEMAS } [ LIKE regex_pattern ]

Hive

SHOW (DATABASES|SCHEMAS) [LIKE 'identifier_with_wildcards']

Presto

SHOW SCHEMAS [ FROM catalog ] [ LIKE pattern ]

MySQL

SHOW {DATABASES | SCHEMAS} [LIKE 'pattern' | WHERE expr]

SnowFlake

SHOW [ TERSE ] DATABASES [ HISTORY ] [ LIKE '<pattern>' ]

                                     [ STARTS WITH '<name_string>' ]

                                     [ LIMIT <rows> [ FROM '<name_string>' ] ]



show functions

Engine

Support or Not, If support show the syntax

Flink

SHOW [USER] FUNCTIONS

Spark

SHOW [ function_kind ] FUNCTIONS [ { FROM | IN } database_name ] [ LIKE regex_pattern ]

Hive

SHOW FUNCTIONS [LIKE "<pattern>"]

Presto

SHOW FUNCTIONS [ LIKE pattern [ ESCAPE 'escape_character' ] ]

MySQL

SELECT ROUTINE_NAME FROM INFORMATION_SCHEMA.ROUTINES WHERE ROUTINE_TYPE = 'FUNCTION' ORDER BY ROUTINE_NAME;

SnowFlake

SHOW FUNCTIONS [ LIKE '<pattern>' ]

               [ IN

                    {

                      ACCOUNT                  |

                      DATABASE                 |

                      DATABASE <database_name> |

                      SCHEMA                   |

                      SCHEMA <schema_name>     |

                      <schema_name>

                    }

               ]


show views

Engine

Support or Not, If support show the syntax

Flink

SHOW VIEWS

Spark

SHOW VIEWS [ { FROM | IN } database_name ] [ LIKE regex_pattern ]

Hive

SHOW VIEWS [IN/FROM database_name] [LIKE 'pattern_with_wildcards']

Presto

No

MySQL

SHOW FULL TABLES [FROM db_name] [LIKE 'pattern' | WHERE expr] (Alternative syntax)

SnowFlake

SHOW [ TERSE ] VIEWS [ LIKE '<pattern>' ]

                     [ IN { ACCOUNT | DATABASE [ <db_name> ] | [ SCHEMA ] [ <schema_name> ] } ]

                     [ STARTS WITH '<name_string>' ]

                     [ LIMIT <rows> [ FROM '<name_string>' ] ]


show modules

Engine

Support or Not, If support show the syntax

Flink

SHOW [FULL] MODULES

Spark

No

Hive

No

Presto

No

MySQL

No

SnowFlake

No


show jars

Engine

Support or Not, If support show the syntax

Note

Flink

SHOW JARS

only work in SQL CLI or SQL Gateway.

Spark

No


Hive

No


Presto

No


MySQL

No


SnowFlake

No




show jobs

Engine

Support or Not, If support show the syntax

Note

Flink

SHOW JOBS

...

describe table

...

Yes

...

describe function

...

No

...

describe view

...

No

current flink only supports describing tables.

So we propose this flip, try to support these useful features.

Proposed Syntax Changes:

We compare flink with other popular engines and give an improved syntax example. Welcome everyone to discuss and improve the final syntax. 

Because it may be modified under discuss, we put it on the google docs. please see FLIP-297: Improve Auxiliary Sql Statements Docs

Note: we both support LIKE and ILIKE in this FLIP.

LIKE: sql_like_pattern, case sensitive

ILIKE: sql_like_pattern, same behavior as LIKE except case insensitive

SHOW CATALOGS[ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

before

after(under discussed)

Note

show catalogs

SHOW CATALOGS

show databases

SHOW DATABASES

SHOW DATABASES [ ( FROM | IN ) catalog_name] [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

show tables

SHOW TABLES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] LIKE <sql_like_pattern> ]

SHOW TABLES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

show columns

SHOW COLUMNS ( FROM | IN ) [[catalog_name.]database.]<table_name> [ [NOT] LIKE <sql_like_pattern>]

SHOW COLUMNS ( FROM | IN ) [[catalog_name.]database.]<table_name> [ [NOT] (LIKE | ILIKE) <sql_like_pattern>]

show functions

SHOW [USER] FUNCTIONS

SHOW [USER] FUNCTIONS [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

show views

SHOW VIEWS

SHOW VIEWS [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

show modules

SHOW [FULL] MODULES

SHOW [FULL] MODULES [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

show jars

SHOW JARS

SHOW JARS [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

only work in SQL CLI or SQL Gateway.

show jobs

SHOW JOBS

SHOW JOBS [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

only work in SQL CLI or SQL Gateway.

describe catalog

Spark

Not Support

{ DESCRIBE | DESC } CATALOG catalog_name

describe database

Not Support

{ DESCRIBE | DESC } DATABASE [ EXTENDED ] db_name

describe function

Not Support

{ DESCRIBE | DESC } FUNCTION [ EXTENDED ] function_name

...

Proposed CataLog API Changes:

No need to add new apis (there is a discussion about api changes, pls see the discuss thread). 

We can use such as TableEnvironment.getCatalog(catalogName).get().listDatabases(), TableEnvironment.getCatalog(catalogName).get().listFunctions(databaseName) for util to get result.

In TableEnvironmentImpl we do filter for returned full results.

Some SqlNodes and Operations could be added or changed.

No


Hive

No


Presto

No


MySQL

No


SnowFlake

No




show create database

Engine

Support or Not, If support show the syntax

Note

Flink

No


Spark

No


Hive

No


Presto

No


MySQL

SHOW CREATE {DATABASE | SCHEMA} [IF NOT EXISTS] db_name

RDBMS, such as Oracle, MariaDB, TiDB also support it.

SnowFlake

No




show create function

Engine

Support or Not, If support show the syntaxes

Note

Flink

No


Spark

No


Hive

No


Presto

SHOW CREATE FUNCTION function_name [ ( parameter_type[, ...] ) ]


MySQL

SHOW CREATE FUNCTION func_name

RDBMS, such as Oracle, MariaDB, TiDB also support it.

SnowFlake

No



describe catalog

Engine

Support or Not, If support show the syntaxes

Note

Flink

No

with catalog-database-table three layer

Spark

No

with database-table layer. schema and database they mean the same thing.

But databricks on aws support it:
{ DESC | DESCRIBE } CATALOG [ EXTENDED ] catalog_name

Hive

No

with database-table layer. schema and database they mean the same thing

Presto

No

Presto is schema-database-table three layer. schema and database are not equal.

MySQL

No

with database-table layer. schema and database they mean the same thing.

SnowFlake

DESC[RIBE] DATABASE <database_name>

SnowFlake is database-schema-table three layer.

database may act as catalog.


describe database

Engine

Support or Not, If support show the syntaxes

Note

Flink

{ DESC | DESCRIBE } DATABASE [ EXTENDED ] db_name

just sqlNode, but not expose to table & sql api

Spark

{ DESC | DESCRIBE } DATABASE [ EXTENDED ] db_name


Hive

DESCRIBE DATABASE [EXTENDED] db_name;

DESCRIBE SCHEMA [EXTENDED] db_name;


Presto

No


MySQL

No


SnowFlake

DESC[RIBE] SCHEMA <schema_name>



describe function

Engine

Support or Not, If support show the syntaxes

Note

Flink

No


Spark

{ DESC | DESCRIBE } FUNCTION [ EXTENDED ] function_name


Hive

DESCRIBEFUNCTION [EXTENDED] <function_name>


Presto

No


MySQL

No


SnowFlake

DESC[RIBE] FUNCTION <name> ( [ <arg_data_type> ] [ , ... ] )



describe view

Engine

Support or Not, If support show the syntaxes

Note

Flink

No


Spark

No


Hive

DESCRIBE [EXTENDED|FORMATTED] <view_name>;


Presto

No


MySQL

No


SnowFlake

DESC[RIBE] VIEW <name>



We can see many popular engines have filtering with show statements, support 'FROM/IN' or 'LIKE/ILIKE' filter clause except flink.
And currently flink only supports describing tables.
And also some unique statements such as show jars/show modules in flink can also support this feature.

So we proposed this FLIP.

Proposed Syntax Changes:

Note: we both support LIKE and ILIKE in this FLIP.

LIKE: sql_like_pattern, case sensitive

ILIKE: sql_like_pattern, same behavior as LIKE except case insensitive

And For consistency, we will add ILIKE support for query either.

Because calcite already support parse ILIKE and exist SqlLikeOperator(SqlLibraryOperators.ILIKE, not std operator). We just need to support it in table & sql api (no need to modify sql keywords and Parser.jj).



before

after(under discussed)

Note

show catalogs

SHOW CATALOGS

SHOW CATALOGS[ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]


show databases

SHOW DATABASES

SHOW DATABASES [ ( FROM | IN ) catalog_name] [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]


show tables

SHOW TABLES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] LIKE <sql_like_pattern> ]

SHOW TABLES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

show columns

SHOW COLUMNS ( FROM | IN ) [[catalog_name.]database.]<table_name> [ [NOT] LIKE <sql_like_pattern>]

SHOW COLUMNS ( FROM | IN ) [[catalog_name.]database.]<table_name> [ [NOT] (LIKE | ILIKE) <sql_like_pattern>]

show functions

SHOW [USER] FUNCTIONS

SHOW [USER] FUNCTIONS [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]


show views

SHOW VIEWS

SHOW VIEWS [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]


show modules

SHOW [FULL] MODULES

SHOW [FULL] MODULES [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]


show jars

SHOW JARS

SHOW JARS [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

only work in SQL CLI or SQL Gateway.

show jobs

SHOW JOBS

SHOW JOBS [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]

only work in SQL CLI or SQL Gateway.

describe catalog

Not Support

{ DESCRIBE | DESC } CATALOG [ EXTENDED ] catalog_name

If the optional EXTENDED option is specified, it returns the basic metadata information along with the properties


describe database

Not Support

{ DESCRIBE | DESC } DATABASE [ EXTENDED ] [catalog_name.]database_name

If the optional EXTENDED option is specified, it returns the basic metadata information along with the properties like spark and hive. 


describe function

Not Support

{ DESCRIBE | DESC } FUNCTION [ EXTENDED ] [[catalog_name.]database_name.]function_name

If the optional EXTENDED option is specified, the basic metadata information is returned along with the extended information.


...





Proposed CataLog API Changes:

No need to add new apis (there is a discussion about api changes, pls see the discuss thread). 

We can use such as TableEnvironment.getCatalog(catalogName).get().listDatabases(), TableEnvironment.getCatalog(catalogName).get().listFunctions(databaseName) for util to get result.

In TableEnvironmentImpl we do filter for returned full results.


Some SqlNodes and Operations could be added or changed.

Code Block
languagejava
titleOperations
linenumberstrue
// Add SqlDescribeCatalog & DescribeCatalogOperation
// Add SqlDescribeDatabase & DescribeDatabaseOperation
// Add SqlDescribeFunction & DescribeFunctionOperation

About DescribeCatalog:

+-------------------------+-----------------------------+

| catalog_description_item|   catalog_description_value |
+-------------------------+-----------------------------+
|            Catalog Name |                       xxx   |
|    
Code Block
languagejava
titleOperations
linenumberstrue
// Add SqlDescribeCatalog & DescribeCatalogOperation
// Add SqlDescribeDatabase & DescribeDatabaseOperation
// Add SqlDescribeFunction & DescribeFunctionOperation

// Changed SqlNodes & Operations

// SqlShowCatalogs 
public class SqlShowCatalogs extends SqlCall {

    public static final SqlSpecialOperator OPERATOR =
            new SqlSpecialOperator("SHOW CATALOGS", SqlKind.OTHER);

    protected final SqlLikeType likeType;
    protected final boolean notLike;
    protected final SqlCharStringLiteral likeLiteral;

    public SqlShowCatalogs(SqlParserPos pos) {
        super(pos);
        this.likeType = null;
        this.notLike = false;
        this.likeLiteral = null;
    }

    public SqlShowCatalogs(
            SqlParserPos pos, String likeType, boolean notLike, SqlCharStringLiteral likeLiteral) {
        super(pos);
        if (likeType != null) {
            this.likeType = SqlLikeType.of(likeType);
            this.likeLiteral = requireNonNull(likeLiteral, "Like pattern must not be null");
        } else {
            this.likeType = null;
Properties| (key1=val1, key2=val2, ...) |
+-------------------------+-----------------------------+


About DescribeDatabase:

+-------------------------+-----------------------------+
|database_description_item|   database_description_value|
+-------------------------+-----------------------------+
|            this.likeLiteral = null;
Database Name|         }
        this.notLike = notLike;
    }employees|

|    public SqlLikeType getLikeType() {
       Description| return likeType;
    }

 For   public boolean isLike() {
    software companies|
|    return likeType == SqlLikeType.LIKE;
    }

    public booleanProperties| isILike() {
        return likeType == SqlLikeType.ILIKE;
    }

    public boolean isWithLike() {
        return isLike() || isILike();
    }

    public boolean isNotLike() {
        return notLike;
    }

    public String getLikeSqlPattern() {
        return Objects.isNull(likeLiteral) ? null : likeLiteral.getValueAs(String.class);
    }

    @Override
    public SqlOperator getOperator() {
        return OPERATOR;
    }

    @Override
    public List<SqlNode> getOperandList() {
        return Collections.emptyList();
    }

    @Override
    public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
        writer.keyword("SHOW CATALOGS");
(key1=val1, key2=val2, ...) |
+-------------------------+-----------------------------+

About DescribeFunction:

+-------------------------+-----------------------------+
|function_description_item|   function_description_value|
+-------------------------+-----------------------------+
|        Function Name    |                    ABS      |
|        Function Language|                    JAVA     |
|        Resource Uri     |                  file://xxx |
|        ifDetails (isWithLike()) {
        |    if (isNotLike()) {
          details      writer.keyword(String.format("NOT %s '%s'", likeType.name(), getLikeSqlPattern()));
            } else {
                writer.keyword(String.format("%s '%s'", likeType.name(), getLikeSqlPattern()));
            }
        }
    }
}

// ShowCatalogsOperation
public class ShowCatalogsOperation implements ShowOperation {|
+-------------------------+-----------------------------+

The concrete describe infos we can get from the specific catalog object.  

// Changed SqlNodes & Operations

// SqlShowCatalogs 
public class SqlShowCatalogs extends SqlCall {

    public static final SqlSpecialOperator OPERATOR =
            new SqlSpecialOperator("SHOW CATALOGS", SqlKind.OTHER);

    privateprotected final OperationLikeTypeSqlLikeType likeType;
    privateprotected final boolean notLike;
    privateprotected final StringSqlCharStringLiteral likePatternlikeLiteral;

    /** Use when there ispublic no sub-clause. */SqlShowCatalogs(SqlParserPos pos) {
    public    ShowCatalogsOperationsuper(pos) {;
        this.likeType = null;
        this.notLike = false;
        this.likePatternlikeLiteral = null;
    }

    public SqlShowCatalogs(
            SqlParserPos pos, String likeType,  /** Use when there is like. */boolean notLike, SqlCharStringLiteral likeLiteral) {
        super(pos);
    public  ShowCatalogsOperation(String likeType, booleanif notLike, String likePattern(likeType != null) {
            this.likeType = OperationLikeTypeSqlLikeType.of(checkNotNull(likeType,);
 "Like type must not be null"));
        this.likePatternlikeLiteral = checkNotNullrequireNonNull(likePatternlikeLiteral, "Like pattern must not be null");
        this.notLike = notLike;
    }

    public boolean isLike() {
        } else return{
 likeType == OperationLikeType.LIKE;
    }

    public boolean isIlike() {this.likeType = null;
        return likeType    this.likeLiteral == OperationLikeType.ILIKEnull;
    }

    public boolean isWithLike() {}
        return isLike() || isIlike()this.notLike = notLike;
    }

    public booleanSqlLikeType isNotLikegetLikeType() {
        return notLikelikeType;
    }

    public Stringboolean getLikePatternisLike() {
        return likePattern return likeType == SqlLikeType.LIKE;
    }

    @Override
    public Stringboolean asSummaryStringisILike() {
        StringBuilderreturn builderlikeType == new StringBuilder().append("SHOW CATALOGS")SqlLikeType.ILIKE;
    }

    public ifboolean (isWithLike()) {
        return isLike() ||  if isILike(notLike) {;
    }

            builder.append(String.format(" NOT %s %s", likeType.name(), likePattern));public boolean isNotLike() {
        return notLike;
    }

 else   {
public String getLikeSqlPattern() {
        return Objects.isNull(likeLiteral) ? null : builderlikeLiteral.appendgetValueAs(String.format(" %s %s", likeType.name(), likePattern));
class);
    }

    @Override
    public SqlOperator getOperator() {
        return OPERATOR;
    }

    @Override
    public List<SqlNode> getOperandList() }{
        return builderCollections.toStringemptyList();
    }
}

// SqlShowDatabases
public class SqlShowDatabases extends SqlCall {@Override

    public staticvoid final SqlSpecialOperator OPERATOR =
   unparse(SqlWriter writer, int leftPrec, int rightPrec) {
         new SqlSpecialOperatorwriter.keyword("SHOW DATABASES", SqlKind.OTHER);
CATALOGS");
        if (isWithLike()) {
    protected final String preposition;
    protected final SqlIdentifier catalogName;if (isNotLike()) {
    protected final SqlLikeType likeType;
    protected final boolean notLike            writer.keyword(String.format("NOT %s '%s'", likeType.name(), getLikeSqlPattern()));
    protected final SqlCharStringLiteral likeLiteral;

    public SqlShowDatabases(SqlParserPos pos)} else {
        super(pos);
          this.preposition = nullwriter.keyword(String.format("%s '%s'", likeType.name(), getLikeSqlPattern()));
        this.catalogName = null;
  }
      this.likeType = null;}
    }
}

// ShowCatalogsOperation
public class ShowCatalogsOperation this.notLikeimplements =ShowOperation false;{

    private final OperationLikeType likeType;
 this.likeLiteral = null;
 private final boolean }notLike;

    publicprivate SqlShowDatabases(
final String likePattern;

    /** Use when there is no SqlParserPos pos,sub-clause. */
    public ShowCatalogsOperation() {
      String preposition,
 this.likeType = null;
        this.notLike SqlIdentifier= catalogName,false;
        this.likePattern    String likeType,= null;
    }

    /** Use   boolean notLike,
   when there is like. */
    public ShowCatalogsOperation(String likeType, boolean notLike, SqlCharStringLiteralString likeLiterallikePattern) {
        super(pos);
        this.preposition = prepositionthis.likeType = OperationLikeType.of(checkNotNull(likeType, "Like type must not be null"));
        this.catalogNamelikePattern =
 checkNotNull(likePattern, "Like pattern must not be null");
        this.notLike preposition != nullnotLike;
    }

    public boolean isLike() {
        return likeType == OperationLikeType.LIKE;
    ? requireNonNull(catalogName, "Catalog name must not be null.")}

    public boolean isIlike() {
        return likeType == OperationLikeType.ILIKE;
    }

    public boolean    : null;isWithLike() {
        ifreturn isLike(likeType) != null) {|| isIlike();
       }

    public this.likeType = SqlLikeType.of(likeType);boolean isNotLike() {
        return notLike;
   this.likeLiteral = requireNonNull(likeLiteral, "Like pattern must not be null"); }

    public String getLikePattern() {
        }return elselikePattern;
 {
   }

    @Override
    public this.likeType = null;String asSummaryString() {
        StringBuilder builder = new this.likeLiteral = nullStringBuilder().append("SHOW CATALOGS");
        }
  if (isWithLike()) {
      this.notLike = notLike;
    }

    public String getPreposition(if (notLike) {
        return  preposition;
    }

  builder.append(String.format(" NOT public%s SqlIdentifier getCatalogName%s", likeType.name(), {likePattern));
         return catalogName;
  } else }{

    public String catalogName() {
        return catalogName != null ? catalogName.getSimple() : null builder.append(String.format(" %s %s", likeType.name(), likePattern));
    }

     public SqlLikeType getLikeType() {}
        return likeType;}
    }

    public boolean isLikereturn builder.toString() {;
    }
}

// SqlShowDatabases
public   return likeType == SqlLikeType.LIKE;
    }class SqlShowDatabases extends SqlCall {

    public boolean isILike() {static final SqlSpecialOperator OPERATOR =
        return likeType == SqlLikeType.ILIKE;
 new SqlSpecialOperator("SHOW  }DATABASES", SqlKind.OTHER);

    publicprotected booleanfinal isNotLike() {String preposition;
    protected final SqlIdentifier  return notLikecatalogName;
    }

protected final   public boolean isWithLike() {SqlLikeType likeType;
        return isLike() || isILike()protected final boolean notLike;
    protected final SqlCharStringLiteral }likeLiteral;

    public SqlCharStringLiteral getLikeLiteral(SqlShowDatabases(SqlParserPos pos) {
        return likeLiteralsuper(pos);
    }

    publicthis.preposition String getLikeSqlPattern() {= null;
        return Objects.isNull(likeLiteral) ? null : likeLiteral.getValueAs(String.class);
this.catalogName = null;
      }

  this.likeType = @Overridenull;
      public  SqlOperator getOperator() {this.notLike = false;
        this.likeLiteral return= OPERATORnull;
    }

    @Overridepublic SqlShowDatabases(
    public List<SqlNode> getOperandList() {
     SqlParserPos   return Objects.isNull(this.catalogName)
pos,
            String preposition,
    ? Collections.emptyList()
       SqlIdentifier catalogName,
        : Collections.singletonList(catalogName);
   String }likeType,

    @Override
    public void unparse(SqlWriter writer, intboolean leftPrecnotLike,
 int rightPrec) {
        if (this.preposition == nullSqlCharStringLiteral likeLiteral) {
        super(pos);
      writer.keyword("SHOW DATABASES")  this.preposition = preposition;
        } else if (this.catalogName !=
 null) {
            writer.keyword("SHOW DATABASES "preposition + this.preposition);!= null
            catalogName.unparse(writer, leftPrec, rightPrec);
        }
   ? requireNonNull(catalogName, "Catalog name must not if (isWithLike()) {
be null.")
             if (isNotLike()) {
         : null;
      writer.keyword(String.format("NOT %s '%s'",if likeType.name(), getLikeSqlPattern()));(likeType != null) {
            this.likeType = SqlLikeType.of(likeType);
  } else {
        this.likeLiteral = requireNonNull(likeLiteral, "Like pattern must not  writer.keyword(String.format("%s '%s'", likeType.name(), getLikeSqlPattern()))be null");
        } else   }{
        }
    }

// ShowDatabasesOperation
public class ShowDatabasesOperation implements ShowOperation {

this.likeType = null;
    private final String preposition;
    private finalthis.likeLiteral String= catalogNamenull;
    private  final boolean withLike;}
    private final boolean    this.notLike = notLike;
    private final String likePattern;}

    public ShowDatabasesOperationString getPreposition() {
        this.preposition = null;return preposition;
    }

    public SqlIdentifier getCatalogName() {
 this.catalogName = null;
     return catalogName;
  this.withLike = false;
    }

    public String catalogName() {
    this.notLike = false;
  return catalogName != null ?  this.likePattern =catalogName.getSimple() : null;
    }

    public SqlLikeType ShowDatabasesOperation(boolean notLike, String likePatterngetLikeType() {
        this.preposition = nullreturn likeType;
    }

    public this.catalogName = null;boolean isLike() {
        this.withLikereturn likeType = likePattern != nullSqlLikeType.LIKE;
    }

    this.notLike = notLike;public boolean isILike() {
        this.likePattern return likeType == likePatternSqlLikeType.ILIKE;
    }

    public ShowDatabasesOperation(boolean isNotLike() {
        return notLike;
   String preposition,}

 String catalogName, boolean notLike,public Stringboolean likePatternisWithLike() {
        this.preposition = checkNotNull(preposition, "Preposition must not be null"return isLike() || isILike();
        this.catalogName = checkNotNull(catalogName, "Catalog name must not be null");}

    public SqlCharStringLiteral getLikeLiteral() {
        this.withLike = likePattern != null;return likeLiteral;
    }

    public String getLikeSqlPattern() {
 this.notLike = notLike;
     return Objects.isNull(likeLiteral) ? this.likePatternnull = likePattern: likeLiteral.getValueAs(String.class);
    }

    @Override
    public StringSqlOperator getPrepositiongetOperator() {
        return prepositionOPERATOR;
    }

    @Override
    public StringList<SqlNode> getCatalogNamegetOperandList() {
        return catalogName;Objects.isNull(this.catalogName)
    }

    public boolean isWithLike() {
        return withLike;
    }

? Collections.emptyList()
     public boolean isNotLike() {
        return notLike: Collections.singletonList(catalogName);
    }

    public String getLikePattern() {@Override
    public void unparse(SqlWriter writer, return likePattern;
    }
int leftPrec, int rightPrec) {
    @Override
    if public String asSummaryString((this.preposition == null) {
        StringBuilder builder = new StringBuilder()writer.appendkeyword("SHOW DATABASES");
        } else if (prepositioncatalogName != null) {
            builder.append(String.formatwriter.keyword("SHOW %sDATABASES %s", preposition, catalogName))" + this.preposition);
            catalogName.unparse(writer, leftPrec, rightPrec);
        }
        if (withLikeisWithLike()) {
            if (isNotLike(notLike)) {
                builderwriter.appendkeyword(String.format("NOT %s LIKE '%s'", "NOT"likeType.name(), likePatterngetLikeSqlPattern()));
            } else {
                builderwriter.appendkeyword(String.format("%s LIKE '%s'", likePatternlikeType.name(), getLikeSqlPattern()));
            }
        }
        return builder.toString();
    }
}
// ShowDatabasesOperation
public class ShowDatabasesOperation implements ShowOperation {

    private final String preposition;
    private final String catalogName;
    private final OperationLikeType likeType;
    private final boolean notLike;
    private final String likePattern;

    /** Use when there is no sub-clause. */
    public ShowDatabasesOperation() {
        this.preposition = null;
        this.catalogName = null;
        this.likeType = null;
        this.notLike = false;
        this.likePattern = null;
    }

    /** Use when there is only like. */
    public ShowDatabasesOperation(String likeType, boolean notLike, String likePattern) {
        this.preposition = null;
        this.catalogName = null;
        this.likeType = OperationLikeType.of(checkNotNull(likeType, "Like type must not be null"));
        this.likePattern = checkNotNull(likePattern, "Like pattern must not be null");
        this.notLike = notLike;
    }

    /** Use when there is preposition and like. */
    public ShowDatabasesOperation(
            String preposition,
            String catalogName,
            String likeType,
            boolean notLike,
            String likePattern) {
        this.preposition = checkNotNull(preposition, "Preposition must not be null");
        this.catalogName = checkNotNull(catalogName, "Catalog name must not be null");
        if (likeType != null) {
            this.likeType = OperationLikeType.of(likeType);
            this.likePattern = checkNotNull(likePattern, "Like pattern must not be null");
        } else {
            this.likeType = null;
            this.likePattern = null;
        }
        this.notLike = notLike;
    }

    public String getPreposition() {
        return preposition;
    }

    public String getCatalogName() {
        return catalogName;
    }

    public boolean isLike() {
        return likeType == OperationLikeType.LIKE;
    }

    public boolean isIlike() {
        return likeType == OperationLikeType.ILIKE;
    }

    public boolean isWithLike() {
        return isLike() || isIlike();
    }

    public boolean isNotLike() {
        return notLike;
    }

    public String getLikePattern() {
        return likePattern;
    }

    @Override
    public String asSummaryString() {
        StringBuilder builder = new StringBuilder().append("SHOW DATABASES");
        if (preposition != null) {
            builder.append(String.format(" %s %s", preposition, catalogName));
        }
        if (isWithLike()) {
            if (notLike) {
                builder.append(String.format(" NOT %s %s", likeType.name(), likePattern));
            } else {
                builder.append(String.format(" %s %s", likeType.name(), likePattern));
            }
        }
        return builder.toString();
    }
}

// omit here. basically same as ShowCatalogs/ShowDatabases above.
// SqlShowFunctions & ShowFunctionsOperation
// SqlShowViews & ShowViewsOperation
// SqlShowModules & ShowModulesOperation
// SqlShowJars & ShowJarsOperation
// SqlShowJobs & ShowJobsOperation

Future Work

to be addedTBD.

Compatibility, Deprecation, and Migration Plan

No compatibility and migrationmigration. We will ensure changes of this FLIP will be compatible with current flink behavior.

Test Plan

Add new cases to cover these features.

...