You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/05/11 01:30:01 UTC

[incubator-doris] branch master updated: [feature] show create materialized view (#9391)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 092a12e983 [feature] show create materialized view (#9391)
092a12e983 is described below

commit 092a12e983e7de2d8642b45423d9785c85ed313f
Author: Stalary <45...@qq.com>
AuthorDate: Wed May 11 09:29:55 2022 +0800

    [feature] show create materialized view (#9391)
---
 docs/.vuepress/sidebar/en.js                       |  1 +
 docs/.vuepress/sidebar/zh-CN.js                    |  1 +
 docs/en/advanced/materialized-view.md              | 10 ++-
 .../SHOW-CREATE-MATERIALIZED-VIEW.md               | 74 +++++++++++++++++
 docs/zh-CN/advanced/materialized-view.md           |  4 +
 .../SHOW-CREATE-MATERIALIZED-VIEW.md               | 74 +++++++++++++++++
 fe/fe-core/src/main/cup/sql_parser.cup             |  4 +
 .../analysis/ShowCreateMaterializedViewStmt.java   | 77 +++++++++++++++++
 .../doris/catalog/MaterializedIndexMeta.java       |  4 +
 .../java/org/apache/doris/qe/ShowExecutor.java     | 31 ++++++-
 .../ShowCreateMaterializedViewStmtTest.java        | 97 ++++++++++++++++++++++
 .../org/apache/doris/utframe/UtFrameUtils.java     |  2 +
 12 files changed, 373 insertions(+), 6 deletions(-)

diff --git a/docs/.vuepress/sidebar/en.js b/docs/.vuepress/sidebar/en.js
index 74b6f2f163..b5030bab59 100644
--- a/docs/.vuepress/sidebar/en.js
+++ b/docs/.vuepress/sidebar/en.js
@@ -735,6 +735,7 @@ module.exports = [
               "SHOW-CREATE-FUNCTION",
               "SHOW-CREATE-ROUTINE-LOAD",
               "SHOW-CREATE-TABLE",
+              "SHOW-CREATE-MATERIALIZED-VIEW",
               "SHOW-DATA",
               "SHOW-DATABASE-ID",
               "SHOW-DATABASES",
diff --git a/docs/.vuepress/sidebar/zh-CN.js b/docs/.vuepress/sidebar/zh-CN.js
index 9cac47e942..173d203ed3 100644
--- a/docs/.vuepress/sidebar/zh-CN.js
+++ b/docs/.vuepress/sidebar/zh-CN.js
@@ -735,6 +735,7 @@ module.exports = [
               "SHOW-CREATE-FUNCTION",
               "SHOW-CREATE-ROUTINE-LOAD",
               "SHOW-CREATE-TABLE",
+              "SHOW-CREATE-MATERIALIZED-VIEW",
               "SHOW-DATA",
               "SHOW-DATABASE-ID",
               "SHOW-DATABASES",
diff --git a/docs/en/advanced/materialized-view.md b/docs/en/advanced/materialized-view.md
index 8a855d1285..ea3d51ead8 100644
--- a/docs/en/advanced/materialized-view.md
+++ b/docs/en/advanced/materialized-view.md
@@ -148,11 +148,13 @@ You can see that the current `mv_test` table has three materialized views: mv\_1
 
 If the user no longer needs the materialized view, you can delete the materialized view by 'DROP' commen.
 
-The specific syntax can be viewed through the following command:
+You can view the specific syntax[SHOW CREATE MATERIALIZED VIEW](../sql-manual/sql-reference/Data-Definition-Statements/Drop/DROP-MATERIALIZED-VIEW.md)
 
-```
-HELP DROP MATERIALIZED VIEW
-```
+### View the materialized view that has been created
+
+Users can view the created materialized views by using commands
+
+You can view the specific syntax[SHOW CREATE MATERIALIZED VIEW](../sql-manual/sql-reference/Show-Statements/SHOW-CREATE-MATERIALIZED-VIEW.md)
 
 ## Best Practice 1
 
diff --git a/docs/en/sql-manual/sql-reference/Show-Statements/SHOW-CREATE-MATERIALIZED-VIEW.md b/docs/en/sql-manual/sql-reference/Show-Statements/SHOW-CREATE-MATERIALIZED-VIEW.md
new file mode 100644
index 0000000000..0dacbc8e29
--- /dev/null
+++ b/docs/en/sql-manual/sql-reference/Show-Statements/SHOW-CREATE-MATERIALIZED-VIEW.md
@@ -0,0 +1,74 @@
+---
+{
+    "title": "SHOW-CREATE-MATERIALIZED-VIEW",
+    "language": "en"
+}
+---
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+## SHOW-CREATE-MATERIALIZED-VIEW
+
+### Name
+
+SHOW CREATE MATERIALIZED VIEW
+
+### Description
+
+This statement is used to query statements that create materialized views.
+
+grammar:
+
+```sql
+SHOW CREATE MATERIALIZED VIEW mv_name ON table_name
+```
+
+1. mv_name:
+   Materialized view name. required.
+
+2. table_name:
+   The table name of materialized view. required.
+
+### Example
+
+Create materialized view
+
+```sql
+create materialized view id_col1 as select id,col1 from table3;
+```
+
+Return after query
+
+```sql
+mysql> show create materialized view id_col1 on table3;
++-----------+----------+----------------------------------------------------------------+
+| TableName | ViewName | CreateStmt                                                     |
++-----------+----------+----------------------------------------------------------------+
+| table3    | id_col1  | create materialized view id_col1 as select id,col1 from table3 |
++-----------+----------+----------------------------------------------------------------+
+1 row in set (0.00 sec)
+```
+
+### Keywords
+
+    SHOW, MATERIALIZED, VIEW
+
+### Best Practice
+
diff --git a/docs/zh-CN/advanced/materialized-view.md b/docs/zh-CN/advanced/materialized-view.md
index 4064f4686b..355b3c8383 100644
--- a/docs/zh-CN/advanced/materialized-view.md
+++ b/docs/zh-CN/advanced/materialized-view.md
@@ -146,7 +146,11 @@ MySQL [test]> desc mv_test all;
 
 具体的语法可查看[DROP MATERIALIZED VIEW](../sql-manual/sql-reference/Data-Definition-Statements/Drop/DROP-MATERIALIZED-VIEW.md) 
 
+### 查看已创建的物化视图
 
+用户可以通过命令查看已创建的物化视图的
+
+具体的语法可查看[SHOW CREATE MATERIALIZED VIEW](../sql-manual/sql-reference/Show-Statements/SHOW-CREATE-MATERIALIZED-VIEW.md)
 
 ## 最佳实践1
 
diff --git a/docs/zh-CN/sql-manual/sql-reference/Show-Statements/SHOW-CREATE-MATERIALIZED-VIEW.md b/docs/zh-CN/sql-manual/sql-reference/Show-Statements/SHOW-CREATE-MATERIALIZED-VIEW.md
new file mode 100644
index 0000000000..a7cfe2a07c
--- /dev/null
+++ b/docs/zh-CN/sql-manual/sql-reference/Show-Statements/SHOW-CREATE-MATERIALIZED-VIEW.md
@@ -0,0 +1,74 @@
+---
+{
+    "title": "SHOW-CREATE-MATERIALIZED-VIEW",
+    "language": "zh-CN"
+}
+---
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+## SHOW-CREATE-MATERIALIZED-VIEW
+
+### Name
+
+SHOW CREATE MATERIALIZED VIEW
+
+### Description
+
+该语句用于查询创建物化视图的语句。
+
+语法:
+
+```sql
+SHOW CREATE MATERIALIZED VIEW mv_name ON table_name
+```
+
+1. mv_name:
+        物化视图的名称。必填项。
+
+2. table_name:
+        物化视图所属的表名。必填项。
+
+### Example
+
+创建物化视图的语句为
+
+```sql
+create materialized view id_col1 as select id,col1 from table3;
+```
+
+查询后返回
+
+```sql
+mysql> show create materialized view id_col1 on table3;
++-----------+----------+----------------------------------------------------------------+
+| TableName | ViewName | CreateStmt                                                     |
++-----------+----------+----------------------------------------------------------------+
+| table3    | id_col1  | create materialized view id_col1 as select id,col1 from table3 |
++-----------+----------+----------------------------------------------------------------+
+1 row in set (0.00 sec)
+```
+
+### Keywords
+
+    SHOW, MATERIALIZED, VIEW
+
+### Best Practice
+
diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 27c88217b8..f5291bb38e 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -2895,6 +2895,10 @@ show_param ::=
     {:
         RESULT = new ShowLastInsertStmt();
     :}
+    | KW_CREATE KW_MATERIALIZED KW_VIEW ident:mvName KW_ON table_name:tableName
+    {:
+        RESULT = new ShowCreateMaterializedViewStmt(mvName, tableName);
+    :}
     ;
 
 opt_tmp ::=
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMaterializedViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMaterializedViewStmt.java
new file mode 100644
index 0000000000..a18ffd8ed8
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMaterializedViewStmt.java
@@ -0,0 +1,77 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.ScalarType;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.common.UserException;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.ShowResultSetMetaData;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+
+/**
+ * SHOW CREATE MATERIALIZED VIEW mv_name ON table_name.
+ **/
+@AllArgsConstructor
+@Getter
+public class ShowCreateMaterializedViewStmt extends ShowStmt {
+
+    private static final ShowResultSetMetaData META_DATA =
+            ShowResultSetMetaData.builder()
+                    .addColumn(new Column("TableName", ScalarType.createVarchar(255)))
+                    .addColumn(new Column("ViewName", ScalarType.createVarchar(255)))
+                    .addColumn(new Column("CreateStmt", ScalarType.createVarchar(65535)))
+                    .build();
+
+    private String mvName;
+
+    private TableName tableName;
+
+    @Override
+    public void analyze(Analyzer analyzer) throws UserException {
+        super.analyze(analyzer);
+        tableName.analyze(analyzer);
+        if (!Catalog.getCurrentCatalog().getAuth()
+                .checkTblPriv(ConnectContext.get(), tableName.getDb(), tableName.getTbl(), PrivPredicate.SHOW)) {
+            ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SHOW CREATE MATERIALIZED",
+                    ConnectContext.get().getQualifiedUser(),
+                    ConnectContext.get().getRemoteIP(),
+                    tableName.toSql());
+        }
+    }
+
+    @Override
+    public ShowResultSetMetaData getMetaData() {
+        return META_DATA;
+    }
+
+    @Override
+    public String toSql() {
+        StringBuilder stringBuilder = new StringBuilder();
+        stringBuilder.append("SHOW CREATE MATERIALIZED VIEW ");
+        stringBuilder.append("`").append(mvName).append("` ");
+        stringBuilder.append("ON ").append(tableName.toSql());
+        return stringBuilder.toString();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java
index d485f129d0..ede7c32a17 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java
@@ -136,6 +136,10 @@ public class MaterializedIndexMeta implements Writable, GsonPostProcessable {
         return null;
     }
 
+    public OriginStatement getDefineStmt() {
+        return defineStmt;
+    }
+
     @Override
     public boolean equals(Object obj) {
         if (!(obj instanceof MaterializedIndexMeta)) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
index 4decb80a00..2a1779c8d6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
@@ -36,6 +36,7 @@ import org.apache.doris.analysis.ShowColumnStatsStmt;
 import org.apache.doris.analysis.ShowColumnStmt;
 import org.apache.doris.analysis.ShowCreateDbStmt;
 import org.apache.doris.analysis.ShowCreateFunctionStmt;
+import org.apache.doris.analysis.ShowCreateMaterializedViewStmt;
 import org.apache.doris.analysis.ShowCreateRoutineLoadStmt;
 import org.apache.doris.analysis.ShowCreateTableStmt;
 import org.apache.doris.analysis.ShowDataSkewStmt;
@@ -102,6 +103,7 @@ import org.apache.doris.catalog.Function;
 import org.apache.doris.catalog.Index;
 import org.apache.doris.catalog.MaterializedIndex;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
+import org.apache.doris.catalog.MaterializedIndexMeta;
 import org.apache.doris.catalog.MetadataViewer;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
@@ -340,6 +342,8 @@ public class ShowExecutor {
             handleAdminShowTabletStorageFormat();
         } else if (stmt instanceof AdminDiagnoseTabletStmt) {
             handleAdminDiagnoseTablet();
+        } else if (stmt instanceof ShowCreateMaterializedViewStmt) {
+            handleShowCreateMaterializedView();
         } else {
             handleEmtpy();
         }
@@ -392,7 +396,7 @@ public class ShowExecutor {
         rowSet.add(Lists.newArrayList("HIVE", "YES", "HIVE database which data is in it", "NO", "NO", "NO"));
         rowSet.add(Lists.newArrayList("ICEBERG", "YES", "ICEBERG data lake which data is in it", "NO", "NO", "NO"));
         rowSet.add(Lists.newArrayList("ODBC", "YES", "ODBC driver which data we can connect", "NO", "NO", "NO"));
-        
+
         // Only success
         resultSet = new ShowResultSet(showStmt.getMetaData(), rowSet);
     }
@@ -1746,7 +1750,7 @@ public class ShowExecutor {
         List<List<String>> infos = Catalog.getCurrentCatalog().getAuth().getRoleInfo();
         resultSet = new ShowResultSet(showStmt.getMetaData(), infos);
     }
-    
+
     private void handleShowTrash() {
         ShowTrashStmt showStmt = (ShowTrashStmt) stmt;
         List<List<String>> infos = Lists.newArrayList();
@@ -2178,4 +2182,27 @@ public class ShowExecutor {
         resultSet = new ShowResultSet(showMetaData, resultRowSet);
     }
 
+    private void handleShowCreateMaterializedView() throws AnalysisException {
+        List<List<String>> resultRowSet = new ArrayList<>();
+        ShowCreateMaterializedViewStmt showStmt = (ShowCreateMaterializedViewStmt) stmt;
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(showStmt.getTableName().getDb());
+        Table table = db.getTableOrAnalysisException(showStmt.getTableName().getTbl());
+        if (table instanceof OlapTable) {
+            OlapTable baseTable = ((OlapTable) table);
+            Long indexIdByName = baseTable.getIndexIdByName(showStmt.getMvName());
+            if (indexIdByName != null) {
+                MaterializedIndexMeta meta = baseTable.getIndexMetaByIndexId(indexIdByName);
+                if (meta != null && meta.getDefineStmt() != null) {
+                    String originStmt = meta.getDefineStmt().originStmt;
+                    List<String> data = new ArrayList<>();
+                    data.add(showStmt.getTableName().getTbl());
+                    data.add(showStmt.getMvName());
+                    data.add(originStmt);
+                    resultRowSet.add(data);
+                }
+            }
+        }
+        resultSet = new ShowResultSet(showStmt.getMetaData(), resultRowSet);
+    }
+
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateMaterializedViewStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateMaterializedViewStmtTest.java
new file mode 100644
index 0000000000..501b50b418
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateMaterializedViewStmtTest.java
@@ -0,0 +1,97 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.ExceptionChecker;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.ShowExecutor;
+import org.apache.doris.utframe.DorisAssert;
+import org.apache.doris.utframe.UtFrameUtils;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.UUID;
+
+/**
+ * test for ShowCreateMaterializedViewStmt.
+ **/
+public class ShowCreateMaterializedViewStmtTest {
+
+    private static String runningDir = "fe/mocked/ShowCreateMaterializedViewStmtTest/" + UUID.randomUUID() + "/";
+
+    private static ConnectContext connectContext;
+
+    private static DorisAssert dorisAssert;
+
+    /**
+     * init.
+     **/
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        UtFrameUtils.createDorisCluster(runningDir);
+
+        // create connect context
+        connectContext = UtFrameUtils.createDefaultCtx();
+        dorisAssert = new DorisAssert(connectContext);
+        dorisAssert.withDatabase("test")
+                .withTable("create table test.table1 (k1 int, k2 int) distributed by hash(k1) "
+                        + "buckets 1 properties(\"replication_num\" = \"1\");")
+                .withMaterializedView("CREATE MATERIALIZED VIEW test_mv as select k1 from test.table1;");
+    }
+
+    @AfterClass
+    public static void tearDown() {
+        File file = new File(runningDir);
+        file.delete();
+    }
+
+    @Test
+    public void testNormal() throws Exception {
+        String showMvSql = "SHOW CREATE MATERIALIZED VIEW test_mv on test.table1;";
+        ShowCreateMaterializedViewStmt showStmt =
+                (ShowCreateMaterializedViewStmt) UtFrameUtils.parseAndAnalyzeStmt(showMvSql, connectContext);
+        ShowExecutor executor = new ShowExecutor(connectContext, showStmt);
+        Assert.assertEquals(executor.execute().getResultRows().get(0).get(2),
+                "CREATE MATERIALIZED VIEW test_mv as select k1 from test.table1;");
+    }
+
+    @Test
+    public void testNoView() throws Exception {
+        String showMvSql = "SHOW CREATE MATERIALIZED VIEW test_mv_empty on test.table1;";
+        ShowCreateMaterializedViewStmt showStmt =
+                (ShowCreateMaterializedViewStmt) UtFrameUtils.parseAndAnalyzeStmt(showMvSql, connectContext);
+        ShowExecutor executor = new ShowExecutor(connectContext, showStmt);
+        Assert.assertTrue(executor.execute().getResultRows().isEmpty());
+    }
+
+    @Test
+    public void testNoTable() throws Exception {
+        String showMvSql = "SHOW CREATE MATERIALIZED VIEW test_mv on test.table1_error;";
+        ShowCreateMaterializedViewStmt showStmt =
+                (ShowCreateMaterializedViewStmt) UtFrameUtils.parseAndAnalyzeStmt(showMvSql, connectContext);
+        ShowExecutor executor = new ShowExecutor(connectContext, showStmt);
+        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class,
+                "Unknown table 'table1_error' in default_cluster:test", executor::execute);
+
+    }
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/UtFrameUtils.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/UtFrameUtils.java
index 229e2d519e..5153931a42 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/utframe/UtFrameUtils.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/UtFrameUtils.java
@@ -34,6 +34,7 @@ import org.apache.doris.common.util.SqlParserUtils;
 import org.apache.doris.mysql.privilege.PaloAuth;
 import org.apache.doris.planner.Planner;
 import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.OriginStatement;
 import org.apache.doris.qe.QueryState;
 import org.apache.doris.qe.StmtExecutor;
 import org.apache.doris.system.Backend;
@@ -107,6 +108,7 @@ public class UtFrameUtils {
             }
         }
         statementBase.analyze(analyzer);
+        statementBase.setOrigStmt(new OriginStatement(originStmt, 0));
         return statementBase;
     }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org