You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/12/28 02:36:10 UTC

[GitHub] [doris] wsjz opened a new pull request, #15418: [feature-wip](iceberg-snapshot) Iceberg snap

wsjz opened a new pull request, #15418:
URL: https://github.com/apache/doris/pull/15418

   # Proposed changes
   
   Issue Number: close #xxx
   
   ## Problem summary
   
   Describe your changes.
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: 
       - [ ] Yes
       - [ ] No
       - [ ] I don't know
   2. Has unit tests been added:
       - [ ] Yes
       - [ ] No
       - [ ] No Need
   3. Has document been added or modified:
       - [ ] Yes
       - [ ] No
       - [ ] No Need
   4. Does it need to update dependencies:
       - [ ] Yes
       - [ ] No
   5. Are there any changes that cannot be rolled back:
       - [ ] Yes (If Yes, please explain WHY)
       - [ ] No
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc...
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] wsjz commented on a diff in pull request #15418: [feature-wip](iceberg-snapshot) Iceberg snap

Posted by GitBox <gi...@apache.org>.
wsjz commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058153494


##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -626,7 +628,7 @@ nonterminal StatementBase stmt, show_stmt, show_param, help_stmt, load_stmt,
     show_routine_load_stmt, show_routine_load_task_stmt, show_create_routine_load_stmt,
     describe_stmt, alter_stmt,
     use_stmt, kill_stmt, drop_stmt, recover_stmt, grant_stmt, revoke_stmt, create_stmt, set_stmt, sync_stmt, cancel_stmt, cancel_param, delete_stmt,
-    link_stmt, migrate_stmt, switch_stmt, enter_stmt, transaction_stmt, unsupported_stmt, export_stmt, admin_stmt, truncate_stmt,
+    link_stmt, migrate_stmt, switch_stmt, enter_stmt, transaction_stmt, time_travel_stmt, unsupported_stmt, export_stmt, admin_stmt, truncate_stmt,

Review Comment:
   remove



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] wsjz commented on a diff in pull request #15418: [feature-wip](multi-catalog) support Iceberg time travel in external table

Posted by GitBox <gi...@apache.org>.
wsjz commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058814877


##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -5095,9 +5098,31 @@ base_table_ref_list ::=
   ;
 
 base_table_ref ::=
-    table_name:name opt_partition_names:partitionNames opt_tablet_list:tabletIds opt_table_alias:alias opt_table_sample:tableSample opt_common_hints:commonHints
+    table_name:name opt_snapshot_version:snapshotVersion opt_partition_names:partitionNames opt_tablet_list:tabletIds opt_table_alias:alias opt_table_sample:tableSample opt_common_hints:commonHints
     {:
-        RESULT = new TableRef(name, alias, partitionNames, tabletIds, tableSample, commonHints);
+        RESULT = new TableRef(name, alias, partitionNames, tabletIds, tableSample, commonHints, snapshotVersion);
+    :}
+    ;
+
+opt_snapshot_version ::=
+    /* empty */
+    {:
+        RESULT = null;
+    :}
+    | snapshot_version:snapshotVersion
+    {:
+        RESULT = snapshotVersion;
+    :}
+    ;
+
+snapshot_version ::=

Review Comment:
   it maybe not only external table support time travel in the future
   but indeed need change the name, I will use table_snapshot



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] wsjz commented on a diff in pull request #15418: [feature-wip](multi-catalog) support Iceberg time travel in external table

Posted by GitBox <gi...@apache.org>.
wsjz commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058816289


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/SnapshotVersion.java:
##########
@@ -0,0 +1,66 @@
+// 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;
+
+public class SnapshotVersion {

Review Comment:
   I will use TableSnapshot



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] wsjz commented on a diff in pull request #15418: [feature-wip](multi-catalog) support Iceberg time travel in external table

Posted by GitBox <gi...@apache.org>.
wsjz commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058816085


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/BaseTableRef.java:
##########
@@ -70,6 +70,7 @@ public void analyze(Analyzer analyzer) throws UserException {
         name.analyze(analyzer);
         desc = analyzer.registerTableRef(this);
         isAnalyzed = true;  // true that we have assigned desc
+        analyzeSnapshotVersion(analyzer);

Review Comment:
   olap table maybe support snapshot read for time travel
   I change it to analyzeTableSnapshot



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] wsjz commented on a diff in pull request #15418: [feature-wip](multi-catalog) support Iceberg time travel in external table

Posted by GitBox <gi...@apache.org>.
wsjz commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058943350


##########
fe/fe-core/src/main/java/org/apache/doris/planner/external/IcebergScanProvider.java:
##########
@@ -168,6 +176,18 @@ public List<InputSplit> getSplits(List<Expr> exprs) throws UserException {
 
         org.apache.iceberg.Table table = getIcebergTable();
         TableScan scan = table.newScan();
+        TableSnapshot tableSnapshot = desc.getRef().getTableSnapshot();
+        if (tableSnapshot != null) {
+            TableSnapshot.VersionType type = tableSnapshot.getType();
+            if (type == TableSnapshot.VersionType.VERSION) {
+                scan = scan.useSnapshot(tableSnapshot.getVersion());

Review Comment:
   catch illegalArgument



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] wsjz commented on a diff in pull request #15418: [feature-wip](multi-catalog) support Iceberg time travel in external table

Posted by GitBox <gi...@apache.org>.
wsjz commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058814877


##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -5095,9 +5098,31 @@ base_table_ref_list ::=
   ;
 
 base_table_ref ::=
-    table_name:name opt_partition_names:partitionNames opt_tablet_list:tabletIds opt_table_alias:alias opt_table_sample:tableSample opt_common_hints:commonHints
+    table_name:name opt_snapshot_version:snapshotVersion opt_partition_names:partitionNames opt_tablet_list:tabletIds opt_table_alias:alias opt_table_sample:tableSample opt_common_hints:commonHints
     {:
-        RESULT = new TableRef(name, alias, partitionNames, tabletIds, tableSample, commonHints);
+        RESULT = new TableRef(name, alias, partitionNames, tabletIds, tableSample, commonHints, snapshotVersion);
+    :}
+    ;
+
+opt_snapshot_version ::=
+    /* empty */
+    {:
+        RESULT = null;
+    :}
+    | snapshot_version:snapshotVersion
+    {:
+        RESULT = snapshotVersion;
+    :}
+    ;
+
+snapshot_version ::=

Review Comment:
   it maybe not only external table support time travel in the future
   but indeed need change the name



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] morningman merged pull request #15418: [feature-wip](multi-catalog) support Iceberg time travel in external table

Posted by GitBox <gi...@apache.org>.
morningman merged PR #15418:
URL: https://github.com/apache/doris/pull/15418


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] wsjz commented on a diff in pull request #15418: [feature-wip](multi-catalog) support Iceberg time travel in external table

Posted by GitBox <gi...@apache.org>.
wsjz commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058816085


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/BaseTableRef.java:
##########
@@ -70,6 +70,7 @@ public void analyze(Analyzer analyzer) throws UserException {
         name.analyze(analyzer);
         desc = analyzer.registerTableRef(this);
         isAnalyzed = true;  // true that we have assigned desc
+        analyzeSnapshotVersion(analyzer);

Review Comment:
   olap table maybe support snapshot read for time travel



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] morningman commented on a diff in pull request #15418: [feature-wip](multi-catalog) support Iceberg time travel in external table

Posted by GitBox <gi...@apache.org>.
morningman commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058934476


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java:
##########
@@ -499,6 +515,27 @@ protected void analyzeHints() throws AnalysisException {
         }
     }
 
+    public void analyzeTableSnapshot(Analyzer analyzer) throws AnalysisException {

Review Comment:
   ```suggestion
       protected void analyzeTableSnapshot(Analyzer analyzer) throws AnalysisException {
   ```



##########
regression-test/suites/external_table_emr_p2/iceberg/test_external_catalog_icebergv2.groovy:
##########
@@ -24,9 +24,9 @@ suite("test_external_catalog_icebergv2", "p2") {
 
         sql """drop catalog if exists ${catalog_name};"""
         sql """
-            create catalog if not exists ${catalog_name} properties (
+            create catalog if not exists test_external_catalog_iceberg properties (
                 'type'='hms',
-                'hive.metastore.uris' = 'thrift://${extHiveHmsHost}:${extHiveHmsPort}'
+                'hive.metastore.uris' = 'thrift://172.21.16.47:7004'

Review Comment:
   Change this



##########
fe/fe-core/src/main/java/org/apache/doris/planner/external/IcebergScanProvider.java:
##########
@@ -168,6 +176,18 @@ public List<InputSplit> getSplits(List<Expr> exprs) throws UserException {
 
         org.apache.iceberg.Table table = getIcebergTable();
         TableScan scan = table.newScan();
+        TableSnapshot tableSnapshot = desc.getRef().getTableSnapshot();
+        if (tableSnapshot != null) {
+            TableSnapshot.VersionType type = tableSnapshot.getType();
+            if (type == TableSnapshot.VersionType.VERSION) {
+                scan = scan.useSnapshot(tableSnapshot.getVersion());
+            } else {
+                LocalDateTime asOfTime = LocalDateTime.parse(tableSnapshot.getTime(), DATE_TIME_FORMATTER);

Review Comment:
   You can use `TimeUtil.timeStringToLong()`



##########
fe/fe-core/src/main/java/org/apache/doris/planner/external/IcebergScanProvider.java:
##########
@@ -168,6 +176,18 @@ public List<InputSplit> getSplits(List<Expr> exprs) throws UserException {
 
         org.apache.iceberg.Table table = getIcebergTable();
         TableScan scan = table.newScan();
+        TableSnapshot tableSnapshot = desc.getRef().getTableSnapshot();
+        if (tableSnapshot != null) {
+            TableSnapshot.VersionType type = tableSnapshot.getType();
+            if (type == TableSnapshot.VersionType.VERSION) {
+                scan = scan.useSnapshot(tableSnapshot.getVersion());

Review Comment:
   What if user give a non-exist snapshot version?



##########
fe/fe-core/src/main/java/org/apache/doris/planner/external/IcebergScanProvider.java:
##########
@@ -168,6 +176,18 @@ public List<InputSplit> getSplits(List<Expr> exprs) throws UserException {
 
         org.apache.iceberg.Table table = getIcebergTable();
         TableScan scan = table.newScan();
+        TableSnapshot tableSnapshot = desc.getRef().getTableSnapshot();
+        if (tableSnapshot != null) {
+            TableSnapshot.VersionType type = tableSnapshot.getType();
+            if (type == TableSnapshot.VersionType.VERSION) {
+                scan = scan.useSnapshot(tableSnapshot.getVersion());

Review Comment:
   Catch the `NotFoundException` and throw `UserException`



##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -5095,12 +5098,33 @@ base_table_ref_list ::=
   ;
 
 base_table_ref ::=
-    table_name:name opt_partition_names:partitionNames opt_tablet_list:tabletIds opt_table_alias:alias opt_table_sample:tableSample opt_common_hints:commonHints
+    table_name:name opt_partition_names:partitionNames opt_tablet_list:tabletIds opt_table_alias:alias opt_table_sample:tableSample opt_common_hints:commonHints opt_snapshot_version:snapshotVersion
     {:
-        RESULT = new TableRef(name, alias, partitionNames, tabletIds, tableSample, commonHints);
+        RESULT = new TableRef(name, alias, partitionNames, tabletIds, tableSample, commonHints, snapshotVersion);
     :}
     ;
 
+opt_snapshot_version ::=
+    /* empty */
+    {:
+        RESULT = null;
+    :}
+    | snapshot_version:snapshotVersion
+    {:
+        RESULT = snapshotVersion;
+    :}
+    ;
+
+snapshot_version ::=
+    KW_FOR KW_VERSION KW_AS KW_OF STRING_LITERAL:version

Review Comment:
   `time` should be string and `version` should be integer.



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java:
##########
@@ -499,6 +515,27 @@ protected void analyzeHints() throws AnalysisException {
         }
     }
 
+    public void analyzeTableSnapshot(Analyzer analyzer) throws AnalysisException {
+        if (tableSnapshot == null) {
+            return;
+        }
+        TableIf.TableType tableType = this.getTable().getType();
+        if (tableType != TableIf.TableType.HMS_EXTERNAL_TABLE) {
+            ErrorReport.reportAnalysisException(ErrorCode.ERR_NONSUPPORT_TIME_TRAVEL_TABLE);
+        }
+        HMSExternalTable extTable = (HMSExternalTable) this.getTable();
+        if (extTable.getDlaType() != HMSExternalTable.DLAType.ICEBERG) {

Review Comment:
   The dlaType may not be initialized, you can wrap this logic inside the `HMSExternalTable`, and call `makeSureInitialized()` before using `dlaType`



##########
fe/fe-core/src/main/java/org/apache/doris/planner/external/IcebergScanProvider.java:
##########
@@ -75,6 +82,7 @@
  */
 public class IcebergScanProvider extends HiveScanProvider {
 
+    private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss");

Review Comment:
   No need to define this. All these can be found in TimeUtil



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] wsjz commented on a diff in pull request #15418: [feature-wip](multi-catalog) support Iceberg time travel in external table

Posted by GitBox <gi...@apache.org>.
wsjz commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058816217


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/SnapshotVersion.java:
##########
@@ -0,0 +1,66 @@
+// 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;

Review Comment:
   olap table maybe support snapshot read for time travel



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] wsjz commented on a diff in pull request #15418: [feature-wip](iceberg-snapshot) Iceberg snap

Posted by GitBox <gi...@apache.org>.
wsjz commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058159975


##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -1083,6 +1086,8 @@ stmt ::=
     {: RESULT = stmt; :}
     | transaction_stmt : stmt
     {: RESULT = stmt; :}
+    | time_travel_stmt : stmt
+    {: RESULT = stmt; :}

Review Comment:
   remove
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] hello-stephen commented on pull request #15418: [feature-wip](iceberg-snapshot) Iceberg snap

Posted by GitBox <gi...@apache.org>.
hello-stephen commented on PR #15418:
URL: https://github.com/apache/doris/pull/15418#issuecomment-1366672811

   TeamCity pipeline, clickbench performance test result:
    the sum of best hot time: 34.8 seconds
    load time: 642 seconds
    storage size: 17123092347 Bytes
    https://doris-community-test-1308700295.cos.ap-hongkong.myqcloud.com/tmp/20221228135933_clickbench_pr_70443.html


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] pengxiangyu commented on a diff in pull request #15418: [feature-wip](multi-catalog) support Iceberg time travel in external table

Posted by GitBox <gi...@apache.org>.
pengxiangyu commented on code in PR #15418:
URL: https://github.com/apache/doris/pull/15418#discussion_r1058792961


##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -5095,9 +5098,31 @@ base_table_ref_list ::=
   ;
 
 base_table_ref ::=
-    table_name:name opt_partition_names:partitionNames opt_tablet_list:tabletIds opt_table_alias:alias opt_table_sample:tableSample opt_common_hints:commonHints
+    table_name:name opt_snapshot_version:snapshotVersion opt_partition_names:partitionNames opt_tablet_list:tabletIds opt_table_alias:alias opt_table_sample:tableSample opt_common_hints:commonHints
     {:
-        RESULT = new TableRef(name, alias, partitionNames, tabletIds, tableSample, commonHints);
+        RESULT = new TableRef(name, alias, partitionNames, tabletIds, tableSample, commonHints, snapshotVersion);
+    :}
+    ;
+
+opt_snapshot_version ::=
+    /* empty */
+    {:
+        RESULT = null;
+    :}
+    | snapshot_version:snapshotVersion
+    {:
+        RESULT = snapshotVersion;
+    :}
+    ;
+
+snapshot_version ::=

Review Comment:
   external_table_snapshot is better, it contains not only version, but also time,



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/BaseTableRef.java:
##########
@@ -70,6 +70,7 @@ public void analyze(Analyzer analyzer) throws UserException {
         name.analyze(analyzer);
         desc = analyzer.registerTableRef(this);
         isAnalyzed = true;  // true that we have assigned desc
+        analyzeSnapshotVersion(analyzer);

Review Comment:
   It is better to be analyzeExternalTable(analyzer), analyzeSnapshotVersion(analyzer) is better in this function.
   So that you can deal with all external table options in this function in the furture.



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/SnapshotVersion.java:
##########
@@ -0,0 +1,66 @@
+// 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;
+
+public class SnapshotVersion {

Review Comment:
   May be ExternalTableSnapshot is better, it doesn't only contain version, but also time.



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/SnapshotVersion.java:
##########
@@ -0,0 +1,66 @@
+// 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;

Review Comment:
   package org.apache.doris.analysis.external;
   external table has more syntax, it will be too many in package: org.apache.doris.analysis



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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