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/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 9847f7789f [Feature](Export) `Export` sql supports to export data of 
`view` and `exrernal table` (#24070)
9847f7789f is described below

commit 9847f7789f260b97218b5ee643695de80b05a7cf
Author: Tiewei Fang <43782773+bepppo...@users.noreply.github.com>
AuthorDate: Wed Sep 13 22:55:19 2023 +0800

    [Feature](Export) `Export` sql supports to export data of `view` and 
`exrernal table` (#24070)
    
    Previously, EXPORT only supported the export of the olap table,
    This pr supports the export of view table and external table.
---
 .../main/java/org/apache/doris/load/ExportJob.java | 145 +++--
 .../org/apache/doris/load/ExportTaskExecutor.java  |  69 +--
 .../trees/plans/commands/ExportCommand.java        |  87 +--
 .../java/org/apache/doris/qe/ShowExecutor.java     |   2 +-
 .../analysis/ExportToOutfileLogicalPlanTest.java   |   9 +-
 .../data/export_p0/test_export_view.out            | 200 +++++++
 .../export/test_export_external_table.out          | 473 +++++++++++++++
 .../suites/export_p0/test_export_view.groovy       | 644 +++++++++++++++++++++
 .../export/test_export_external_table.groovy       | 552 ++++++++++++++++++
 9 files changed, 2064 insertions(+), 117 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java 
b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java
index 7e719f617b..f1fae800b8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java
@@ -42,7 +42,8 @@ import org.apache.doris.catalog.MaterializedIndex;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
-import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.TableIf;
+import org.apache.doris.catalog.TableIf.TableType;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.Pair;
@@ -51,6 +52,7 @@ import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.common.util.SqlParserUtils;
 import org.apache.doris.common.util.TimeUtils;
+import org.apache.doris.datasource.InternalCatalog;
 import org.apache.doris.nereids.StatementContext;
 import org.apache.doris.nereids.analyzer.UnboundRelation;
 import org.apache.doris.nereids.analyzer.UnboundSlot;
@@ -199,7 +201,7 @@ public class ExportJob implements Writable {
 
     private List<String> exportColumns = Lists.newArrayList();
 
-    private Table exportTable;
+    private TableIf exportTable;
 
     // when set to true, means this job instance is created by replay 
thread(FE restarted or master changed)
     private boolean isReplayed = false;
@@ -242,17 +244,6 @@ public class ExportJob implements Writable {
         this.id = jobId;
     }
 
-    /**
-     * For an ExportJob:
-     * The ExportJob is divided into multiple 'ExportTaskExecutor'
-     * according to the 'parallelism' set by the user.
-     * The tablets which will be exported by this ExportJob are divided into 
'parallelism' copies,
-     * and each ExportTaskExecutor is responsible for a list of tablets.
-     * The tablets responsible for an ExportTaskExecutor will be assigned to 
multiple OutfileStmt
-     * according to the 'TABLETS_NUM_PER_OUTFILE_IN_EXPORT'.
-     *
-     * @throws UserException
-     */
     public void generateOutfileStatement() throws UserException {
         exportTable.readLock();
         try {
@@ -264,39 +255,35 @@ public class ExportJob implements Writable {
         generateExportJobExecutor();
     }
 
-    public void generateOutfileLogicalPlans(List<String> nameParts)
+    /**
+     * For an ExportJob:
+     * The ExportJob is divided into multiple 'ExportTaskExecutor'
+     * according to the 'parallelism' set by the user.
+     * The tablets which will be exported by this ExportJob are divided into 
'parallelism' copies,
+     * and each ExportTaskExecutor is responsible for a list of tablets.
+     * The tablets responsible for an ExportTaskExecutor will be assigned to 
multiple OutfileStmt
+     * according to the 'TABLETS_NUM_PER_OUTFILE_IN_EXPORT'.
+     *
+     * @throws UserException
+     */
+    public void generateOutfileLogicalPlans(List<String> qualifiedTableName)
             throws UserException {
+        String catalogType = 
Env.getCurrentEnv().getCatalogMgr().getCatalog(this.tableName.getCtl()).getType();
         exportTable.readLock();
         try {
-            // build source columns
-            List<NamedExpression> selectLists = Lists.newArrayList();
-            if (exportColumns.isEmpty()) {
-                selectLists.add(new UnboundStar(ImmutableList.of()));
-            } else {
-                this.exportColumns.stream().forEach(col -> {
-                    selectLists.add(new UnboundSlot(this.tableName.getTbl(), 
col));
-                });
-            }
-
-            // get all tablets
-            List<List<Long>> tabletsListPerParallel = splitTablets();
-
-            // Each Outfile clause responsible for 
MAXIMUM_TABLETS_OF_OUTFILE_IN_EXPORT tablets
-            for (List<Long> tabletsList : tabletsListPerParallel) {
-                List<StatementBase> logicalPlanAdapters = Lists.newArrayList();
-                for (int i = 0; i < tabletsList.size(); i += 
MAXIMUM_TABLETS_OF_OUTFILE_IN_EXPORT) {
-                    int end = i + MAXIMUM_TABLETS_OF_OUTFILE_IN_EXPORT < 
tabletsList.size()
-                            ? i + MAXIMUM_TABLETS_OF_OUTFILE_IN_EXPORT : 
tabletsList.size();
-                    List<Long> tabletIds = new 
ArrayList<>(tabletsList.subList(i, end));
-
-                    // generate LogicalPlan
-                    LogicalPlan plan = generateOneLogicalPlan(nameParts, 
tabletIds, selectLists);
-                    // generate  LogicalPlanAdapter
-                    StatementBase statementBase = 
generateLogicalPlanAdapter(plan);
-
-                    logicalPlanAdapters.add(statementBase);
+            if (InternalCatalog.INTERNAL_CATALOG_NAME.equals(catalogType)) {
+                if (exportTable.getType() == TableType.VIEW) {
+                    // view table
+                    generateViewOrExternalTableOutfile(qualifiedTableName);
+                } else if (exportTable.getType() == TableType.OLAP) {
+                    // olap table
+                    generateOlapTableOutfile(qualifiedTableName);
+                } else {
+                    throw new UserException("Do not support export table type 
[" + exportTable.getType() + "]");
                 }
-                selectStmtListPerParallel.add(logicalPlanAdapters);
+            } else {
+                // external table
+                generateViewOrExternalTableOutfile(qualifiedTableName);
             }
 
             // debug LOG output
@@ -315,11 +302,77 @@ public class ExportJob implements Writable {
         generateExportJobExecutor();
     }
 
-    private LogicalPlan generateOneLogicalPlan(List<String> nameParts, 
List<Long> tabletIds,
-            List<NamedExpression> selectLists) {
+    private void generateOlapTableOutfile(List<String> qualifiedTableName) 
throws UserException {
+        // build source columns
+        List<NamedExpression> selectLists = Lists.newArrayList();
+        if (exportColumns.isEmpty()) {
+            selectLists.add(new UnboundStar(ImmutableList.of()));
+        } else {
+            this.exportColumns.stream().forEach(col -> {
+                selectLists.add(new UnboundSlot(this.tableName.getTbl(), col));
+            });
+        }
+
+        // get all tablets
+        List<List<Long>> tabletsListPerParallel = splitTablets();
+
+        // Each Outfile clause responsible for 
MAXIMUM_TABLETS_OF_OUTFILE_IN_EXPORT tablets
+        for (List<Long> tabletsList : tabletsListPerParallel) {
+            List<StatementBase> logicalPlanAdapters = Lists.newArrayList();
+            for (int i = 0; i < tabletsList.size(); i += 
MAXIMUM_TABLETS_OF_OUTFILE_IN_EXPORT) {
+                int end = i + MAXIMUM_TABLETS_OF_OUTFILE_IN_EXPORT < 
tabletsList.size()
+                        ? i + MAXIMUM_TABLETS_OF_OUTFILE_IN_EXPORT : 
tabletsList.size();
+                List<Long> tabletIds = new ArrayList<>(tabletsList.subList(i, 
end));
+
+                // generate LogicalPlan
+                LogicalPlan plan = generateOneLogicalPlan(qualifiedTableName, 
tabletIds,
+                        this.partitionNames, selectLists);
+                // generate  LogicalPlanAdapter
+                StatementBase statementBase = generateLogicalPlanAdapter(plan);
+
+                logicalPlanAdapters.add(statementBase);
+            }
+            selectStmtListPerParallel.add(logicalPlanAdapters);
+        }
+    }
+
+    /**
+     * This method used to generate outfile sql for view table or external 
table.
+     * @throws UserException
+     */
+    private void generateViewOrExternalTableOutfile(List<String> 
qualifiedTableName) {
+        // Because there is no division of tablets in view and external table
+        // we set parallelism = 1;
+        this.parallelism = 1;
+        LOG.debug("Because there is no division of tablets in view and 
external table, we set parallelism = 1");
+
+        // build source columns
+        List<NamedExpression> selectLists = Lists.newArrayList();
+        if (exportColumns.isEmpty()) {
+            selectLists.add(new UnboundStar(ImmutableList.of()));
+        } else {
+            this.exportColumns.stream().forEach(col -> {
+                selectLists.add(new UnboundSlot(this.tableName.getTbl(), col));
+            });
+        }
+
+        List<StatementBase> logicalPlanAdapters = Lists.newArrayList();
+
+        // generate LogicalPlan
+        LogicalPlan plan = generateOneLogicalPlan(qualifiedTableName, 
ImmutableList.of(),
+                ImmutableList.of(), selectLists);
+        // generate  LogicalPlanAdapter
+        StatementBase statementBase = generateLogicalPlanAdapter(plan);
+
+        logicalPlanAdapters.add(statementBase);
+        selectStmtListPerParallel.add(logicalPlanAdapters);
+    }
+
+    private LogicalPlan generateOneLogicalPlan(List<String> 
qualifiedTableName, List<Long> tabletIds,
+            List<String> partitions, List<NamedExpression> selectLists) {
         // UnboundRelation
-        LogicalPlan plan = new 
UnboundRelation(StatementScopeIdGenerator.newRelationId(), nameParts,
-                this.partitionNames, false, tabletIds, ImmutableList.of());
+        LogicalPlan plan = new 
UnboundRelation(StatementScopeIdGenerator.newRelationId(), qualifiedTableName,
+                partitions, false, tabletIds, ImmutableList.of());
         // LogicalCheckPolicy
         plan = new LogicalCheckPolicy<>(plan);
         // LogicalFilter
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/load/ExportTaskExecutor.java 
b/fe/fe-core/src/main/java/org/apache/doris/load/ExportTaskExecutor.java
index cb562e9120..c7d7c4032c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportTaskExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportTaskExecutor.java
@@ -24,6 +24,7 @@ import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
+import org.apache.doris.catalog.TableIf.TableType;
 import org.apache.doris.catalog.TabletMeta;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.load.ExportFailMsg.CancelType;
@@ -84,47 +85,49 @@ public class ExportTaskExecutor implements 
TransientTaskExecutor {
                 throw new JobException("Export executor has been canceled, 
task id: {}", taskId);
             }
             // check the version of tablets
-            try {
-                Database db = 
Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException(
-                        exportJob.getTableName().getDb());
-                OlapTable table = 
db.getOlapTableOrAnalysisException(exportJob.getTableName().getTbl());
-                table.readLock();
+            if (exportJob.getExportTable().getType() == TableType.OLAP) {
                 try {
-                    List<Long> tabletIds;
-                    if 
(exportJob.getSessionVariables().isEnableNereidsPlanner()) {
-                        LogicalPlanAdapter logicalPlanAdapter = 
(LogicalPlanAdapter) selectStmtLists.get(idx);
-                        Optional<UnboundRelation> unboundRelation = 
findUnboundRelation(
-                                logicalPlanAdapter.getLogicalPlan());
-                        tabletIds = unboundRelation.get().getTabletIds();
-                    } else {
-                        SelectStmt selectStmt = (SelectStmt) 
selectStmtLists.get(idx);
-                        tabletIds = 
selectStmt.getTableRefs().get(0).getSampleTabletIds();
-                    }
+                    Database db = 
Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException(
+                            exportJob.getTableName().getDb());
+                    OlapTable table = 
db.getOlapTableOrAnalysisException(exportJob.getTableName().getTbl());
+                    table.readLock();
+                    try {
+                        List<Long> tabletIds;
+                        if 
(exportJob.getSessionVariables().isEnableNereidsPlanner()) {
+                            LogicalPlanAdapter logicalPlanAdapter = 
(LogicalPlanAdapter) selectStmtLists.get(idx);
+                            Optional<UnboundRelation> unboundRelation = 
findUnboundRelation(
+                                    logicalPlanAdapter.getLogicalPlan());
+                            tabletIds = unboundRelation.get().getTabletIds();
+                        } else {
+                            SelectStmt selectStmt = (SelectStmt) 
selectStmtLists.get(idx);
+                            tabletIds = 
selectStmt.getTableRefs().get(0).getSampleTabletIds();
+                        }
 
-                    for (Long tabletId : tabletIds) {
-                        TabletMeta tabletMeta = 
Env.getCurrentEnv().getTabletInvertedIndex().getTabletMeta(
-                                tabletId);
-                        Partition partition = 
table.getPartition(tabletMeta.getPartitionId());
-                        long nowVersion = partition.getVisibleVersion();
-                        long oldVersion = 
exportJob.getPartitionToVersion().get(partition.getName());
-                        if (nowVersion != oldVersion) {
-                            
exportJob.updateExportJobState(ExportJobState.CANCELLED, taskId, null,
-                                    CancelType.RUN_FAIL, "The version of 
tablet {" + tabletId + "} has changed");
-                            throw new JobException("Export Job[{}]: Tablet {} 
has changed version, old version = {}, "
-                                    + "now version = {}", exportJob.getId(), 
tabletId, oldVersion, nowVersion);
+                        for (Long tabletId : tabletIds) {
+                            TabletMeta tabletMeta = 
Env.getCurrentEnv().getTabletInvertedIndex().getTabletMeta(
+                                    tabletId);
+                            Partition partition = 
table.getPartition(tabletMeta.getPartitionId());
+                            long nowVersion = partition.getVisibleVersion();
+                            long oldVersion = 
exportJob.getPartitionToVersion().get(partition.getName());
+                            if (nowVersion != oldVersion) {
+                                
exportJob.updateExportJobState(ExportJobState.CANCELLED, taskId, null,
+                                        CancelType.RUN_FAIL, "The version of 
tablet {" + tabletId + "} has changed");
+                                throw new JobException("Export Job[{}]: Tablet 
{} has changed version, old version = {}"
+                                        + ", now version = {}", 
exportJob.getId(), tabletId, oldVersion, nowVersion);
+                            }
                         }
+                    } catch (Exception e) {
+                        
exportJob.updateExportJobState(ExportJobState.CANCELLED, taskId, null,
+                                ExportFailMsg.CancelType.RUN_FAIL, 
e.getMessage());
+                        throw new JobException(e);
+                    } finally {
+                        table.readUnlock();
                     }
-                } catch (Exception e) {
+                } catch (AnalysisException e) {
                     exportJob.updateExportJobState(ExportJobState.CANCELLED, 
taskId, null,
                             ExportFailMsg.CancelType.RUN_FAIL, e.getMessage());
                     throw new JobException(e);
-                } finally {
-                    table.readUnlock();
                 }
-            } catch (AnalysisException e) {
-                exportJob.updateExportJobState(ExportJobState.CANCELLED, 
taskId, null,
-                        ExportFailMsg.CancelType.RUN_FAIL, e.getMessage());
-                throw new JobException(e);
             }
 
             try (AutoCloseConnectContext r = buildConnectContext()) {
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExportCommand.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExportCommand.java
index 12d826b2bc..a48753a119 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExportCommand.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExportCommand.java
@@ -24,17 +24,19 @@ import org.apache.doris.analysis.Separator;
 import org.apache.doris.analysis.StorageBackend;
 import org.apache.doris.analysis.TableName;
 import org.apache.doris.catalog.BrokerMgr;
-import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.DatabaseIf;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.TableIf;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.FeNameFormat;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.PropertyAnalyzer;
-import org.apache.doris.common.util.Util;
+import org.apache.doris.datasource.CatalogIf;
+import org.apache.doris.datasource.InternalCatalog;
 import org.apache.doris.load.ExportJob;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.nereids.exceptions.AnalysisException;
@@ -119,7 +121,9 @@ public class ExportCommand extends Command implements 
ForwardWithSync {
     @Override
     public void run(ConnectContext ctx, StmtExecutor executor) throws 
Exception {
         // get tblName
-        TableName tblName = getTableName(ctx);
+        List<String> qualifiedTableName = RelationUtil.getQualifierName(ctx, 
this.nameParts);
+        TableName tblName = new TableName(qualifiedTableName.get(0), 
qualifiedTableName.get(1),
+                qualifiedTableName.get(2));
 
         // check auth
         if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ctx, 
tblName.getDb(), tblName.getTbl(),
@@ -141,9 +145,9 @@ public class ExportCommand extends Command implements 
ForwardWithSync {
     private void checkAllParameters(ConnectContext ctx, TableName tblName, 
Map<String, String> fileProperties)
             throws UserException {
         checkPropertyKey(fileProperties);
-        checkPartitions(ctx.getEnv(), tblName);
+        checkPartitions(ctx, tblName);
         checkBrokerDesc(ctx);
-        checkFileProperties(fileProperties, tblName);
+        checkFileProperties(ctx, fileProperties, tblName);
     }
 
     // check property key
@@ -156,24 +160,28 @@ public class ExportCommand extends Command implements 
ForwardWithSync {
     }
 
     // check partitions specified by user are belonged to the table.
-    private void checkPartitions(Env env, TableName tblName) throws 
AnalysisException, UserException {
+    private void checkPartitions(ConnectContext ctx, TableName tblName) throws 
AnalysisException, UserException {
         if (this.partitionsNames.isEmpty()) {
             return;
         }
 
+        CatalogIf catalog = 
ctx.getEnv().getCatalogMgr().getCatalogOrAnalysisException(tblName.getCtl());
+        // As for external table, we do not support export PARTITION
+        if (!InternalCatalog.INTERNAL_CATALOG_NAME.equals(catalog.getType())) {
+            throw new AnalysisException("Table[" + tblName.getTbl() + "] is 
EXTERNAL TABLE type, "
+                    + "do not support export PARTITION.");
+        }
+
+        DatabaseIf db = catalog.getDbOrAnalysisException(tblName.getDb());
+        Table table = (Table) db.getTableOrAnalysisException(tblName.getTbl());
+
         if (this.partitionsNames.size() > 
Config.maximum_number_of_export_partitions) {
             throw new AnalysisException("The partitions number of this export 
job is larger than the maximum number"
                     + " of partitions allowed by an export job");
         }
 
-        Database db = 
env.getInternalCatalog().getDbOrAnalysisException(tblName.getDb());
-        Table table = db.getTableOrAnalysisException(tblName.getTbl());
         table.readLock();
         try {
-            // check table
-            if (!table.isPartitioned()) {
-                throw new AnalysisException("Table[" + tblName.getTbl() + "] 
is not partitioned.");
-            }
             Table.TableType tblType = table.getType();
             switch (tblType) {
                 case MYSQL:
@@ -181,15 +189,23 @@ public class ExportCommand extends Command implements 
ForwardWithSync {
                 case JDBC:
                 case OLAP:
                     break;
+                case VIEW: // We support export view, so we do not need to 
check partition here.
+                    if (this.partitionsNames.size() > 0) {
+                        throw new AnalysisException("Table[" + 
tblName.getTbl() + "] is VIEW type, "
+                                + "do not support export PARTITION.");
+                    }
+                    return;
                 case BROKER:
                 case SCHEMA:
                 case INLINE_VIEW:
-                case VIEW:
                 default:
                     throw new AnalysisException("Table[" + tblName.getTbl() + 
"] is "
                             + tblType + " type, do not support EXPORT.");
             }
-
+            // check table
+            if (!table.isPartitioned()) {
+                throw new AnalysisException("Table[" + tblName.getTbl() + "] 
is not partitioned.");
+            }
             for (String partitionName : this.partitionsNames) {
                 Partition partition = table.getPartition(partitionName);
                 if (partition == null) {
@@ -220,12 +236,15 @@ public class ExportCommand extends Command implements 
ForwardWithSync {
     private ExportJob generateExportJob(ConnectContext ctx, Map<String, 
String> fileProperties, TableName tblName)
             throws UserException {
         ExportJob exportJob = new ExportJob();
-        // set export job
-        Database db = 
Env.getCurrentInternalCatalog().getDbOrDdlException(tblName.getDb());
+        // set export job and check catalog/db/table
+        CatalogIf catalog = 
ctx.getEnv().getCatalogMgr().getCatalogOrAnalysisException(tblName.getCtl());
+        DatabaseIf db = catalog.getDbOrAnalysisException(tblName.getDb());
+        TableIf table = db.getTableOrAnalysisException(tblName.getTbl());
+
         exportJob.setDbId(db.getId());
         exportJob.setTableName(tblName);
-        exportJob.setExportTable(db.getTableOrDdlException(tblName.getTbl()));
-        
exportJob.setTableId(db.getTableOrDdlException(tblName.getTbl()).getId());
+        exportJob.setExportTable(table);
+        exportJob.setTableId(table.getId());
 
         // set partitions
         exportJob.setPartitionNames(this.partitionsNames);
@@ -292,23 +311,15 @@ public class ExportCommand extends Command implements 
ForwardWithSync {
                 .getQueryTimeoutS());
 
         // exportJob generate outfile sql
-        exportJob.generateOutfileLogicalPlans(this.nameParts);
+        
exportJob.generateOutfileLogicalPlans(RelationUtil.getQualifierName(ctx, 
this.nameParts));
         return exportJob;
     }
 
-    private TableName getTableName(ConnectContext ctx) throws UserException {
-        // get tblName
-        List<String> qualifiedTableName = RelationUtil.getQualifierName(ctx, 
this.nameParts);
-        TableName tblName = new TableName(qualifiedTableName.get(0), 
qualifiedTableName.get(1),
-                qualifiedTableName.get(2));
-        Util.prohibitExternalCatalog(tblName.getCtl(), 
this.getClass().getSimpleName());
-        return tblName;
-    }
-
-    private void checkFileProperties(Map<String, String> fileProperties, 
TableName tblName) throws UserException {
+    private void checkFileProperties(ConnectContext ctx, Map<String, String> 
fileProperties, TableName tblName)
+            throws UserException {
         // check user specified columns
         if (fileProperties.containsKey(LoadStmt.KEY_IN_PARAM_COLUMNS)) {
-            checkColumns(fileProperties.get(LoadStmt.KEY_IN_PARAM_COLUMNS), 
tblName);
+            checkColumns(ctx, 
fileProperties.get(LoadStmt.KEY_IN_PARAM_COLUMNS), tblName);
         }
 
         // check user specified label
@@ -317,12 +328,18 @@ public class ExportCommand extends Command implements 
ForwardWithSync {
         }
     }
 
-    private void checkColumns(String columns, TableName tblName) throws 
AnalysisException, UserException {
+    private void checkColumns(ConnectContext ctx, String columns, TableName 
tblName)
+            throws AnalysisException, UserException {
         if (columns.isEmpty()) {
             throw new AnalysisException("columns can not be empty");
         }
-        Database db = 
Env.getCurrentInternalCatalog().getDbOrDdlException(tblName.getDb());
-        Table table = db.getTableOrDdlException(tblName.getTbl());
+
+        CatalogIf catalog = 
ctx.getEnv().getCatalogMgr().getCatalogOrAnalysisException(tblName.getCtl());
+        DatabaseIf db = catalog.getDbOrAnalysisException(tblName.getDb());
+        TableIf table = db.getTableOrAnalysisException(tblName.getTbl());
+
+        // As for external table
+        // their base schemas are equals to full schemas
         List<String> tableColumns = table.getBaseSchema().stream().map(column 
-> column.getName())
                 .collect(Collectors.toList());
         Splitter split = Splitter.on(',').trimResults().omitEmptyStrings();
@@ -339,6 +356,10 @@ public class ExportCommand extends Command implements 
ForwardWithSync {
         return this.fileProperties;
     }
 
+    public List<String> getNameParts() {
+        return this.nameParts;
+    }
+
     @Override
     public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
         return visitor.visitExportCommand(this, context);
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 3b4049338b..465f6a3bd5 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
@@ -1958,7 +1958,7 @@ public class ShowExecutor {
     private void handleShowExport() throws AnalysisException {
         ShowExportStmt showExportStmt = (ShowExportStmt) stmt;
         Env env = Env.getCurrentEnv();
-        Database db = 
env.getInternalCatalog().getDbOrAnalysisException(showExportStmt.getDbName());
+        DatabaseIf db = 
env.getCurrentCatalog().getDbOrAnalysisException(showExportStmt.getDbName());
         long dbId = db.getId();
 
         ExportMgr exportMgr = env.getExportMgr();
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/analysis/ExportToOutfileLogicalPlanTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/analysis/ExportToOutfileLogicalPlanTest.java
index 955147902c..633ab0cdd4 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/analysis/ExportToOutfileLogicalPlanTest.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/analysis/ExportToOutfileLogicalPlanTest.java
@@ -25,6 +25,7 @@ import org.apache.doris.nereids.glue.LogicalPlanAdapter;
 import org.apache.doris.nereids.parser.NereidsParser;
 import org.apache.doris.nereids.trees.plans.commands.ExportCommand;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.util.RelationUtil;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.utframe.TestWithFeService;
 
@@ -412,9 +413,6 @@ public class ExportToOutfileLogicalPlanTest extends 
TestWithFeService {
         ExportCommand exportCommand = (ExportCommand) parseSql(exportSql);
         List<List<StatementBase>> selectStmtListPerParallel = 
Lists.newArrayList();
         try {
-            Method getTableName = 
exportCommand.getClass().getDeclaredMethod("getTableName", 
ConnectContext.class);
-            getTableName.setAccessible(true);
-
             Method checkAllParameters = 
exportCommand.getClass().getDeclaredMethod("checkAllParameters",
                     ConnectContext.class, TableName.class, Map.class);
             checkAllParameters.setAccessible(true);
@@ -423,7 +421,10 @@ public class ExportToOutfileLogicalPlanTest extends 
TestWithFeService {
                     ConnectContext.class, Map.class, TableName.class);
             generateExportJob.setAccessible(true);
 
-            TableName tblName = (TableName) getTableName.invoke(exportCommand, 
connectContext);
+            // get tblName
+            List<String> qualifiedTableName = 
RelationUtil.getQualifierName(connectContext, exportCommand.getNameParts());
+            TableName tblName = new TableName(qualifiedTableName.get(0), 
qualifiedTableName.get(1),
+                    qualifiedTableName.get(2));
             checkAllParameters.invoke(exportCommand, connectContext, tblName, 
exportCommand.getFileProperties());
 
             ExportJob job = (ExportJob) generateExportJob.invoke(
diff --git a/regression-test/data/export_p0/test_export_view.out 
b/regression-test/data/export_p0/test_export_view.out
new file mode 100644
index 0000000000..2f06e611d0
--- /dev/null
+++ b/regression-test/data/export_p0/test_export_view.out
@@ -0,0 +1,200 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !select_export --
+1      zhangsan        19      1
+2      zhangsan        20      2
+3      zhangsan        21      3
+4      zhangsan        22      4
+5      zhangsan        23      5
+6      zhangsan        24      6
+7      zhangsan        25      7
+8      zhangsan        26      8
+9      zhangsan        27      9
+10     zhangsan        28      10
+11     zhangsan        29      11
+12     zhangsan        30      12
+13     zhangsan        31      13
+14     zhangsan        32      14
+15     zhangsan        33      15
+16     zhangsan        34      16
+17     zhangsan        35      17
+18     zhangsan        36      18
+19     zhangsan        37      19
+20     zhangsan        38      20
+21     zhangsan        39      21
+22     zhangsan        40      22
+23     zhangsan        41      23
+24     zhangsan        42      24
+25     zhangsan        43      25
+26     zhangsan        44      26
+27     zhangsan        45      27
+28     zhangsan        46      28
+29     zhangsan        47      29
+30     zhangsan        48      30
+31     zhangsan        49      31
+32     zhangsan        50      32
+33     zhangsan        51      33
+34     zhangsan        52      34
+35     zhangsan        53      35
+36     zhangsan        54      36
+37     zhangsan        55      37
+38     zhangsan        56      38
+39     zhangsan        57      39
+40     zhangsan        58      40
+41     zhangsan        59      41
+42     zhangsan        60      42
+43     zhangsan        61      43
+44     zhangsan        62      44
+45     zhangsan        63      45
+46     zhangsan        64      46
+47     zhangsan        65      47
+48     zhangsan        66      48
+49     zhangsan        67      49
+50     lisi    68      50
+51     lisi    69      51
+52     lisi    70      52
+53     lisi    71      53
+54     lisi    72      54
+55     lisi    73      55
+56     lisi    74      56
+57     lisi    75      57
+58     lisi    76      58
+59     lisi    77      59
+60     lisi    78      60
+61     lisi    79      61
+62     lisi    80      62
+63     lisi    81      63
+64     lisi    82      64
+65     lisi    83      65
+66     lisi    84      66
+67     lisi    85      67
+68     lisi    86      68
+69     lisi    87      69
+70     lisi    88      70
+71     lisi    89      71
+72     lisi    90      72
+73     lisi    91      73
+74     lisi    92      74
+75     lisi    93      75
+76     lisi    94      76
+77     lisi    95      77
+78     lisi    96      78
+79     lisi    97      79
+80     wangwu  98      80
+81     wangwu  99      81
+82     wangwu  100     82
+83     wangwu  101     83
+84     wangwu  102     84
+85     wangwu  103     85
+86     wangwu  104     86
+87     wangwu  105     87
+88     wangwu  106     88
+89     wangwu  107     89
+90     wangwu  108     90
+91     wangwu  109     91
+92     wangwu  110     92
+93     wangwu  111     93
+94     wangwu  112     94
+95     wangwu  113     95
+96     wangwu  114     96
+97     wangwu  115     97
+98     wangwu  116     98
+99     wangwu  117     99
+100    wangwu  118     100
+101    wangwu  119     101
+102    wangwu  120     102
+103    wangwu  121     103
+104    wangwu  122     104
+105    wangwu  123     105
+106    wangwu  124     106
+107    wangwu  125     107
+108    wangwu  126     108
+109    wangwu  127     109
+110    wangwu  128     110
+111    wangwu  129     111
+112    wangwu  130     112
+113    wangwu  131     113
+114    wangwu  132     114
+115    wangwu  133     115
+116    wangwu  134     116
+117    wangwu  135     117
+118    wangwu  136     118
+119    wangwu  137     119
+120    fangfang        138     120
+121    fangfang        139     121
+122    fangfang        140     122
+123    fangfang        141     123
+124    fangfang        142     124
+125    fangfang        143     125
+126    fangfang        144     126
+127    fangfang        145     127
+128    fangfang        146     128
+129    fangfang        147     129
+130    fangfang        148     130
+131    fangfang        149     131
+132    fangfang        150     132
+133    fangfang        151     133
+134    fangfang        152     134
+135    fangfang        153     135
+136    fangfang        154     136
+137    fangfang        155     137
+138    fangfang        156     138
+139    fangfang        157     139
+140    fangfang        158     140
+141    fangfang        159     141
+142    fangfang        160     142
+143    fangfang        161     143
+144    fangfang        162     144
+145    fangfang        163     145
+146    fangfang        164     146
+147    fangfang        165     147
+148    fangfang        166     148
+149    fangfang        167     149
+150    xiexie  \N      150
+
+-- !select_load1 --
+fangfang       120     167     4035
+lisi   50      97      1935
+wangwu 80      137     3980
+xiexie 150     \N      150
+zhangsan       31      67      760
+
+-- !select_load2 --
+fangfang       120     167     4035
+lisi   50      97      1935
+wangwu 80      137     3980
+xiexie 150     \N      150
+zhangsan       31      67      760
+
+-- !select_load3 --
+fangfang       120     167     4035
+zhangsan       31      67      760
+
+-- !select_load4 --
+fangfang       120     \N      4035
+zhangsan       31      \N      760
+
+-- !select_load5 --
+fangfang       120     167     4035
+lisi   50      97      1935
+wangwu 80      137     3980
+xiexie 150     \N      150
+zhangsan       31      67      760
+
+-- !select_load6 --
+fangfang       120     167     4035
+lisi   50      97      1935
+wangwu 80      137     3980
+xiexie 150     \N      150
+zhangsan       31      67      760
+
+-- !select_load7 --
+fangfang       120     167     4035
+lisi   50      97      1935
+wangwu 80      137     3980
+xiexie 150     \N      150
+zhangsan       31      67      760
+
+-- !select_load8 --
+fangfang       120     \N      4035
+zhangsan       31      \N      760
+
diff --git 
a/regression-test/data/external_table_p0/export/test_export_external_table.out 
b/regression-test/data/external_table_p0/export/test_export_external_table.out
new file mode 100644
index 0000000000..16d43c645d
--- /dev/null
+++ 
b/regression-test/data/external_table_p0/export/test_export_external_table.out
@@ -0,0 +1,473 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !sql --
+internal
+
+-- !sql --
+mysql_jdbc_catalog
+
+-- !export_table --
+1      abc     efg     2022-10-01      3.4     1       2       0       100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       1       100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       10      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       11      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       12      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       13      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       14      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       15      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       16      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       17      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       18      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       19      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       2       100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       20      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       21      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       22      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       23      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       24      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       25      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       26      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       27      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       28      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       29      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       3       100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       30      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       31      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       32      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       33      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       34      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       35      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       36      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       37      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       38      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       39      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       4       100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       40      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       41      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       42      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       43      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       44      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       45      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       46      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       47      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       48      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       49      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       5       100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       50      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       51      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       52      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       53      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       54      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       55      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       56      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       57      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       58      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       59      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       6       100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       60      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       61      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       62      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       63      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       64      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       65      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       66      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       67      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       68      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       69      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       7       100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       70      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       71      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       72      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       73      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       74      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       75      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       76      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       77      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       78      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       79      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       8       100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       80      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       81      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       82      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       83      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       84      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       85      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       86      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       87      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       88      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       89      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       9       100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       90      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       91      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       92      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       93      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       94      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       95      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       96      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       97      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       98      100000  
1.2     2022-10-02T12:59:01     24.000
+1      abc     efg     2022-10-01      3.4     1       2       99      100000  
1.2     2022-10-02T12:59:01     24.000
+
+-- !select_load1 --
+true   abc     efg     2022-10-01      3.4     1       2       0       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       1       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       10      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       11      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       12      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       13      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       14      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       15      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       16      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       17      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       18      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       19      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       2       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       20      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       21      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       22      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       23      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       24      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       25      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       26      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       27      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       28      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       29      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       3       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       30      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       31      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       32      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       33      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       34      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       35      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       36      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       37      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       38      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       39      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       4       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       40      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       41      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       42      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       43      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       44      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       45      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       46      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       47      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       48      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       49      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       5       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       50      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       51      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       52      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       53      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       54      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       55      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       56      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       57      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       58      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       59      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       6       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       60      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       61      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       62      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       63      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       64      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       65      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       66      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       67      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       68      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       69      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       7       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       70      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       71      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       72      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       73      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       74      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       75      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       76      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       77      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       78      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       79      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       8       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       80      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       81      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       82      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       83      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       84      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       85      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       86      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       87      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       88      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       89      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       9       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       90      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       91      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       92      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       93      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       94      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       95      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       96      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       97      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       98      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       99      100000  
1.2     2022-10-02T12:59:01     24.000
+
+-- !select_load2 --
+true   abc     efg     2022-10-01      3.4     1       2       0       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       1       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       10      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       11      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       12      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       13      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       14      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       15      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       16      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       17      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       18      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       19      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       2       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       20      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       21      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       22      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       23      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       24      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       25      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       26      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       27      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       28      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       29      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       3       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       30      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       31      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       32      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       33      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       34      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       35      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       36      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       37      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       38      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       39      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       4       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       40      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       41      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       42      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       43      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       44      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       45      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       46      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       47      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       48      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       49      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       5       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       50      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       51      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       52      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       53      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       54      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       55      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       56      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       57      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       58      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       59      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       6       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       60      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       61      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       62      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       63      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       64      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       65      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       66      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       67      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       68      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       69      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       7       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       70      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       71      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       72      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       73      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       74      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       75      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       76      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       77      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       78      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       79      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       8       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       80      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       81      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       82      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       83      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       84      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       85      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       86      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       87      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       88      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       89      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       9       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       90      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       91      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       92      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       93      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       94      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       95      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       96      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       97      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       98      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       99      100000  
1.2     2022-10-02T12:59:01     24.000
+
+-- !select_load3 --
+true   abc     efg     2022-10-01      3.4     1       2       0       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       1       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       10      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       11      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       12      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       13      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       14      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       15      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       16      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       17      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       18      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       19      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       2       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       20      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       21      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       22      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       23      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       24      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       25      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       26      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       27      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       28      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       29      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       3       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       4       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       5       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       6       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       7       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       8       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       9       100000  
1.2     2022-10-02T12:59:01     24.000
+
+-- !select_load4 --
+true   abc     efg     2022-10-01      3.4     1       2       0       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       1       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       10      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       11      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       12      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       13      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       14      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       15      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       16      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       17      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       18      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       19      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       2       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       20      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       21      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       22      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       23      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       24      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       25      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       26      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       27      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       28      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       29      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       3       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       4       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       5       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       6       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       7       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       8       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       9       100000  
1.2     2022-10-02T12:59:01     24.000
+
+-- !select_load5 --
+true   abc     efg     2022-10-01      3.4     1       2       0       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       1       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       10      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       11      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       12      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       13      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       14      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       15      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       16      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       17      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       18      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       19      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       2       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       20      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       21      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       22      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       23      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       24      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       25      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       26      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       27      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       28      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       29      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       3       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       4       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       5       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       6       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       7       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       8       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       9       100000  
1.2     2022-10-02T12:59:01     24.000
+
+-- !select_load6 --
+true   abc     efg     2022-10-01      3.4     1       2       0       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       1       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       10      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       11      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       12      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       13      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       14      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       15      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       16      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       17      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       18      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       19      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       2       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       20      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       21      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       22      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       23      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       24      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       25      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       26      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       27      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       28      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       29      100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       3       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       4       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       5       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       6       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       7       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       8       100000  
1.2     2022-10-02T12:59:01     24.000
+true   abc     efg     2022-10-01      3.4     1       2       9       100000  
1.2     2022-10-02T12:59:01     24.000
+
+-- !select_load7 --
+true   \N      efg     \N      3.4     \N      2       0       \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       1       \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       10      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       11      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       12      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       13      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       14      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       15      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       16      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       17      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       18      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       19      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       2       \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       20      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       21      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       22      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       23      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       24      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       25      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       26      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       27      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       28      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       29      \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       3       \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       4       \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       5       \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       6       \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       7       \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       8       \N      \N      
\N      \N
+true   \N      efg     \N      3.4     \N      2       9       \N      \N      
\N      \N
+
diff --git a/regression-test/suites/export_p0/test_export_view.groovy 
b/regression-test/suites/export_p0/test_export_view.groovy
new file mode 100644
index 0000000000..29bc0e96c6
--- /dev/null
+++ b/regression-test/suites/export_p0/test_export_view.groovy
@@ -0,0 +1,644 @@
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+import java.nio.charset.StandardCharsets
+import java.nio.file.Files
+import java.nio.file.Paths
+
+suite("test_export_view", "p0") {
+    // open nereids
+    sql """ set enable_nereids_planner=true """
+    sql """ set enable_fallback_to_original_planner=false """
+
+    // check whether the FE config 'enable_outfile_to_local' is true
+    StringBuilder strBuilder = new StringBuilder()
+    strBuilder.append("curl --location-trusted -u " + context.config.jdbcUser 
+ ":" + context.config.jdbcPassword)
+    strBuilder.append(" http://"; + context.config.feHttpAddress + 
"/rest/v1/config/fe")
+
+    String command = strBuilder.toString()
+    def process = command.toString().execute()
+    def code = process.waitFor()
+    def err = IOGroovyMethods.getText(new BufferedReader(new 
InputStreamReader(process.getErrorStream())));
+    def out = process.getText()
+    logger.info("Request FE Config: code=" + code + ", out=" + out + ", err=" 
+ err)
+    assertEquals(code, 0)
+    def response = parseJson(out.trim())
+    assertEquals(response.code, 0)
+    assertEquals(response.msg, "success")
+    def configJson = response.data.rows
+    boolean enableOutfileToLocal = false
+    for (Object conf: configJson) {
+        assert conf instanceof Map
+        if (((Map<String, String>) conf).get("Name").toLowerCase() == 
"enable_outfile_to_local") {
+            enableOutfileToLocal = ((Map<String, String>) 
conf).get("Value").toLowerCase() == "true"
+        }
+    }
+    if (!enableOutfileToLocal) {
+        logger.warn("Please set enable_outfile_to_local to true to run 
test_outfile")
+        return
+    }
+
+    def check_path_exists = { dir_path ->
+        File path = new File(dir_path)
+        if (!path.exists()) {
+            assert path.mkdirs()
+        } else {
+            throw new IllegalStateException("""${dir_path} already exists! """)
+        }
+    }
+    def check_file_amounts = { dir_path, amount ->
+        File path = new File(dir_path)
+        File[] files = path.listFiles()
+        assert files.length == amount
+    }
+    def delete_files = { dir_path ->
+        File path = new File(dir_path)
+        if (path.exists()) {
+            for (File f: path.listFiles()) {
+                f.delete();
+            }
+            path.delete();
+        }
+    }
+    def waiting_export = { export_label ->
+        while (true) {
+            def res = sql """ show export where label = "${export_label}" """
+            logger.info("export state: " + res[0][2])
+            if (res[0][2] == "FINISHED") {
+                break;
+            } else if (res[0][2] == "CANCELLED") {
+                throw new IllegalStateException("""export failed: 
${res[0][10]}""")
+            } else {
+                sleep(5000)
+            }
+        }
+    }
+    def create_load_table = {table_name ->
+        sql """ DROP TABLE IF EXISTS ${table_name} """
+        sql """
+        CREATE TABLE IF NOT EXISTS ${table_name} (
+            `s1` varchar NULL,
+            `k1` int(11) NULL,
+            `k2` int(11) NULL,
+            `k3` int(11) NULL
+            )
+            DISTRIBUTED BY HASH(s1)
+            PROPERTIES("replication_num" = "1");
+        """
+    }
+
+    def table_export_name = "test_export_base_table"
+    def table_export_view_name = "test_export_view_table"
+    def table_load_name = "test_load_basic"
+    def outfile_path_prefix = """/tmp/test_export"""
+
+    // create table and insert
+    sql """ DROP TABLE IF EXISTS ${table_export_name} """
+    sql """
+    CREATE TABLE IF NOT EXISTS ${table_export_name} (
+        `k1` int(11) NULL,
+        `k2` string NULL,
+        `k3` int(11) NULL,
+        `v1` int(11) NULL
+        )
+        PARTITION BY RANGE(k1)
+        (
+            PARTITION less_than_20 VALUES LESS THAN ("20"),
+            PARTITION between_20_70 VALUES [("20"),("70")),
+            PARTITION more_than_70 VALUES LESS THAN ("151")
+        )
+        DISTRIBUTED BY HASH(k1) BUCKETS 3
+        PROPERTIES("replication_num" = "1");
+    """
+    StringBuilder sb = new StringBuilder()
+    int i = 1
+    for (; i < 150; i ++) {
+        if (i < 50) {
+            sb.append("""
+                    (${i}, 'zhangsan', ${i + 18}, ${i}),
+                """)
+        } else if (i < 80) {
+            sb.append("""
+                    (${i}, 'lisi', ${i + 18}, ${i}),
+                """)
+        } else if (i < 120) {
+            sb.append("""
+                    (${i}, 'wangwu', ${i + 18}, ${i}),
+                """)
+        } else {
+            sb.append("""
+                    (${i}, 'fangfang', ${i + 18}, ${i}),
+                """)
+        }
+    }
+    sb.append("""
+            (${i}, 'xiexie', NULL, ${i})
+        """)
+    sql """ INSERT INTO ${table_export_name} VALUES
+            ${sb.toString()}
+        """
+    def insert_res = sql "show last insert;"
+    logger.info("insert result: " + insert_res.toString())
+    qt_select_export """ SELECT * FROM ${table_export_name} t ORDER BY k1; """
+
+
+    sql """ DROP VIEW IF EXISTS ${table_export_view_name} """
+    sql """
+    CREATE VIEW ${table_export_view_name}
+        (
+            s1 COMMENT "first key",
+            k1 COMMENT "second key",
+            k2 COMMENT "third key",
+            k3 COMMENT "first value"
+        )
+        COMMENT "my first view"
+        AS
+        SELECT k2, min(k1), max(k3), SUM(v1) FROM ${table_export_name}
+        WHERE k1 > 30 GROUP BY k2;
+    """
+
+    // 1. basic test
+    def uuid = UUID.randomUUID().toString()
+    def outFilePath = """${outfile_path_prefix}_${uuid}"""
+    def label = "label_${uuid}"
+    try {
+        // check export path
+        check_path_exists.call("${outFilePath}")
+
+        // exec export
+        sql """
+            EXPORT TABLE ${table_export_view_name} TO "file://${outFilePath}/"
+            PROPERTIES(
+                "label" = "${label}",
+                "format" = "csv",
+                "column_separator"=","
+            );
+        """
+        waiting_export.call(label)
+        
+        // check file amounts
+        check_file_amounts.call("${outFilePath}", 1)
+
+        // check data correctness
+        create_load_table(table_load_name)
+
+        File[] files = new File("${outFilePath}").listFiles()
+        String file_path = files[0].getAbsolutePath()
+        streamLoad {
+            table "${table_load_name}"
+
+            set 'column_separator', ','
+            set 'strict_mode', 'true'
+
+            file "${file_path}"
+            time 10000 // limit inflight 10s
+
+            check { result, exception, startTime, endTime ->
+                if (exception != null) {
+                    throw exception
+                }
+                log.info("Stream load result: ${result}".toString())
+                def json = parseJson(result)
+                assertEquals("success", json.Status.toLowerCase())
+                assertEquals(5, json.NumberTotalRows)
+                assertEquals(0, json.NumberFilteredRows)
+            }
+        }
+
+        order_qt_select_load1 """ SELECT * FROM ${table_load_name} t; """
+    
+    } finally {
+        try_sql("DROP TABLE IF EXISTS ${table_load_name}")
+        delete_files.call("${outFilePath}")
+    }
+
+    // 2. test csv_with_names
+    uuid = UUID.randomUUID().toString()
+    outFilePath = """${outfile_path_prefix}_${uuid}"""
+    label = "label_${uuid}"
+    try {
+        // check export path
+        check_path_exists.call("${outFilePath}")
+
+        // exec export
+        sql """
+            EXPORT TABLE ${table_export_view_name} TO "file://${outFilePath}/"
+            PROPERTIES(
+                "label" = "${label}",
+                "format" = "csv_with_names",
+                "max_file_size" = "512MB",
+                "parallelISM" = "5",
+                "column_separator"=","
+            );
+        """
+        waiting_export.call(label)
+        
+        // check file amounts
+        check_file_amounts.call("${outFilePath}", 1)
+
+        // check data correctness
+        create_load_table(table_load_name)
+
+        File[] files = new File("${outFilePath}").listFiles()
+        String file_path = files[0].getAbsolutePath()
+        streamLoad {
+            table "${table_load_name}"
+
+            set 'column_separator', ','
+            set 'strict_mode', 'true'
+            set 'format', 'csv_with_names'
+
+            file "${file_path}"
+            time 10000 // limit inflight 10s
+
+            check { result, exception, startTime, endTime ->
+                if (exception != null) {
+                    throw exception
+                }
+                log.info("Stream load result: ${result}".toString())
+                def json = parseJson(result)
+                assertEquals("success", json.Status.toLowerCase())
+                assertEquals(5, json.NumberTotalRows)
+                assertEquals(0, json.NumberFilteredRows)
+            }
+        }
+
+        order_qt_select_load2 """ SELECT * FROM ${table_load_name} t; """
+    
+    } finally {
+        try_sql("DROP TABLE IF EXISTS ${table_load_name}")
+        delete_files.call("${outFilePath}")
+    }
+
+    
+    // 3. test where clause
+    uuid = UUID.randomUUID().toString()
+    outFilePath = """${outfile_path_prefix}_${uuid}"""
+    label = "label_${uuid}"
+    try {
+        // check export path
+        check_path_exists.call("${outFilePath}")
+
+        // exec export
+        sql """
+            EXPORT TABLE ${table_export_view_name} where s1 = 'fangfang' OR k1 
= 31
+            TO "file://${outFilePath}/"
+            PROPERTIES(
+                "label" = "${label}",
+                "format" = "csv_with_names",
+                "max_file_size" = "512MB",
+                "parallelISM" = "5",
+                "column_separator"=","
+            );
+        """
+        waiting_export.call(label)
+        
+        // check file amounts
+        check_file_amounts.call("${outFilePath}", 1)
+
+        // check data correctness
+        create_load_table(table_load_name)
+
+        File[] files = new File("${outFilePath}").listFiles()
+        String file_path = files[0].getAbsolutePath()
+        streamLoad {
+            table "${table_load_name}"
+
+            set 'column_separator', ','
+            set 'strict_mode', 'true'
+            set 'format', 'csv_with_names'
+
+            file "${file_path}"
+            time 10000 // limit inflight 10s
+
+            check { result, exception, startTime, endTime ->
+                if (exception != null) {
+                    throw exception
+                }
+                log.info("Stream load result: ${result}".toString())
+                def json = parseJson(result)
+                assertEquals("success", json.Status.toLowerCase())
+                assertEquals(2, json.NumberTotalRows)
+                assertEquals(0, json.NumberFilteredRows)
+            }
+        }
+
+        order_qt_select_load3 """ SELECT * FROM ${table_load_name} t; """
+    
+    } finally {
+        try_sql("DROP TABLE IF EXISTS ${table_load_name}")
+        delete_files.call("${outFilePath}")
+    }
+
+
+    // 4. test where clause and columns property
+    uuid = UUID.randomUUID().toString()
+    outFilePath = """${outfile_path_prefix}_${uuid}"""
+    label = "label_${uuid}"
+    try {
+        // check export path
+        check_path_exists.call("${outFilePath}")
+
+        // exec export
+        sql """
+            EXPORT TABLE ${table_export_view_name} where s1 = 'fangfang' OR k1 
= 31
+            TO "file://${outFilePath}/"
+            PROPERTIES(
+                "label" = "${label}",
+                "format" = "csv_with_names",
+                "max_file_size" = "512MB",
+                "parallelISM" = "5",
+                "columns" = "k3, s1, k1",
+                "column_separator"=","
+            );
+        """
+
+        waiting_export.call(label)
+        
+        // check file amounts
+        check_file_amounts.call("${outFilePath}", 1)
+
+        // check data correctness
+        create_load_table(table_load_name)
+
+        File[] files = new File("${outFilePath}").listFiles()
+        String file_path = files[0].getAbsolutePath()
+        streamLoad {
+            table "${table_load_name}"
+
+            set 'column_separator', ','
+            set 'strict_mode', 'true'
+            set 'columns', 'k3, s1, k1'
+            set 'format', 'csv_with_names'
+
+            file "${file_path}"
+            time 10000 // limit inflight 10s
+
+            check { result, exception, startTime, endTime ->
+                if (exception != null) {
+                    throw exception
+                }
+                log.info("Stream load result: ${result}".toString())
+                def json = parseJson(result)
+                assertEquals("success", json.Status.toLowerCase())
+                assertEquals(2, json.NumberTotalRows)
+                assertEquals(0, json.NumberFilteredRows)
+            }
+        }
+
+        order_qt_select_load4 """ SELECT * FROM ${table_load_name} t; """
+    
+    } finally {
+        try_sql("DROP TABLE IF EXISTS ${table_load_name}")
+        delete_files.call("${outFilePath}")
+    }
+
+
+    // 5. test csv_with_names_and_types
+    uuid = UUID.randomUUID().toString()
+    outFilePath = """${outfile_path_prefix}_${uuid}"""
+    label = "label_${uuid}"
+    try {
+        // check export path
+        check_path_exists.call("${outFilePath}")
+
+        // exec export
+        sql """
+            EXPORT TABLE ${table_export_view_name} TO "file://${outFilePath}/"
+            PROPERTIES(
+                "label" = "${label}",
+                "format" = "csv_with_names_and_types",
+                "max_file_size" = "512MB",
+                "parallelISM" = "5",
+                "column_separator"=","
+            );
+        """
+        waiting_export.call(label)
+        
+        // check file amounts
+        check_file_amounts.call("${outFilePath}", 1)
+
+        // check data correctness
+        create_load_table(table_load_name)
+
+        File[] files = new File("${outFilePath}").listFiles()
+        String file_path = files[0].getAbsolutePath()
+        streamLoad {
+            table "${table_load_name}"
+
+            set 'strict_mode', 'true'
+            set 'format', 'csv_with_names_and_types'
+            set 'column_separator', ','
+
+            file "${file_path}"
+            time 10000 // limit inflight 10s
+
+            check { result, exception, startTime, endTime ->
+                if (exception != null) {
+                    throw exception
+                }
+                log.info("Stream load result: ${result}".toString())
+                def json = parseJson(result)
+                assertEquals("success", json.Status.toLowerCase())
+                assertEquals(5, json.NumberTotalRows)
+                assertEquals(0, json.NumberFilteredRows)
+            }
+        }
+
+        order_qt_select_load5 """ SELECT * FROM ${table_load_name} t; """
+    
+    } finally {
+        try_sql("DROP TABLE IF EXISTS ${table_load_name}")
+        delete_files.call("${outFilePath}")
+    }
+
+
+    // 6. test orc type
+    uuid = UUID.randomUUID().toString()
+    outFilePath = """${outfile_path_prefix}_${uuid}"""
+    label = "label_${uuid}"
+    try {
+        // check export path
+        check_path_exists.call("${outFilePath}")
+
+        // exec export
+        sql """
+            EXPORT TABLE ${table_export_view_name} TO "file://${outFilePath}/"
+            PROPERTIES(
+                "label" = "${label}",
+                "format" = "orc",
+                "max_file_size" = "512MB"
+            );
+        """
+        waiting_export.call(label)
+        
+        // check file amounts
+        check_file_amounts.call("${outFilePath}", 1)
+
+        // check data correctness
+        create_load_table(table_load_name)
+
+        File[] files = new File("${outFilePath}").listFiles()
+        String file_path = files[0].getAbsolutePath()
+        streamLoad {
+            table "${table_load_name}"
+
+            set 'strict_mode', 'true'
+            set 'format', 'orc'
+
+            file "${file_path}"
+            time 10000 // limit inflight 10s
+
+            check { result, exception, startTime, endTime ->
+                if (exception != null) {
+                    throw exception
+                }
+                log.info("Stream load result: ${result}".toString())
+                def json = parseJson(result)
+                assertEquals("success", json.Status.toLowerCase())
+                assertEquals(5, json.NumberTotalRows)
+                assertEquals(0, json.NumberFilteredRows)
+            }
+        }
+
+        order_qt_select_load6 """ SELECT * FROM ${table_load_name} t; """
+    
+    } finally {
+        try_sql("DROP TABLE IF EXISTS ${table_load_name}")
+        delete_files.call("${outFilePath}")
+    }
+
+
+    // 7. test parquet type
+    uuid = UUID.randomUUID().toString()
+    outFilePath = """${outfile_path_prefix}_${uuid}"""
+    label = "label_${uuid}"
+    try {
+        // check export path
+        check_path_exists.call("${outFilePath}")
+
+        // exec export
+        sql """
+            EXPORT TABLE ${table_export_view_name} TO "file://${outFilePath}/"
+            PROPERTIES(
+                "label" = "${label}",
+                "format" = "parquet",
+                "max_file_size" = "512MB",
+                "parallelISM" = "5",
+                "column_separator"=","
+            );
+        """
+        waiting_export.call(label)
+        
+        // check file amounts
+        check_file_amounts.call("${outFilePath}", 1)
+
+        // check data correctness
+        create_load_table(table_load_name)
+
+        File[] files = new File("${outFilePath}").listFiles()
+        String file_path = files[0].getAbsolutePath()
+        streamLoad {
+            table "${table_load_name}"
+
+            set 'strict_mode', 'true'
+            set 'format', 'parquet'
+
+            file "${file_path}"
+            time 10000 // limit inflight 10s
+
+            check { result, exception, startTime, endTime ->
+                if (exception != null) {
+                    throw exception
+                }
+                log.info("Stream load result: ${result}".toString())
+                def json = parseJson(result)
+                assertEquals("success", json.Status.toLowerCase())
+                assertEquals(5, json.NumberTotalRows)
+                assertEquals(0, json.NumberFilteredRows)
+            }
+        }
+
+        order_qt_select_load7 """ SELECT * FROM ${table_load_name} t; """
+    
+    } finally {
+        try_sql("DROP TABLE IF EXISTS ${table_load_name}")
+        delete_files.call("${outFilePath}")
+    }
+
+    // 8. test orc type, where clause and columns property
+    uuid = UUID.randomUUID().toString()
+    outFilePath = """${outfile_path_prefix}_${uuid}"""
+    label = "label_${uuid}"
+    try {
+        // check export path
+        check_path_exists.call("${outFilePath}")
+
+        // exec export
+        sql """
+            EXPORT TABLE ${table_export_view_name} where s1 = 'fangfang' OR k1 
= 31
+            TO "file://${outFilePath}/"
+            PROPERTIES(
+                "label" = "${label}",
+                "format" = "orc",
+                "max_file_size" = "512MB",
+                "parallelISM" = "5",
+                "columns" = "k3, s1, k1"
+            );
+        """
+
+        waiting_export.call(label)
+        
+        // check file amounts
+        check_file_amounts.call("${outFilePath}", 1)
+
+        // check data correctness
+        create_load_table(table_load_name)
+
+        File[] files = new File("${outFilePath}").listFiles()
+        String file_path = files[0].getAbsolutePath()
+        streamLoad {
+            table "${table_load_name}"
+
+            set 'strict_mode', 'true'
+            set 'columns', 'k3, s1, k1'
+            set 'format', 'orc'
+
+            file "${file_path}"
+            time 10000 // limit inflight 10s
+
+            check { result, exception, startTime, endTime ->
+                if (exception != null) {
+                    throw exception
+                }
+                log.info("Stream load result: ${result}".toString())
+                def json = parseJson(result)
+                assertEquals("success", json.Status.toLowerCase())
+                assertEquals(2, json.NumberTotalRows)
+                assertEquals(0, json.NumberFilteredRows)
+            }
+        }
+
+        order_qt_select_load8 """ SELECT * FROM ${table_load_name} t; """
+    
+    } finally {
+        try_sql("DROP TABLE IF EXISTS ${table_load_name}")
+        delete_files.call("${outFilePath}")
+    }
+}
\ No newline at end of file
diff --git 
a/regression-test/suites/external_table_p0/export/test_export_external_table.groovy
 
b/regression-test/suites/external_table_p0/export/test_export_external_table.groovy
new file mode 100644
index 0000000000..adbabe6340
--- /dev/null
+++ 
b/regression-test/suites/external_table_p0/export/test_export_external_table.groovy
@@ -0,0 +1,552 @@
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+import java.nio.charset.StandardCharsets
+import java.nio.file.Files
+import java.nio.file.Paths
+
+suite("test_export_external_table", 
"p0,external,mysql,external_docker,external_docker_mysql") {
+    // open nereids
+    sql """ set enable_nereids_planner=true """
+    sql """ set enable_fallback_to_original_planner=false """
+
+    // check whether the FE config 'enable_outfile_to_local' is true
+    StringBuilder strBuilder = new StringBuilder()
+    strBuilder.append("curl --location-trusted -u " + context.config.jdbcUser 
+ ":" + context.config.jdbcPassword)
+    strBuilder.append(" http://"; + context.config.feHttpAddress + 
"/rest/v1/config/fe")
+
+    String command = strBuilder.toString()
+    def process = command.toString().execute()
+    def code = process.waitFor()
+    def err = IOGroovyMethods.getText(new BufferedReader(new 
InputStreamReader(process.getErrorStream())));
+    def out = process.getText()
+    logger.info("Request FE Config: code=" + code + ", out=" + out + ", err=" 
+ err)
+    assertEquals(code, 0)
+    def response = parseJson(out.trim())
+    assertEquals(response.code, 0)
+    assertEquals(response.msg, "success")
+    def configJson = response.data.rows
+    boolean enableOutfileToLocal = false
+    for (Object conf: configJson) {
+        assert conf instanceof Map
+        if (((Map<String, String>) conf).get("Name").toLowerCase() == 
"enable_outfile_to_local") {
+            enableOutfileToLocal = ((Map<String, String>) 
conf).get("Value").toLowerCase() == "true"
+        }
+    }
+    if (!enableOutfileToLocal) {
+        logger.warn("Please set enable_outfile_to_local to true to run 
test_outfile")
+        return
+    }
+
+    def check_path_exists = { dir_path ->
+        File path = new File(dir_path)
+        if (!path.exists()) {
+            assert path.mkdirs()
+        } else {
+            throw new IllegalStateException("""${dir_path} already exists! """)
+        }
+    }
+    def check_file_amounts = { dir_path, amount ->
+        File path = new File(dir_path)
+        File[] files = path.listFiles()
+        assert files.length == amount
+    }
+    def delete_files = { dir_path ->
+        File path = new File(dir_path)
+        if (path.exists()) {
+            for (File f: path.listFiles()) {
+                f.delete();
+            }
+            path.delete();
+        }
+    }
+    def waiting_export = { export_label ->
+        while (true) {
+            def res = sql """ show export where label = "${export_label}" """
+            logger.info("export state: " + res[0][2])
+            if (res[0][2] == "FINISHED") {
+                break;
+            } else if (res[0][2] == "CANCELLED") {
+                throw new IllegalStateException("""export failed: 
${res[0][10]}""")
+            } else {
+                sleep(5000)
+            }
+        }
+    }
+    
+
+
+    def table_export_name = "test1"
+    def table_load_name = "test_load_basic"
+    def outfile_path_prefix = """/tmp/test_export"""
+
+    String enabled = context.config.otherConfigs.get("enableJdbcTest")
+    String externalEnvIp = context.config.otherConfigs.get("externalEnvIp")
+    String s3_endpoint = getS3Endpoint()
+    String bucket = getS3BucketName()
+    String driver_url = 
"https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-java-8.0.25.jar";
+
+    if (enabled != null && enabled.equalsIgnoreCase("true")) {
+        String catalog_name = "mysql_jdbc_catalog";
+        String internal_db_name = "regression_test_external_table_p0_export";
+        String ex_db_name = "doris_test";
+        String mysql_port = context.config.otherConfigs.get("mysql_57_port");
+
+        def create_load_table = {table_name ->
+            sql """ DROP TABLE IF EXISTS 
internal.${internal_db_name}.${table_name} """
+            sql """
+            CREATE TABLE IF NOT EXISTS 
internal.${internal_db_name}.${table_name} (
+                    k1 boolean,
+                    k2 char(100),
+                    k3 varchar(128),
+                    k4 date,
+                    k5 float,
+                    k6 tinyint,
+                    k7 smallint,
+                    k8 int,
+                    k9 bigint,
+                    k10 double,
+                    k11 datetime,
+                    k12 decimal(10, 3)
+                )
+                DISTRIBUTED BY HASH(k8)
+                PROPERTIES("replication_num" = "1");
+            """
+        }
+
+        sql """create database if not exists ${internal_db_name}; """
+
+        sql """drop catalog if exists ${catalog_name} """
+        sql """create catalog if not exists ${catalog_name} properties(
+            "type"="jdbc",
+            "user"="root",
+            "password"="123456",
+            "jdbc_url" = 
"jdbc:mysql://${externalEnvIp}:${mysql_port}/doris_test?useSSL=false&zeroDateTimeBehavior=convertToNull",
+            "driver_url" = "${driver_url}",
+            "driver_class" = "com.mysql.cj.jdbc.Driver"
+        );"""
+
+        sql  """ use ${internal_db_name} """
+
+        qt_sql """select current_catalog()"""
+        sql """switch ${catalog_name}"""
+        qt_sql """select current_catalog()"""
+        sql """ use ${ex_db_name}"""
+
+        order_qt_export_table """ select * from ${table_export_name} where k8 
< 100 order by k8; """
+
+        // 1. basic test
+        def uuid = UUID.randomUUID().toString()
+        def outFilePath = """${outfile_path_prefix}_${uuid}"""
+        def label = "label_${uuid}"
+        try {
+            // check export path
+            check_path_exists.call("${outFilePath}")
+
+            // exec export
+            sql """
+                EXPORT TABLE ${table_export_name} where k8 < 100
+                TO "file://${outFilePath}/"
+                PROPERTIES(
+                    "label" = "${label}",
+                    "format" = "csv",
+                    "column_separator"=","
+                );
+            """
+            waiting_export.call(label)
+            
+            // check file amounts
+            check_file_amounts.call("${outFilePath}", 1)
+
+            // check data correctness
+            create_load_table(table_load_name)
+
+            File[] files = new File("${outFilePath}").listFiles()
+            String file_path = files[0].getAbsolutePath()
+            streamLoad {
+                table "${table_load_name}"
+
+                set 'column_separator', ','
+                set 'strict_mode', 'true'
+
+                file "${file_path}"
+                time 10000 // limit inflight 10s
+
+                check { result, exception, startTime, endTime ->
+                    if (exception != null) {
+                        throw exception
+                    }
+                    log.info("Stream load result: ${result}".toString())
+                    def json = parseJson(result)
+                    assertEquals("success", json.Status.toLowerCase())
+                    assertEquals(100, json.NumberTotalRows)
+                    assertEquals(0, json.NumberFilteredRows)
+                }
+            }
+
+            order_qt_select_load1 """ SELECT * FROM 
internal.${internal_db_name}.${table_load_name} order by k8; """
+        
+        } finally {
+            delete_files.call("${outFilePath}")
+        }
+
+        // 2. export external table under internal catalog
+        uuid = UUID.randomUUID().toString()
+        outFilePath = """${outfile_path_prefix}_${uuid}"""
+        label = "label_${uuid}"
+        try {
+            // check export path
+            check_path_exists.call("${outFilePath}")
+            
+            sql """ switch internal """
+            // exec export
+            sql """
+                EXPORT TABLE 
${catalog_name}.${ex_db_name}.${table_export_name} where k8 < 100
+                TO "file://${outFilePath}/"
+                PROPERTIES(
+                    "label" = "${label}",
+                    "format" = "csv",
+                    "column_separator"=","
+                );
+            """
+
+            sql """ switch ${catalog_name} """
+            sql """ use ${ex_db_name} """
+
+            waiting_export.call(label)
+            
+            // check file amounts
+            check_file_amounts.call("${outFilePath}", 1)
+
+            // check data correctness
+            create_load_table(table_load_name)
+
+            File[] files = new File("${outFilePath}").listFiles()
+            String file_path = files[0].getAbsolutePath()
+            streamLoad {
+                table "${table_load_name}"
+
+                set 'column_separator', ','
+                set 'strict_mode', 'true'
+
+                file "${file_path}"
+                time 10000 // limit inflight 10s
+
+                check { result, exception, startTime, endTime ->
+                    if (exception != null) {
+                        throw exception
+                    }
+                    log.info("Stream load result: ${result}".toString())
+                    def json = parseJson(result)
+                    assertEquals("success", json.Status.toLowerCase())
+                    assertEquals(100, json.NumberTotalRows)
+                    assertEquals(0, json.NumberFilteredRows)
+                }
+            }
+
+            order_qt_select_load2 """ SELECT * FROM 
internal.${internal_db_name}.${table_load_name} order by k8; """
+        
+        } finally {
+            delete_files.call("${outFilePath}")
+        }
+
+        sql """ switch ${catalog_name} """
+        // 3. csv_with_names
+        uuid = UUID.randomUUID().toString()
+        outFilePath = """${outfile_path_prefix}_${uuid}"""
+        label = "label_${uuid}"
+        try {
+            // check export path
+            check_path_exists.call("${outFilePath}")
+
+            // exec export
+            sql """
+                EXPORT TABLE ${table_export_name} where k8 < 30
+                TO "file://${outFilePath}/"
+                PROPERTIES(
+                    "label" = "${label}",
+                    "format" = "csv_with_names",
+                    "column_separator"=","
+                );
+            """
+            waiting_export.call(label)
+            
+            // check file amounts
+            check_file_amounts.call("${outFilePath}", 1)
+
+            // check data correctness
+            create_load_table(table_load_name)
+
+            File[] files = new File("${outFilePath}").listFiles()
+            String file_path = files[0].getAbsolutePath()
+            streamLoad {
+                table "${table_load_name}"
+
+                set 'column_separator', ','
+                set 'strict_mode', 'true'
+                set 'format', 'csv_with_names'
+
+                file "${file_path}"
+                time 10000 // limit inflight 10s
+
+                check { result, exception, startTime, endTime ->
+                    if (exception != null) {
+                        throw exception
+                    }
+                    log.info("Stream load result: ${result}".toString())
+                    def json = parseJson(result)
+                    assertEquals("success", json.Status.toLowerCase())
+                    assertEquals(30, json.NumberTotalRows)
+                    assertEquals(0, json.NumberFilteredRows)
+                }
+            }
+
+            order_qt_select_load3 """ SELECT * FROM 
internal.${internal_db_name}.${table_load_name} order by k8; """
+        
+        } finally {
+            delete_files.call("${outFilePath}")
+        }
+
+
+        // 4. csv_with_names_and_types
+        uuid = UUID.randomUUID().toString()
+        outFilePath = """${outfile_path_prefix}_${uuid}"""
+        label = "label_${uuid}"
+        try {
+            // check export path
+            check_path_exists.call("${outFilePath}")
+
+            // exec export
+            sql """
+                EXPORT TABLE ${table_export_name} where k8 < 30
+                TO "file://${outFilePath}/"
+                PROPERTIES(
+                    "label" = "${label}",
+                    "format" = "csv_with_names_and_types",
+                    "column_separator"=","
+                );
+            """
+            waiting_export.call(label)
+            
+            // check file amounts
+            check_file_amounts.call("${outFilePath}", 1)
+
+            // check data correctness
+            create_load_table(table_load_name)
+
+            File[] files = new File("${outFilePath}").listFiles()
+            String file_path = files[0].getAbsolutePath()
+            streamLoad {
+                table "${table_load_name}"
+
+                set 'column_separator', ','
+                set 'strict_mode', 'true'
+                set 'format', 'csv_with_names_and_types'
+
+                file "${file_path}"
+                time 10000 // limit inflight 10s
+
+                check { result, exception, startTime, endTime ->
+                    if (exception != null) {
+                        throw exception
+                    }
+                    log.info("Stream load result: ${result}".toString())
+                    def json = parseJson(result)
+                    assertEquals("success", json.Status.toLowerCase())
+                    assertEquals(30, json.NumberTotalRows)
+                    assertEquals(0, json.NumberFilteredRows)
+                }
+            }
+
+            order_qt_select_load4 """ SELECT * FROM 
internal.${internal_db_name}.${table_load_name} order by k8; """
+        
+        } finally {
+            delete_files.call("${outFilePath}")
+        }
+
+
+        // 5. orc
+        uuid = UUID.randomUUID().toString()
+        outFilePath = """${outfile_path_prefix}_${uuid}"""
+        label = "label_${uuid}"
+        try {
+            // check export path
+            check_path_exists.call("${outFilePath}")
+
+            // exec export
+            sql """
+                EXPORT TABLE ${table_export_name} where k8 < 30
+                TO "file://${outFilePath}/"
+                PROPERTIES(
+                    "label" = "${label}",
+                    "format" = "orc"
+                );
+            """
+            waiting_export.call(label)
+            
+            // check file amounts
+            check_file_amounts.call("${outFilePath}", 1)
+
+            // check data correctness
+            create_load_table(table_load_name)
+
+            File[] files = new File("${outFilePath}").listFiles()
+            String file_path = files[0].getAbsolutePath()
+            streamLoad {
+                table "${table_load_name}"
+
+                set 'strict_mode', 'true'
+                set 'format', 'orc'
+
+                file "${file_path}"
+                time 10000 // limit inflight 10s
+
+                check { result, exception, startTime, endTime ->
+                    if (exception != null) {
+                        throw exception
+                    }
+                    log.info("Stream load result: ${result}".toString())
+                    def json = parseJson(result)
+                    assertEquals("success", json.Status.toLowerCase())
+                    assertEquals(30, json.NumberTotalRows)
+                    assertEquals(0, json.NumberFilteredRows)
+                }
+            }
+
+            order_qt_select_load5 """ SELECT * FROM 
internal.${internal_db_name}.${table_load_name} order by k8; """
+        
+        } finally {
+            delete_files.call("${outFilePath}")
+        }
+
+
+        // 5. parquet
+        uuid = UUID.randomUUID().toString()
+        outFilePath = """${outfile_path_prefix}_${uuid}"""
+        label = "label_${uuid}"
+        try {
+            // check export path
+            check_path_exists.call("${outFilePath}")
+
+            // exec export
+            sql """
+                EXPORT TABLE ${table_export_name} where k8 < 30
+                TO "file://${outFilePath}/"
+                PROPERTIES(
+                    "label" = "${label}",
+                    "format" = "parquet"
+                );
+            """
+            waiting_export.call(label)
+            
+            // check file amounts
+            check_file_amounts.call("${outFilePath}", 1)
+
+            // check data correctness
+            create_load_table(table_load_name)
+
+            File[] files = new File("${outFilePath}").listFiles()
+            String file_path = files[0].getAbsolutePath()
+            streamLoad {
+                table "${table_load_name}"
+
+                set 'strict_mode', 'true'
+                set 'format', 'parquet'
+
+                file "${file_path}"
+                time 10000 // limit inflight 10s
+
+                check { result, exception, startTime, endTime ->
+                    if (exception != null) {
+                        throw exception
+                    }
+                    log.info("Stream load result: ${result}".toString())
+                    def json = parseJson(result)
+                    assertEquals("success", json.Status.toLowerCase())
+                    assertEquals(30, json.NumberTotalRows)
+                    assertEquals(0, json.NumberFilteredRows)
+                }
+            }
+
+            order_qt_select_load6 """ SELECT * FROM 
internal.${internal_db_name}.${table_load_name} order by k8; """
+        
+        } finally {
+            delete_files.call("${outFilePath}")
+        }
+
+
+        // 7. test columns property
+        uuid = UUID.randomUUID().toString()
+        outFilePath = """${outfile_path_prefix}_${uuid}"""
+        label = "label_${uuid}"
+        try {
+            // check export path
+            check_path_exists.call("${outFilePath}")
+
+            // exec export
+            sql """
+                EXPORT TABLE ${table_export_name} where k8 < 30
+                TO "file://${outFilePath}/"
+                PROPERTIES(
+                    "label" = "${label}",
+                    "format" = "csv_with_names",
+                    "columns" = "k8, k1, k5, k3, k7",
+                    "column_separator"=","
+                );
+            """
+            waiting_export.call(label)
+            
+            // check file amounts
+            check_file_amounts.call("${outFilePath}", 1)
+
+            // check data correctness
+            create_load_table(table_load_name)
+
+            File[] files = new File("${outFilePath}").listFiles()
+            String file_path = files[0].getAbsolutePath()
+            streamLoad {
+                table "${table_load_name}"
+
+                set 'column_separator', ','
+                set 'strict_mode', 'true'
+                set 'format', 'csv_with_names'
+                set 'columns', 'k8, k1, k5, k3, k7'
+
+                file "${file_path}"
+                time 10000 // limit inflight 10s
+
+                check { result, exception, startTime, endTime ->
+                    if (exception != null) {
+                        throw exception
+                    }
+                    log.info("Stream load result: ${result}".toString())
+                    def json = parseJson(result)
+                    assertEquals("success", json.Status.toLowerCase())
+                    assertEquals(30, json.NumberTotalRows)
+                    assertEquals(0, json.NumberFilteredRows)
+                }
+            }
+
+            order_qt_select_load7 """ SELECT * FROM 
internal.${internal_db_name}.${table_load_name} order by k8; """
+        
+        } finally {
+            delete_files.call("${outFilePath}")
+        }
+    }
+}


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

Reply via email to