Skip to content

Commit 0270244

Browse files
committed
[FLINK-22341][hive] Fix describe table for hive dialect
This closes apache#15660
1 parent 04ce600 commit 0270244

File tree

2 files changed

+24
-1
lines changed

2 files changed

+24
-1
lines changed

flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveParserDDLSemanticAnalyzer.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1153,7 +1153,7 @@ private Serializable analyzeDescribeTable(HiveParserASTNode ast) throws Semantic
11531153
if (partSpec != null) {
11541154
handleUnsupportedOperation("DESCRIBE PARTITION is not supported");
11551155
}
1156-
if (colPath != null) {
1156+
if (!colPath.equals(tableName)) {
11571157
handleUnsupportedOperation("DESCRIBE COLUMNS is not supported");
11581158
}
11591159

flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,10 +28,12 @@
2828
import org.apache.flink.table.api.internal.TableEnvironmentInternal;
2929
import org.apache.flink.table.catalog.CatalogPartitionSpec;
3030
import org.apache.flink.table.catalog.CatalogTable;
31+
import org.apache.flink.table.catalog.ObjectIdentifier;
3132
import org.apache.flink.table.catalog.ObjectPath;
3233
import org.apache.flink.table.catalog.hive.HiveCatalog;
3334
import org.apache.flink.table.catalog.hive.HiveTestUtils;
3435
import org.apache.flink.table.delegation.Parser;
36+
import org.apache.flink.table.operations.DescribeTableOperation;
3537
import org.apache.flink.table.operations.command.ClearOperation;
3638
import org.apache.flink.table.operations.command.HelpOperation;
3739
import org.apache.flink.table.operations.command.QuitOperation;
@@ -255,6 +257,27 @@ public void testCreateTable() throws Exception {
255257
tableEnv.executeSql("create table if not exists tbl5 (m map<bigint,string>)");
256258
hiveTable = hiveCatalog.getHiveTable(new ObjectPath("default", "tbl5"));
257259
assertEquals(createdTimeForTableExists, hiveTable.getCreateTime());
260+
261+
// test describe table
262+
Parser parser = ((TableEnvironmentInternal) tableEnv).getParser();
263+
DescribeTableOperation operation =
264+
(DescribeTableOperation) parser.parse("desc tbl1").get(0);
265+
assertFalse(operation.isExtended());
266+
assertEquals(
267+
ObjectIdentifier.of(hiveCatalog.getName(), "default", "tbl1"),
268+
operation.getSqlIdentifier());
269+
270+
operation = (DescribeTableOperation) parser.parse("describe default.tbl2").get(0);
271+
assertFalse(operation.isExtended());
272+
assertEquals(
273+
ObjectIdentifier.of(hiveCatalog.getName(), "default", "tbl2"),
274+
operation.getSqlIdentifier());
275+
276+
operation = (DescribeTableOperation) parser.parse("describe extended tbl3").get(0);
277+
assertTrue(operation.isExtended());
278+
assertEquals(
279+
ObjectIdentifier.of(hiveCatalog.getName(), "default", "tbl3"),
280+
operation.getSqlIdentifier());
258281
}
259282

260283
@Test

0 commit comments

Comments
 (0)