diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveParserDDLSemanticAnalyzer.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveParserDDLSemanticAnalyzer.java index 7a89ae10254c6..e9f6ef5723484 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveParserDDLSemanticAnalyzer.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveParserDDLSemanticAnalyzer.java @@ -1153,7 +1153,7 @@ private Serializable analyzeDescribeTable(HiveParserASTNode ast) throws Semantic if (partSpec != null) { handleUnsupportedOperation("DESCRIBE PARTITION is not supported"); } - if (colPath != null) { + if (!colPath.equals(tableName)) { handleUnsupportedOperation("DESCRIBE COLUMNS is not supported"); } diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java index 0b56b1bb84823..43d1099d9a045 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java @@ -28,10 +28,12 @@ import org.apache.flink.table.api.internal.TableEnvironmentInternal; import org.apache.flink.table.catalog.CatalogPartitionSpec; import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.hive.HiveCatalog; import org.apache.flink.table.catalog.hive.HiveTestUtils; import org.apache.flink.table.delegation.Parser; +import org.apache.flink.table.operations.DescribeTableOperation; import org.apache.flink.table.operations.command.ClearOperation; import org.apache.flink.table.operations.command.HelpOperation; import org.apache.flink.table.operations.command.QuitOperation; @@ -255,6 +257,27 @@ public void testCreateTable() throws Exception { tableEnv.executeSql("create table if not exists tbl5 (m map)"); hiveTable = hiveCatalog.getHiveTable(new ObjectPath("default", "tbl5")); assertEquals(createdTimeForTableExists, hiveTable.getCreateTime()); + + // test describe table + Parser parser = ((TableEnvironmentInternal) tableEnv).getParser(); + DescribeTableOperation operation = + (DescribeTableOperation) parser.parse("desc tbl1").get(0); + assertFalse(operation.isExtended()); + assertEquals( + ObjectIdentifier.of(hiveCatalog.getName(), "default", "tbl1"), + operation.getSqlIdentifier()); + + operation = (DescribeTableOperation) parser.parse("describe default.tbl2").get(0); + assertFalse(operation.isExtended()); + assertEquals( + ObjectIdentifier.of(hiveCatalog.getName(), "default", "tbl2"), + operation.getSqlIdentifier()); + + operation = (DescribeTableOperation) parser.parse("describe extended tbl3").get(0); + assertTrue(operation.isExtended()); + assertEquals( + ObjectIdentifier.of(hiveCatalog.getName(), "default", "tbl3"), + operation.getSqlIdentifier()); } @Test