EXPLAIN Statements

EXPLAIN statements are used to explain the logical and optimized query plans of a query or an INSERT statement.

Run an EXPLAIN statement

Java

EXPLAIN statements can be executed with the executeSql() method of the TableEnvironment. The executeSql() method returns explain result for a successful EXPLAIN operation, otherwise will throw an exception.

The following examples show how to run an EXPLAIN statement in TableEnvironment.

Scala

EXPLAIN statements can be executed with the executeSql() method of the TableEnvironment. The executeSql() method returns explain result for a successful EXPLAIN operation, otherwise will throw an exception.

The following examples show how to run an EXPLAIN statement in TableEnvironment.

Python

EXPLAIN statements can be executed with the execute_sql() method of the TableEnvironment. The execute_sql() method returns explain result for a successful EXPLAIN operation, otherwise will throw an exception.

The following examples show how to run an EXPLAIN statement in TableEnvironment.

SQL CLI

EXPLAIN statements can be executed in SQL CLI.

The following examples show how to run an EXPLAIN statement in SQL CLI.

Java

  1. StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  2. StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
  3. // register a table named "Orders"
  4. tEnv.executeSql("CREATE TABLE MyTable1 (`count` bigint, word VARCHAR(256) WITH (...)");
  5. tEnv.executeSql("CREATE TABLE MyTable2 (`count` bigint, word VARCHAR(256) WITH (...)");
  6. // explain SELECT statement through TableEnvironment.explainSql()
  7. String explanation = tEnv.explainSql(
  8. "SELECT `count`, word FROM MyTable1 WHERE word LIKE 'F%' " +
  9. "UNION ALL " +
  10. "SELECT `count`, word FROM MyTable2");
  11. System.out.println(explanation);
  12. // explain SELECT statement through TableEnvironment.executeSql()
  13. TableResult tableResult = tEnv.executeSql(
  14. "EXPLAIN PLAN FOR " +
  15. "SELECT `count`, word FROM MyTable1 WHERE word LIKE 'F%' " +
  16. "UNION ALL " +
  17. "SELECT `count`, word FROM MyTable2");
  18. tableResult.print();

Scala

  1. val env = StreamExecutionEnvironment.getExecutionEnvironment()
  2. val tEnv = StreamTableEnvironment.create(env)
  3. // register a table named "Orders"
  4. tEnv.executeSql("CREATE TABLE MyTable1 (`count` bigint, word VARCHAR(256) WITH (...)")
  5. tEnv.executeSql("CREATE TABLE MyTable2 (`count` bigint, word VARCHAR(256) WITH (...)")
  6. // explain SELECT statement through TableEnvironment.explainSql()
  7. val explanation = tEnv.explainSql(
  8. "SELECT `count`, word FROM MyTable1 WHERE word LIKE 'F%' " +
  9. "UNION ALL " +
  10. "SELECT `count`, word FROM MyTable2")
  11. println(explanation)
  12. // explain SELECT statement through TableEnvironment.executeSql()
  13. val tableResult = tEnv.executeSql(
  14. "EXPLAIN PLAN FOR " +
  15. "SELECT `count`, word FROM MyTable1 WHERE word LIKE 'F%' " +
  16. "UNION ALL " +
  17. "SELECT `count`, word FROM MyTable2")
  18. tableResult.print()

Python

  1. settings = EnvironmentSettings.new_instance()...
  2. table_env = StreamTableEnvironment.create(env, settings)
  3. t_env.execute_sql("CREATE TABLE MyTable1 (`count` bigint, word VARCHAR(256) WITH (...)")
  4. t_env.execute_sql("CREATE TABLE MyTable2 (`count` bigint, word VARCHAR(256) WITH (...)")
  5. # explain SELECT statement through TableEnvironment.explain_sql()
  6. explanation1 = t_env.explain_sql(
  7. "SELECT `count`, word FROM MyTable1 WHERE word LIKE 'F%' "
  8. "UNION ALL "
  9. "SELECT `count`, word FROM MyTable2")
  10. print(explanation1)
  11. # explain SELECT statement through TableEnvironment.execute_sql()
  12. table_result = t_env.execute_sql(
  13. "EXPLAIN PLAN FOR "
  14. "SELECT `count`, word FROM MyTable1 WHERE word LIKE 'F%' "
  15. "UNION ALL "
  16. "SELECT `count`, word FROM MyTable2")
  17. table_result.print()

SQL CLI

  1. Flink SQL> CREATE TABLE MyTable1 (`count` bigint, word VARCHAR(256);
  2. [INFO] Table has been created.
  3. Flink SQL> CREATE TABLE MyTable2 (`count` bigint, word VARCHAR(256);
  4. [INFO] Table has been created.
  5. Flink SQL> EXPLAIN PLAN FOR SELECT `count`, word FROM MyTable1 WHERE word LIKE 'F%'
  6. > UNION ALL
  7. > SELECT `count`, word FROM MyTable2;

The EXPLAIN result is:

Blink Planner

  1. == Abstract Syntax Tree ==
  2. LogicalUnion(all=[true])
  3. LogicalFilter(condition=[LIKE($1, _UTF-16LE'F%')])
  4. LogicalTableScan(table=[[default_catalog, default_database, MyTable1]], fields=[count, word])
  5. LogicalTableScan(table=[[default_catalog, default_database, MyTable2]], fields=[count, word])
  6. == Optimized Physical Plan ==
  7. Union(all=[true], union all=[count, word])
  8. Calc(select=[count, word], where=[LIKE(word, _UTF-16LE'F%')])
  9. TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[count, word])
  10. TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[count, word])
  11. == Optimized Execution Plan ==
  12. Union(all=[true], union all=[count, word])
  13. Calc(select=[count, word], where=[LIKE(word, _UTF-16LE'F%')])
  14. TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[count, word])
  15. TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[count, word])

Legacy Planner

  1. == Abstract Syntax Tree ==
  2. LogicalUnion(all=[true])
  3. LogicalFilter(condition=[LIKE($1, _UTF-16LE'F%')])
  4. FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[count, word])
  5. FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[count, word])
  6. == Optimized Logical Plan ==
  7. DataStreamUnion(all=[true], union all=[count, word])
  8. DataStreamCalc(select=[count, word], where=[LIKE(word, _UTF-16LE'F%')])
  9. TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[count, word])
  10. TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[count, word])
  11. == Physical Execution Plan ==
  12. Stage 1 : Data Source
  13. content : collect elements with CollectionInputFormat
  14. Stage 2 : Data Source
  15. content : collect elements with CollectionInputFormat
  16. Stage 3 : Operator
  17. content : from: (count, word)
  18. ship_strategy : REBALANCE
  19. Stage 4 : Operator
  20. content : where: (LIKE(word, _UTF-16LE'F%')), select: (count, word)
  21. ship_strategy : FORWARD
  22. Stage 5 : Operator
  23. content : from: (count, word)
  24. ship_strategy : REBALANCE

Syntax

  1. EXPLAIN PLAN FOR <query_statement_or_insert_statement>

For query syntax, please refer to Queries page. For INSERT, please refer to INSERT page.