You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@linkis.apache.org by pe...@apache.org on 2023/05/06 02:44:20 UTC

[linkis] branch dev-1.4.0 updated: feat: Reduce scala modules and publish jar packages with scala versions at the same time(#4192) (#4459)

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

peacewong pushed a commit to branch dev-1.4.0
in repository https://gitbox.apache.org/repos/asf/linkis.git


The following commit(s) were added to refs/heads/dev-1.4.0 by this push:
     new 29cf010d2 feat: Reduce scala modules and publish jar packages with scala versions at the same time(#4192) (#4459)
29cf010d2 is described below

commit 29cf010d20c89fb0f65761f1503b5e38ae1946e4
Author: CharlieYan <16...@users.noreply.github.com>
AuthorDate: Sat May 6 10:44:12 2023 +0800

    feat: Reduce scala modules and publish jar packages with scala versions at the same time(#4192) (#4459)
---
 .../java/org/apache/linkis/common/io/FsPath.java   |   3 +-
 .../java/org/apache/linkis/storage/FSFactory.java  |  96 ++++++
 .../org/apache/linkis/storage/LineMetaData.java}   |  30 +-
 .../org/apache/linkis/storage/LineRecord.java}     |  33 +-
 .../linkis/storage/conf/LinkisStorageConf.java     |  65 ++++
 .../apache/linkis/storage/csv/CSVFsWriter.java}    |  21 +-
 .../linkis/storage/csv/StorageCSVWriter.java       | 128 +++++++
 .../org/apache/linkis/storage/domain/Column.java   |  35 +-
 .../org/apache/linkis/storage/domain/DataType.java | 226 ++++++++++++
 .../org/apache/linkis/storage/domain/Dolphin.java  | 130 +++++++
 .../apache/linkis/storage/domain/MethodEntity.java | 132 +++++++
 .../storage/domain/MethodEntitySerializer.java     |  84 +++++
 .../storage/excel/ExcelAnalysisException.java}     |  19 +-
 .../linkis/storage/excel/ExcelFsWriter.java}       |  27 +-
 .../linkis/storage/excel/ExcelXlsReader.java       | 278 +++++++++++++++
 .../apache/linkis/storage/excel/FirstRowDeal.java  |  55 +++
 .../linkis/storage/excel/IExcelRowDeal.java}       |  10 +-
 .../apache/linkis/storage/excel/RowToCsvDeal.java  |  66 ++++
 .../linkis/storage/excel/StorageExcelWriter.java   | 310 +++++++++++++++++
 .../storage/excel/StorageMultiExcelWriter.java}    |  43 +--
 .../storage/factory/impl/BuildHDFSFileSystem.java  |   6 +-
 .../storage/factory/impl/BuildLocalFileSystem.java |   8 +-
 .../storage/factory/impl/BuildOSSSystem.java       |   2 +-
 .../storage/factory/impl/BuildS3FileSystem.java    |   2 +-
 .../linkis/storage/fs/impl/HDFSFileSystem.java     |  21 +-
 .../linkis/storage/fs/impl/LocalFileSystem.java    |  12 +-
 .../linkis/storage/fs/impl/OSSFileSystem.java      |  23 +-
 .../linkis/storage/fs/impl/S3FileSystem.java       |  16 +-
 .../org/apache/linkis/storage/io/IOClient.java}    |  14 +-
 .../apache/linkis/storage/io/IOClientFactory.java} |  46 +--
 .../storage/io/IOMethodInterceptorCreator.java}    |   9 +-
 .../storage/io/IOMethodInterceptorFactory.java}    |  36 +-
 .../storage/resultset/DefaultResultSetFactory.java | 188 ++++++++++
 .../linkis/storage/resultset/ResultMetaData.java}  |   6 +-
 .../linkis/storage/resultset/ResultRecord.java}    |   6 +-
 .../linkis/storage/resultset/ResultSetFactory.java |  74 ++++
 .../storage/resultset/ResultSetReaderFactory.java  | 115 +++++++
 .../storage/resultset/ResultSetWriterFactory.java  |  88 +++++
 .../linkis/storage/resultset/StorageResultSet.java |  79 +++++
 .../storage/resultset/StorageResultSetReader.java  | 180 ++++++++++
 .../storage/resultset/StorageResultSetWriter.java  | 260 ++++++++++++++
 .../storage/resultset/html/HtmlResultSet.java      |  48 +++
 .../linkis/storage/resultset/io/IOMetaData.java}   |  21 +-
 .../linkis/storage/resultset/io/IORecord.java}     |  19 +-
 .../storage/resultset/io/IOResultDeserializer.java |  21 +-
 .../storage/resultset/io/IOResultSerializer.java   |  47 +++
 .../linkis/storage/resultset/io/IOResultSet.java   |  27 +-
 .../resultset/picture/PictureResultSet.java        |  48 +++
 .../storage/resultset/table/TableMetaData.java     |  28 +-
 .../storage/resultset/table/TableRecord.java       |  28 +-
 .../resultset/table/TableResultDeserializer.java   | 100 ++++++
 .../resultset/table/TableResultSerializer.java     | 111 ++++++
 .../storage/resultset/table/TableResultSet.java    |  28 +-
 .../resultset/txt/TextResultDeserializer.java      |  20 +-
 .../resultset/txt/TextResultSerializer.java        |  50 +++
 .../storage/resultset/txt/TextResultSet.java       |  46 +++
 .../apache/linkis/storage/script/Compaction.java}  |  30 +-
 .../org/apache/linkis/storage/script/Parser.java}  |  14 +-
 .../linkis/storage/script/ParserFactory.java}      |  23 +-
 .../linkis/storage/script/ScriptFsReader.java}     |  29 +-
 .../linkis/storage/script/ScriptFsWriter.java}     |  25 +-
 .../linkis/storage/script/ScriptMetaData.java}     |  32 +-
 .../linkis/storage/script/ScriptRecord.java}       |  10 +-
 .../apache/linkis/storage/script/Variable.java}    |  34 +-
 .../linkis/storage/script/VariableParser.java      | 131 +++++++
 .../script/compaction/CommonScriptCompaction.java  |  31 +-
 .../script/compaction/PYScriptCompaction.java      |  29 +-
 .../script/compaction/QLScriptCompaction.java      |  29 +-
 .../script/compaction/ScalaScriptCompaction.java   |  29 +-
 .../script/compaction/ShellScriptCompaction.java   |  29 +-
 .../storage/script/parser/CommonScriptParser.java  |  75 ++++
 .../storage/script/parser/PYScriptParser.java      |  29 +-
 .../storage/script/parser/QLScriptParser.java}     |  29 +-
 .../storage/script/parser/ScalaScriptParser.java   |  29 +-
 .../storage/script/parser/ShellScriptParser.java   |  29 +-
 .../script/reader/StorageScriptFsReader.java       | 161 +++++++++
 .../script/writer/StorageScriptFsWriter.java       | 124 +++++++
 .../linkis/storage/source/AbstractFileSource.java  | 109 ++++++
 .../apache/linkis/storage/source/FileSource.java   | 172 ++++++++++
 .../apache/linkis/storage/source/FileSplit.java    | 281 +++++++++++++++
 .../linkis/storage/source/ResultsetFileSource.java |  54 +++
 .../linkis/storage/source/TextFileSource.java      |  71 ++++
 .../linkis/storage/utils/FileSystemUtils.java      | 136 ++++++++
 .../linkis/storage/utils/StorageConfiguration.java | 158 +++++++++
 .../apache/linkis/storage/utils/StorageHelper.java | 156 +++++++++
 .../apache/linkis/storage/utils/StorageUtils.java  | 270 +++++++++++++++
 .../org/apache/linkis/storage/FSFactory.scala      |  98 ------
 .../linkis/storage/conf/LinkisStorageConf.scala    |  63 ----
 .../linkis/storage/csv/StorageCSVReader.scala      |  20 --
 .../linkis/storage/csv/StorageCSVWriter.scala      |  93 -----
 .../apache/linkis/storage/domain/DataType.scala    | 148 --------
 .../org/apache/linkis/storage/domain/Dolphin.scala | 104 ------
 .../linkis/storage/domain/MethodEntity.scala       |  97 ------
 .../linkis/storage/excel/ExcelFsWriter.scala       |  47 ---
 .../linkis/storage/excel/ExcelXlsReader.java       | 378 ---------------------
 .../linkis/storage/excel/StorageExcelReader.scala  |  20 --
 .../linkis/storage/excel/StorageExcelWriter.scala  | 236 -------------
 .../org/apache/linkis/storage/io/IOClient.scala    |  75 ----
 .../resultset/DefaultResultSetFactory.scala        | 130 -------
 .../storage/resultset/ResultSetFactory.scala       |  69 ----
 .../linkis/storage/resultset/ResultSetReader.scala |  93 -----
 .../linkis/storage/resultset/ResultSetWriter.scala |  74 ----
 .../storage/resultset/StorageResultSet.scala       |  51 ---
 .../storage/resultset/StorageResultSetReader.scala | 154 ---------
 .../storage/resultset/StorageResultSetWriter.scala | 222 ------------
 .../storage/resultset/html/HtmlResultSet.scala     |  34 --
 .../storage/resultset/io/IOResultSerializer.scala  |  43 ---
 .../linkis/storage/resultset/io/IOResultSet.scala  |  32 --
 .../resultset/picture/PictureResultSet.scala       |  34 --
 .../resultset/table/TableResultDeserializer.scala  |  90 -----
 .../resultset/table/TableResultSerializer.scala    |  92 -----
 .../storage/resultset/table/TableResultSet.scala   |  32 --
 .../resultset/txt/TextResultSerializer.scala       |  46 ---
 .../storage/resultset/txt/TextResultSet.scala      |  33 --
 .../linkis/storage/script/ScriptFsWriter.scala     | 115 -------
 .../linkis/storage/script/VariableParser.scala     | 117 -------
 .../script/compaction/PYScriptCompaction.scala     |  39 ---
 .../script/compaction/QLScriptCompaction.scala     |  38 ---
 .../script/compaction/ScalaScriptCompaction.scala  |  38 ---
 .../script/compaction/ShellScriptCompaction.scala  |  39 ---
 .../storage/script/parser/CommonScriptParser.scala |  63 ----
 .../storage/script/parser/PYScriptParser.scala     |  39 ---
 .../storage/script/parser/QLScriptParser.scala     |  38 ---
 .../storage/script/parser/ScalaScriptParser.scala  |  39 ---
 .../storage/script/parser/ShellScriptParser.scala  |  39 ---
 .../script/reader/StorageScriptFsReader.scala      | 126 -------
 .../script/writer/StorageScriptFsWriter.scala      | 120 -------
 .../linkis/storage/source/AbstractFileSource.scala |  71 ----
 .../apache/linkis/storage/source/FileSource.scala  | 154 ---------
 .../apache/linkis/storage/source/FileSplit.scala   | 162 ---------
 .../storage/source/ResultsetFileSource.scala       |  36 --
 .../linkis/storage/source/TextFileSource.scala     |  52 ---
 .../linkis/storage/utils/FileSystemUtils.scala     | 116 -------
 .../storage/utils/StorageConfiguration.scala       | 126 -------
 .../linkis/storage/utils/StorageHelper.scala       | 107 ------
 .../apache/linkis/storage/utils/StorageUtils.scala | 234 -------------
 .../apache/linkis/storage/LineMetaDataTest.java}   |  22 +-
 .../StorageResultSetWriterFactoryTest.java         |  53 +++
 .../script/writer/StorageScriptFsWriterTest.java   |   8 +-
 .../storage/source/ResultsetFileSourceTest.java    |  50 +++
 .../src/test/resources/result-read-test.dolphin    |   1 +
 .../src/test/resources/storage-read-test.dolphin   |   1 +
 .../executor/OnceExecutorExecutionContext.scala    |   8 +-
 .../computation/executor/cs/CSTableRegister.scala  |   2 +-
 .../executor/cs/CSTableResultSetWriter.scala       |   2 +-
 .../executor/execute/EngineExecutionContext.scala  |  20 +-
 .../apache/linkis/entrance/parser/ParserUtils.java |   2 +-
 .../persistence/EntranceResultSetEngine.scala      |   2 +-
 .../AbstractEngineConnBmlResourceGenerator.scala   |   5 +-
 .../executor/client/ResponseHandler.scala          |  10 +
 .../executor/client/impl/ResponseHandlerImpl.scala |  19 +-
 .../flink/executor/FlinkExecutor.scala             |   6 +-
 .../HiveEngineConcurrentConnExecutor.scala         |   8 +-
 .../hive/executor/HiveEngineConnExecutor.scala     |   8 +-
 .../io/executor/IoEngineConnExecutor.scala         |  88 ++---
 .../engineplugin/io/service/FsProxyService.scala   |  18 +-
 .../manager/engineplugin/io/utils/IOHelp.scala     |  34 +-
 .../src/test/resources/testIoResult.dolphin        |   0
 .../io/executor/IoEngineConnExecutorTest.java      |  73 ++++
 .../jdbc/executor/JDBCEngineConnExecutor.scala     |   4 +-
 .../engineplugin/jdbc/executor/JDBCHelper.java     |  44 +--
 .../builder/OpenLooKengProcessECLaunchBuilder.java |   2 +-
 .../executor/OpenLooKengEngineConnExecutor.java    |   4 +-
 .../presto/executor/PrestoEngineConnExecutor.scala |   4 +-
 .../python/executor/PythonSession.scala            |  18 +-
 .../engineplugin/spark/executor/SQLSession.scala   |   6 +-
 .../spark/executor/SparkPythonExecutor.scala       |   4 +-
 .../spark/executor/SparkScalaExecutor.scala        |   4 +-
 .../engineplugin/spark/utils/EngineUtils.scala     |   4 +-
 .../execution/datasources/csv/DolphinToSpark.scala |  26 +-
 .../trino/executor/TrinoEngineConnExecutor.scala   |   5 +-
 .../linkis/storage/io/client/DefaultIOClient.scala |   4 +-
 .../io/iteraceptor/IOMethodInterceptor.scala       |  14 +-
 .../IOMethodInterceptorCreatorImpl.scala           |   4 +-
 .../linkis/storage/io/utils/IOClientUtils.scala    |   3 +-
 .../linkis/bml/service/impl/TaskServiceImpl.java   |   2 +-
 .../filesystem/restful/api/BMLFsRestfulApi.java    |  11 +-
 .../filesystem/restful/api/FsRestfulApi.java       |  34 +-
 pom.xml                                            |   1 +
 179 files changed, 6041 insertions(+), 5095 deletions(-)

diff --git a/linkis-commons/linkis-common/src/main/java/org/apache/linkis/common/io/FsPath.java b/linkis-commons/linkis-common/src/main/java/org/apache/linkis/common/io/FsPath.java
index 7427a567c..7908e5a50 100644
--- a/linkis-commons/linkis-common/src/main/java/org/apache/linkis/common/io/FsPath.java
+++ b/linkis-commons/linkis-common/src/main/java/org/apache/linkis/common/io/FsPath.java
@@ -176,7 +176,7 @@ public class FsPath {
     return owner.equals(user);
   }
 
-  public FsPath getParent() throws IOException {
+  public FsPath getParent() {
     String path = uri.getPath();
     int lastSlash = path.lastIndexOf('/');
     int start = startPositionWithoutWindowsDrive(path);
@@ -269,6 +269,7 @@ public class FsPath {
   }
 
   public String getSchemaPath() {
+    // local file system
     if (WINDOWS && !"hdfs".equals(getFsType())) {
       return getFsType() + "://" + uri.getAuthority() + uri.getPath();
     }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/FSFactory.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/FSFactory.java
new file mode 100644
index 000000000..34a92ead8
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/FSFactory.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage;
+
+import org.apache.linkis.common.io.Fs;
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.storage.exception.StorageWarnException;
+import org.apache.linkis.storage.factory.BuildFactory;
+import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.storage.utils.StorageUtils;
+
+import java.text.MessageFormat;
+import java.util.Map;
+
+import static org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.UNSUPPORTED_FILE;
+
+public class FSFactory {
+  private static final Map<String, BuildFactory> buildClasses =
+      StorageUtils.loadClass(
+          StorageConfiguration.STORAGE_BUILD_FS_CLASSES.getValue(), t -> t.fsName());
+
+  public static BuildFactory getBuildFactory(String fsName) {
+    if (!buildClasses.containsKey(fsName)) {
+      throw new StorageWarnException(
+          UNSUPPORTED_FILE.getErrorCode(),
+          MessageFormat.format(UNSUPPORTED_FILE.getErrorDesc(), fsName));
+    }
+    return buildClasses.get(fsName);
+  }
+
+  public static Fs getFs(String fsType, String proxyUser) {
+    String user = StorageUtils.getJvmUser();
+    return getBuildFactory(fsType).getFs(user, proxyUser);
+  }
+
+  public static Fs getFs(String fsType) {
+    String user = StorageUtils.getJvmUser();
+    return getBuildFactory(fsType).getFs(user, user);
+  }
+
+  /**
+   * 1. If this machine has shared storage, the file:// type FS obtained here is the FS of the
+   * process user. 2. If this machine does not have shared storage, then the file:// type FS
+   * obtained is the proxy to the Remote (shared storage machine root) FS. 3. If it is HDFS, it
+   * returns the FS of the process user. 1、如果这台机器装有共享存储则这里获得的file://类型的FS为该进程用户的FS
+   * 2、如果这台机器没有共享存储则获得的file://类型的FS为代理到Remote(共享存储机器root)的FS 3、如果是HDFS则返回的就是该进程用户的FS
+   *
+   * @param fsPath
+   * @return
+   */
+  public static Fs getFs(FsPath fsPath) {
+    return getFs(fsPath.getFsType());
+  }
+
+  /**
+   * 1. If the process user is passed and the proxy user and the process user are consistent, the
+   * file:// type FS is the FS of the process user (the shared storage exists) 2. If the process
+   * user is passed and the proxy user and the process user are consistent and there is no shared
+   * storage, the file:// type FS is the proxy to the remote (shared storage machine root) FS 3. If
+   * the passed proxy user and process user are consistent, the hdfs type is the FS of the process
+   * user. 4. If the proxy user and the process user are inconsistent, the hdfs type is the FS after
+   * the proxy.
+   *
+   * @param fsPath
+   * @param proxyUser
+   * @return
+   */
+  public static Fs getFsByProxyUser(FsPath fsPath, String proxyUser) {
+    return getFs(fsPath.getFsType(), proxyUser);
+  }
+
+  public Fs getFSByLabel(String fs, String label) {
+    String user = StorageUtils.getJvmUser();
+    return getBuildFactory(fs).getFs(user, user, label);
+  }
+
+  public Fs getFSByLabelAndUser(String fs, String label, String proxy) {
+    String user = StorageUtils.getJvmUser();
+    return getBuildFactory(fs).getFs(user, proxy, label);
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/CommonScriptCompaction.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/LineMetaData.java
similarity index 60%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/CommonScriptCompaction.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/LineMetaData.java
index e085d2aac..a6dab8b37 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/CommonScriptCompaction.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/LineMetaData.java
@@ -15,20 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.script.compaction
+package org.apache.linkis.storage;
 
-import org.apache.linkis.storage.script.{Compaction, Variable}
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.storage.resultset.ResultMetaData;
 
-abstract class CommonScriptCompaction extends Compaction {
+public class LineMetaData implements ResultMetaData {
 
-  override def compact(variable: Variable): String = {
-    variable.sortParent match {
-      case "variable" => prefix + " " + variable.key + "=" + variable.value
-      case _ =>
-        prefixConf + " " + variable.sortParent + " " + variable.sort + " " + variable.key + "=" + variable.value
-    }
+  private String metaData = null;
+
+  public LineMetaData() {}
+
+  public LineMetaData(String metaData) {
+    this.metaData = metaData;
+  }
+
+  public String getMetaData() {
+    return metaData;
   }
 
-  override def getAnnotationSymbol(): String = prefix.split('@')(0)
+  public void setMetaData(String metaData) {
+    this.metaData = metaData;
+  }
 
+  public MetaData cloneMeta() {
+    return new LineMetaData(metaData);
+  }
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableRecord.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/LineRecord.java
similarity index 61%
copy from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableRecord.scala
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/LineRecord.java
index d8e056096..7f6209000 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableRecord.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/LineRecord.java
@@ -15,22 +15,33 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset.table
+package org.apache.linkis.storage;
 
-import org.apache.linkis.common.io.Record
-import org.apache.linkis.storage.resultset.ResultRecord
-import org.apache.linkis.storage.utils.StorageUtils
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.storage.resultset.ResultRecord;
 
-class TableRecord(val row: Array[Any]) extends ResultRecord {
+public class LineRecord implements ResultRecord {
+  private String line;
 
-  override def cloneRecord(): Record = {
-    new TableRecord(row)
+  public LineRecord(String line) {
+    this.line = line;
   }
 
-  def tableRecordToString(nullValue: String = "NULL"): Array[String] = {
-    row.map { col =>
-      StorageUtils.colToString(col, nullValue)
-    }
+  public String getLine() {
+    return line;
   }
 
+  public void setLine(String line) {
+    this.line = line;
+  }
+
+  @Override
+  public Record cloneRecord() {
+    return new LineRecord(line);
+  }
+
+  @Override
+  public String toString() {
+    return line;
+  }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/conf/LinkisStorageConf.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/conf/LinkisStorageConf.java
new file mode 100644
index 000000000..66fafeacd
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/conf/LinkisStorageConf.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.conf;
+
+import org.apache.linkis.common.conf.CommonVars;
+import org.apache.linkis.common.utils.ByteTimeUtils;
+
+import org.apache.commons.lang3.StringUtils;
+
+public class LinkisStorageConf {
+  private static final Object CONF_LOCK = new Object();
+
+  public static final String HDFS_FILE_SYSTEM_REST_ERRS =
+      CommonVars.apply(
+              "wds.linkis.hdfs.rest.errs",
+              ".*Filesystem closed.*|.*Failed to find any Kerberos tgt.*")
+          .getValue();
+
+  public static final String ROW_BYTE_MAX_LEN_STR =
+      CommonVars.apply("wds.linkis.resultset.row.max.str", "2m").getValue();
+
+  public static final long ROW_BYTE_MAX_LEN = ByteTimeUtils.byteStringAsBytes(ROW_BYTE_MAX_LEN_STR);
+
+  public static final String FILE_TYPE =
+      CommonVars.apply(
+              "wds.linkis.storage.file.type",
+              "dolphin,sql,scala,py,hql,python,out,log,text,sh,jdbc,ngql,psql,fql,tsql")
+          .getValue();
+
+  private static volatile String[] fileTypeArr = null;
+
+  private static String[] fileTypeArrParser(String fileType) {
+    if (StringUtils.isBlank(fileType)) {
+      return new String[0];
+    } else {
+      return fileType.split(",");
+    }
+  }
+
+  public static String[] getFileTypeArr() {
+    if (fileTypeArr == null) {
+      synchronized (CONF_LOCK) {
+        if (fileTypeArr == null) {
+          fileTypeArr = fileTypeArrParser(FILE_TYPE);
+        }
+      }
+    }
+    return fileTypeArr;
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineRecord.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/csv/CSVFsWriter.java
similarity index 60%
copy from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineRecord.scala
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/csv/CSVFsWriter.java
index d6e3220cf..f05e93c65 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineRecord.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/csv/CSVFsWriter.java
@@ -15,20 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage
+package org.apache.linkis.storage.csv;
 
-import org.apache.linkis.common.io.Record
-import org.apache.linkis.storage.resultset.ResultRecord
+import org.apache.linkis.common.io.FsWriter;
 
-class LineRecord(private var line: String) extends ResultRecord {
+import java.io.OutputStream;
 
-  def getLine: String = line
+public abstract class CSVFsWriter extends FsWriter {
+  public abstract String getCharset();
 
-  def setLine(line: String): Unit = {
-    this.line = line
-  }
+  public abstract String getSeparator();
 
-  override def cloneRecord(): Record = new LineRecord(line)
+  public abstract boolean isQuoteRetouchEnable();
 
-  override def toString: String = line
+  public static CSVFsWriter getCSVFSWriter(
+      String charset, String separator, boolean quoteRetouchEnable, OutputStream outputStream) {
+    return new StorageCSVWriter(charset, separator, quoteRetouchEnable, outputStream);
+  }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/csv/StorageCSVWriter.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/csv/StorageCSVWriter.java
new file mode 100644
index 000000000..620b1b3c1
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/csv/StorageCSVWriter.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.csv;
+
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.storage.domain.Column;
+import org.apache.linkis.storage.domain.DataType;
+import org.apache.linkis.storage.resultset.table.TableMetaData;
+import org.apache.linkis.storage.resultset.table.TableRecord;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StorageCSVWriter extends CSVFsWriter {
+  private static final Logger logger = LoggerFactory.getLogger(StorageCSVWriter.class);
+
+  private final String charset;
+  private final String separator;
+  private final boolean quoteRetouchEnable;
+  private final OutputStream outputStream;
+
+  private final String delimiter;
+  private final StringBuilder buffer;
+
+  public StorageCSVWriter(
+      String charset, String separator, boolean quoteRetouchEnable, OutputStream outputStream) {
+    this.charset = charset;
+    this.separator = separator;
+    this.quoteRetouchEnable = quoteRetouchEnable;
+    this.outputStream = outputStream;
+
+    this.delimiter = StringUtils.isNotEmpty(separator) ? separator : "\t";
+    this.buffer = new StringBuilder(50000);
+  }
+
+  @Override
+  public String getCharset() {
+    return charset;
+  }
+
+  @Override
+  public String getSeparator() {
+    return separator;
+  }
+
+  @Override
+  public boolean isQuoteRetouchEnable() {
+    return quoteRetouchEnable;
+  }
+
+  @Override
+  public void addMetaData(MetaData metaData) throws IOException {
+    Column[] columns = ((TableMetaData) metaData).getColumns();
+    String[] head = Stream.of(columns).map(Column::getColumnName).toArray(String[]::new);
+    write(head);
+  }
+
+  private String compact(String[] row) {
+    String quotationMarks = "\"";
+    StringBuilder rowBuilder = new StringBuilder();
+    for (String value : row) {
+      String decoratedValue =
+          StringUtils.isBlank(value)
+              ? value
+              : quoteRetouchEnable
+                  ? quotationMarks + value.replaceAll(quotationMarks, "") + quotationMarks
+                  : value;
+      rowBuilder.append(decoratedValue).append(delimiter);
+    }
+    rowBuilder.append("\n");
+    if (logger.isDebugEnabled()) {
+      logger.debug("delimiter:" + delimiter);
+    }
+    return rowBuilder.toString();
+  }
+
+  private void write(String[] row) throws IOException {
+    String content = compact(row);
+    if (buffer.length() + content.length() > 49500) {
+      IOUtils.write(buffer.toString().getBytes(charset), outputStream);
+      buffer.setLength(0);
+    }
+    buffer.append(content);
+  }
+
+  @Override
+  public void addRecord(Record record) throws IOException {
+    Object[] rows = ((TableRecord) record).row;
+    String[] body =
+        Stream.of(rows).map(dataType -> DataType.valueToString(dataType)).toArray(String[]::new);
+    write(body);
+  }
+
+  @Override
+  public void flush() throws IOException {
+    IOUtils.write(buffer.toString().getBytes(charset), outputStream);
+    buffer.setLength(0);
+  }
+
+  @Override
+  public void close() throws IOException {
+    flush();
+    IOUtils.closeQuietly(outputStream);
+  }
+}
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/Column.java
similarity index 53%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/Column.java
index babeb1f6c..0d59805b0 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/Column.java
@@ -15,16 +15,37 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.domain;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+public final class Column {
+  public String columnName;
+  public DataType dataType;
+  public String comment;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+  public Column(String columnName, DataType dataType, String comment) {
+    this.columnName = columnName;
+    this.dataType = dataType;
+    this.comment = comment;
+  }
+
+  public String getColumnName() {
+    return columnName;
+  }
+
+  public DataType getDataType() {
+    return dataType;
+  }
+
+  public String getComment() {
+    return comment;
+  }
+
+  public Object[] toArray() {
+    return new Object[] {columnName, dataType, comment};
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public String toString() {
+    return "columnName:" + columnName + ",dataType:" + dataType + ",comment:" + comment;
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/DataType.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/DataType.java
new file mode 100644
index 000000000..ad9e0ee88
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/DataType.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.domain;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public enum DataType {
+  NullType("void", 0),
+  StringType("string", 12),
+  BooleanType("boolean", 16),
+  TinyIntType("tinyint", -6),
+  ShortIntType("short", 5),
+  IntType("int", 4),
+  LongType("long", -5),
+  BigIntType("bigint", -5),
+  FloatType("float", 6),
+  DoubleType("double", 8),
+  CharType("char", 1),
+  VarcharType("varchar", 12),
+  DateType("date", 91),
+  TimestampType("timestamp", 93),
+  BinaryType("binary", -2),
+  DecimalType("decimal", 3),
+  ArrayType("array", 2003),
+  MapType("map", 2000),
+  ListType("list", 2001),
+  StructType("struct", 2002),
+  BigDecimalType("bigdecimal", 3);
+
+  private final String typeName;
+  private final int javaSQLType;
+
+  DataType(String typeName, int javaSQLType) {
+    this.typeName = typeName;
+    this.javaSQLType = javaSQLType;
+  }
+
+  private static Logger logger = LoggerFactory.getLogger(DataType.class);
+
+  public static final String NULL_VALUE = "NULL";
+  public static final String LOWCASE_NULL_VALUE = "null";
+
+  // TODO Change to fine-grained regular expressions(改为精细化正则表达式)
+  public static final Pattern DECIMAL_REGEX = Pattern.compile("^decimal\\(\\d*\\,\\d*\\)");
+
+  public static final Pattern SHORT_REGEX = Pattern.compile("^short.*");
+  public static final Pattern INT_REGEX = Pattern.compile("^int.*");
+  public static final Pattern LONG_REGEX = Pattern.compile("^long.*");
+  public static final Pattern BIGINT_REGEX = Pattern.compile("^bigint.*");
+  public static final Pattern FLOAT_REGEX = Pattern.compile("^float.*");
+  public static final Pattern DOUBLE_REGEX = Pattern.compile("^double.*");
+
+  public static final Pattern VARCHAR_REGEX = Pattern.compile("^varchar.*");
+  public static final Pattern CHAR_REGEX = Pattern.compile("^char.*");
+
+  public static final Pattern ARRAY_REGEX = Pattern.compile("array.*");
+
+  public static final Pattern MAP_REGEX = Pattern.compile("map.*");
+
+  public static final Pattern LIST_REGEX = Pattern.compile("list.*");
+
+  public static final Pattern STRUCT_REGEX = Pattern.compile("struct.*");
+
+  public static DataType toDataType(String dataType) {
+    if (dataType.equals("void") || dataType.equals("null")) {
+      return DataType.NullType;
+    } else if (dataType.equals("string")) {
+      return DataType.StringType;
+    } else if (dataType.equals("boolean")) {
+      return DataType.BooleanType;
+    } else if (SHORT_REGEX.matcher(dataType).matches()) {
+      return DataType.ShortIntType;
+    } else if (LONG_REGEX.matcher(dataType).matches()) {
+      return DataType.LongType;
+    } else if (BIGINT_REGEX.matcher(dataType).matches()) {
+      return DataType.BigIntType;
+    } else if (INT_REGEX.matcher(dataType).matches()
+        || dataType.equals("integer")
+        || dataType.equals("smallint")) {
+      return DataType.IntType;
+    } else if (FLOAT_REGEX.matcher(dataType).matches()) {
+      return DataType.FloatType;
+    } else if (DOUBLE_REGEX.matcher(dataType).matches()) {
+      return DataType.DoubleType;
+    } else if (VARCHAR_REGEX.matcher(dataType).matches()) {
+      return DataType.VarcharType;
+    } else if (CHAR_REGEX.matcher(dataType).matches()) {
+      return DataType.CharType;
+    } else if (dataType.equals("date")) {
+      return DataType.DateType;
+    } else if (dataType.equals("timestamp")) {
+      return DataType.TimestampType;
+    } else if (dataType.equals("binary")) {
+      return DataType.BinaryType;
+    } else if (dataType.equals("decimal") || DECIMAL_REGEX.matcher(dataType).matches()) {
+      return DataType.DecimalType;
+    } else if (ARRAY_REGEX.matcher(dataType).matches()) {
+      return DataType.ArrayType;
+    } else if (MAP_REGEX.matcher(dataType).matches()) {
+      return DataType.MapType;
+    } else if (LIST_REGEX.matcher(dataType).matches()) {
+      return DataType.ListType;
+    } else if (STRUCT_REGEX.matcher(dataType).matches()) {
+      return DataType.StructType;
+    } else {
+      return DataType.StringType;
+    }
+  }
+
+  public static Object toValue(DataType dataType, String value) {
+    Object result = null;
+    try {
+      switch (dataType) {
+        case NullType:
+          result = null;
+          break;
+        case StringType:
+        case CharType:
+        case VarcharType:
+        case StructType:
+        case ListType:
+        case ArrayType:
+        case MapType:
+          result = value;
+          break;
+        case BooleanType:
+          result = isNumberNull(value) ? null : Boolean.valueOf(value);
+          break;
+        case ShortIntType:
+          result = isNumberNull(value) ? null : Short.valueOf(value);
+          break;
+        case IntType:
+          result = isNumberNull(value) ? null : Integer.valueOf(value);
+          break;
+        case LongType:
+        case BigIntType:
+          result = isNumberNull(value) ? null : Long.valueOf(value);
+          break;
+        case FloatType:
+          result = isNumberNull(value) ? null : Float.valueOf(value);
+          break;
+        case DoubleType:
+          result = isNumberNull(value) ? null : Double.valueOf(value);
+          break;
+        case DecimalType:
+          result = isNumberNull(value) ? null : new BigDecimal(value);
+          break;
+        case DateType:
+          result = isNumberNull(value) ? null : Date.valueOf(value);
+          break;
+        case TimestampType:
+          result =
+              isNumberNull(value)
+                  ? null
+                  : Optional.of(value)
+                      .map(Timestamp::valueOf)
+                      .map(Timestamp::toString)
+                      .map(s -> s.endsWith(".0") ? s.substring(0, s.length() - 2) : s)
+                      .orElse(null);
+          break;
+        case BinaryType:
+          result = isNull(value) ? null : value.getBytes();
+          break;
+        default:
+          result = value;
+      }
+    } catch (Exception e) {
+      logger.debug("Failed to " + value + " switch to dataType:", e);
+      result = value;
+    }
+    return result;
+  }
+
+  public static boolean isNull(String value) {
+    return value == null || value.equals(NULL_VALUE) || value.trim().equals("");
+  }
+
+  public static boolean isNumberNull(String value) {
+    return value == null || value.equalsIgnoreCase(NULL_VALUE) || value.trim().equals("");
+  }
+
+  public static String valueToString(Object value) {
+    if (value == null) {
+      return LOWCASE_NULL_VALUE;
+    } else if (value instanceof BigDecimal) {
+      return ((BigDecimal) value).toPlainString();
+    } else {
+      return value.toString();
+    }
+  }
+
+  public String getTypeName() {
+    return typeName;
+  }
+
+  public int getJavaSQLType() {
+    return javaSQLType;
+  }
+
+  @Override
+  public String toString() {
+    return typeName;
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/Dolphin.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/Dolphin.java
new file mode 100644
index 000000000..b6badd284
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/Dolphin.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.domain;
+
+import org.apache.linkis.storage.exception.StorageWarnException;
+import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.storage.utils.StorageUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.Charset;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.FAILED_TO_READ_INTEGER;
+
+public class Dolphin {
+  private static final Logger logger = LoggerFactory.getLogger(Dolphin.class);
+
+  public static final Charset CHAR_SET =
+      Charset.forName(StorageConfiguration.STORAGE_RS_FILE_TYPE.getValue());
+  public static final String MAGIC = "dolphin";
+
+  public static byte[] MAGIC_BYTES = new byte[0];
+
+  static {
+    try {
+      MAGIC_BYTES = MAGIC.getBytes(StorageConfiguration.STORAGE_RS_FILE_TYPE.getValue());
+    } catch (UnsupportedEncodingException e) {
+      logger.warn("Dolphin getBytes failed", e);
+    }
+  }
+
+  public static final int MAGIC_LEN = MAGIC_BYTES.length;
+
+  public static final String DOLPHIN_FILE_SUFFIX = ".dolphin";
+
+  public static final String COL_SPLIT = ",";
+  public static final byte[] COL_SPLIT_BYTES = COL_SPLIT.getBytes(Charset.forName("utf-8"));
+  public static final int COL_SPLIT_LEN = COL_SPLIT_BYTES.length;
+
+  public static final String NULL = "NULL";
+  public static final byte[] NULL_BYTES = "NULL".getBytes(Charset.forName("utf-8"));
+
+  public static final int INT_LEN = 10;
+
+  public static final int FILE_EMPTY = 31;
+
+  public static byte[] getBytes(Object value) {
+    return value.toString().getBytes(CHAR_SET);
+  }
+
+  /**
+   * Convert a bytes array to a String content 将bytes数组转换为String内容
+   *
+   * @param bytes
+   * @param start
+   * @param len
+   * @return
+   */
+  public static String getString(byte[] bytes, int start, int len) {
+    return new String(bytes, start, len, Dolphin.CHAR_SET);
+  }
+
+  /**
+   * Read an integer value that converts the array to a byte of length 10 bytes
+   * 读取整数值,该值为将数组转换为10字节长度的byte
+   *
+   * @param inputStream
+   * @return
+   * @throws IOException
+   */
+  public static int readInt(InputStream inputStream) throws IOException {
+    byte[] bytes = new byte[INT_LEN + 1];
+    if (StorageUtils.readBytes(inputStream, bytes, INT_LEN) != INT_LEN) {
+      throw new StorageWarnException(
+          FAILED_TO_READ_INTEGER.getErrorCode(), FAILED_TO_READ_INTEGER.getErrorDesc());
+    }
+    return Integer.parseInt(getString(bytes, 0, INT_LEN));
+  }
+
+  /**
+   * Print integers at a fixed length(将整数按固定长度打印)
+   *
+   * @param value
+   * @return
+   */
+  public static byte[] getIntBytes(int value) {
+    String str = Integer.toString(value);
+    StringBuilder res = new StringBuilder();
+    for (int i = 0; i < INT_LEN - str.length(); i++) {
+      res.append("0");
+    }
+    res.append(str);
+    return Dolphin.getBytes(res.toString());
+  }
+
+  public static String getType(InputStream inputStream) throws IOException {
+    byte[] bytes = new byte[100];
+    int len = StorageUtils.readBytes(inputStream, bytes, Dolphin.MAGIC_LEN + INT_LEN);
+    if (len == -1) return null;
+    return getType(Dolphin.getString(bytes, 0, len));
+  }
+
+  public static String getType(String content) {
+    if (content.length() < MAGIC.length() || !content.substring(0, MAGIC.length()).equals(MAGIC)) {
+      throw new RuntimeException(
+          "File header type must be dolphin, content: " + content + " is not");
+    }
+    return Integer.toString(
+        Integer.parseInt(content.substring(MAGIC.length(), MAGIC.length() + INT_LEN)));
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/MethodEntity.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/MethodEntity.java
new file mode 100644
index 000000000..c1e16e223
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/MethodEntity.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.domain;
+
+/**
+ * Engine unique Id(engine唯一的Id)
+ *
+ * <p>Fs type(fs类型)
+ *
+ * <p>Create a user to start the corresponding jvm user(创建用户为对应启动的jvm用户)
+ *
+ * <p>Proxy user(代理用户)
+ *
+ * <p>client Ip for whitelist control(ip用于白名单控制)
+ *
+ * <p>Method name called(调用的方法名)
+ *
+ * <p>Method parameter(方法参数)
+ */
+public class MethodEntity {
+  private long id;
+  private String fsType;
+  private String creatorUser;
+  private String proxyUser;
+  private String clientIp;
+  private String methodName;
+  private Object[] params;
+
+  public MethodEntity(
+      long id,
+      String fsType,
+      String creatorUser,
+      String proxyUser,
+      String clientIp,
+      String methodName,
+      Object[] params) {
+    this.id = id;
+    this.fsType = fsType;
+    this.creatorUser = creatorUser;
+    this.proxyUser = proxyUser;
+    this.clientIp = clientIp;
+    this.methodName = methodName;
+    this.params = params;
+  }
+
+  public long getId() {
+    return id;
+  }
+
+  public void setId(long id) {
+    this.id = id;
+  }
+
+  public String getFsType() {
+    return fsType;
+  }
+
+  public void setFsType(String fsType) {
+    this.fsType = fsType;
+  }
+
+  public String getCreatorUser() {
+    return creatorUser;
+  }
+
+  public void setCreatorUser(String creatorUser) {
+    this.creatorUser = creatorUser;
+  }
+
+  public String getProxyUser() {
+    return proxyUser;
+  }
+
+  public void setProxyUser(String proxyUser) {
+    this.proxyUser = proxyUser;
+  }
+
+  public String getClientIp() {
+    return clientIp;
+  }
+
+  public void setClientIp(String clientIp) {
+    this.clientIp = clientIp;
+  }
+
+  public String getMethodName() {
+    return methodName;
+  }
+
+  public void setMethodName(String methodName) {
+    this.methodName = methodName;
+  }
+
+  public Object[] getParams() {
+    return params;
+  }
+
+  public void setParams(Object[] params) {
+    this.params = params;
+  }
+
+  @Override
+  public String toString() {
+    return "id:"
+        + id
+        + ", methodName:"
+        + methodName
+        + ", fsType:"
+        + fsType
+        + ", creatorUser:"
+        + creatorUser
+        + ", proxyUser:"
+        + proxyUser
+        + ", clientIp:"
+        + clientIp;
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/MethodEntitySerializer.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/MethodEntitySerializer.java
new file mode 100644
index 000000000..777b756a7
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/domain/MethodEntitySerializer.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.domain;
+
+import java.lang.reflect.Type;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+
+/**
+ * Serialize MethodEntity to code 序列化MethodEntity为code
+ *
+ * <p>Serialized to code as a MethodEntity object 序列化为code为MethodEntity对象
+ *
+ * <p>Serialize a java object as a string 序列化java对象为字符串
+ *
+ * <p>Deserialize a string into a java object 将字符串解序列化为java对象
+ */
+public class MethodEntitySerializer {
+
+  private static final Gson gson =
+      new GsonBuilder().setDateFormat("yyyy-MM-dd'T'HH:mm:ssZ").create();
+
+  /**
+   * Serialized to code as a MethodEntity object 序列化为code为MethodEntity对象
+   *
+   * @param code
+   * @return
+   */
+  public static MethodEntity deserializer(String code) {
+    return gson.fromJson(code, MethodEntity.class);
+  }
+
+  /**
+   * Serialize MethodEntity to code 序列化MethodEntity为code
+   *
+   * @param methodEntity
+   * @return
+   */
+  public static String serializer(MethodEntity methodEntity) {
+    return gson.toJson(methodEntity);
+  }
+
+  /**
+   * Serialize a java object as a string 序列化java对象为字符串
+   *
+   * @param value
+   * @return
+   */
+  public static String serializerJavaObject(Object value) {
+    return gson.toJson(value);
+  }
+
+  /**
+   * Deserialize a string into a java object 将字符串解序列化为java对象
+   *
+   * @param json
+   * @param classType
+   * @param <T>
+   * @return
+   */
+  public static <T> T deserializerToJavaObject(String json, Class<T> classType) {
+    return gson.fromJson(json, classType);
+  }
+
+  public static <T> T deserializerToJavaObject(String json, Type oType) {
+    return gson.fromJson(json, oType);
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IORecord.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/ExcelAnalysisException.java
similarity index 67%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IORecord.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/ExcelAnalysisException.java
index 54d3dc022..87b988f24 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IORecord.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/ExcelAnalysisException.java
@@ -15,11 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset.io
+package org.apache.linkis.storage.excel;
 
-import org.apache.linkis.common.io.Record
-import org.apache.linkis.storage.resultset.ResultRecord
+class ExcelAnalysisException extends RuntimeException {
+  public ExcelAnalysisException() {}
 
-class IORecord(val value: Array[Byte]) extends ResultRecord {
-  override def cloneRecord(): Record = new IORecord(value)
+  public ExcelAnalysisException(String message) {
+    super(message);
+  }
+
+  public ExcelAnalysisException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public ExcelAnalysisException(Throwable cause) {
+    super(cause);
+  }
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineRecord.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/ExcelFsWriter.java
similarity index 57%
copy from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineRecord.scala
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/ExcelFsWriter.java
index d6e3220cf..cd3969e04 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineRecord.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/ExcelFsWriter.java
@@ -15,20 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage
+package org.apache.linkis.storage.excel;
 
-import org.apache.linkis.common.io.Record
-import org.apache.linkis.storage.resultset.ResultRecord
+import org.apache.linkis.common.io.FsWriter;
 
-class LineRecord(private var line: String) extends ResultRecord {
+import java.io.OutputStream;
 
-  def getLine: String = line
+public abstract class ExcelFsWriter extends FsWriter {
+  public abstract String getCharset();
 
-  def setLine(line: String): Unit = {
-    this.line = line
-  }
+  public abstract String getSheetName();
+
+  public abstract String getDateFormat();
 
-  override def cloneRecord(): Record = new LineRecord(line)
+  public abstract boolean isAutoFormat();
 
-  override def toString: String = line
+  public static ExcelFsWriter getExcelFsWriter(
+      String charset,
+      String sheetName,
+      String dateFormat,
+      OutputStream outputStream,
+      boolean autoFormat) {
+    return new StorageExcelWriter(charset, sheetName, dateFormat, outputStream, autoFormat);
+  }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/ExcelXlsReader.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/ExcelXlsReader.java
new file mode 100644
index 000000000..98df7421f
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/ExcelXlsReader.java
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.excel;
+
+import org.apache.poi.hssf.eventusermodel.*;
+import org.apache.poi.hssf.eventusermodel.EventWorkbookBuilder.SheetRecordCollectingListener;
+import org.apache.poi.hssf.eventusermodel.dummyrecord.LastCellOfRowDummyRecord;
+import org.apache.poi.hssf.eventusermodel.dummyrecord.MissingCellDummyRecord;
+import org.apache.poi.hssf.model.HSSFFormulaParser;
+import org.apache.poi.hssf.record.*;
+import org.apache.poi.hssf.usermodel.HSSFWorkbook;
+import org.apache.poi.poifs.filesystem.POIFSFileSystem;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExcelXlsReader implements HSSFListener {
+  private static final Logger logger = LoggerFactory.getLogger(ExcelXlsReader.class);
+
+  private int minColumns = -1;
+
+  private POIFSFileSystem fs;
+
+  private InputStream inputStream;
+
+  private int lastRowNumber;
+
+  private int lastColumnNumber;
+
+  /** Should we output the formula, or the value it has? */
+  private boolean outputFormulaValues = true;
+
+  /** For parsing Formulas */
+  private SheetRecordCollectingListener workbookBuildingListener;
+
+  // excel2003Workbook(工作薄)
+  private HSSFWorkbook stubWorkbook;
+
+  // Records we pick up as we process
+  private SSTRecord sstRecord;
+
+  private FormatTrackingHSSFListener formatListener;
+
+  // Table index(表索引)
+  private int sheetIndex = -1;
+
+  private BoundSheetRecord[] orderedBSRs;
+
+  @SuppressWarnings("unchecked")
+  private ArrayList boundSheetRecords = new ArrayList();
+
+  // For handling formulas with string results
+  private int nextRow;
+
+  private int nextColumn;
+
+  private boolean outputNextStringRecord;
+
+  // Current line(当前行)
+  private int curRow = 0;
+
+  // a container that stores row records(存储行记录的容器)
+  private List<String> rowlist = new ArrayList<String>();
+
+  @SuppressWarnings("unused")
+  private String sheetName;
+
+  private IExcelRowDeal excelRowDeal;
+
+  public void init(IExcelRowDeal excelRowDeal, InputStream inputStream) {
+    this.excelRowDeal = excelRowDeal;
+    this.inputStream = inputStream;
+  }
+
+  /**
+   * Traverse all the sheets under excel 遍历excel下所有的sheet
+   *
+   * @throws IOException
+   */
+  public void process() throws IOException {
+    this.fs = new POIFSFileSystem(this.inputStream);
+    MissingRecordAwareHSSFListener listener = new MissingRecordAwareHSSFListener(this);
+    formatListener = new FormatTrackingHSSFListener(listener);
+    HSSFEventFactory factory = new HSSFEventFactory();
+    HSSFRequest request = new HSSFRequest();
+    if (outputFormulaValues) {
+      request.addListenerForAllRecords(formatListener);
+    } else {
+      workbookBuildingListener = new SheetRecordCollectingListener(formatListener);
+      request.addListenerForAllRecords(workbookBuildingListener);
+    }
+    factory.processWorkbookEvents(request, fs);
+  }
+
+  /** HSSFListener listener method, processing Record HSSFListener 监听方法,处理 Record */
+  @Override
+  @SuppressWarnings("unchecked")
+  public void processRecord(Record record) {
+    int thisRow = -1;
+    int thisColumn = -1;
+    String thisStr = null;
+    String value = null;
+    switch (record.getSid()) {
+      case BoundSheetRecord.sid:
+        boundSheetRecords.add(record);
+        break;
+      case BOFRecord.sid:
+        BOFRecord br = (BOFRecord) record;
+        if (br.getType() == BOFRecord.TYPE_WORKSHEET) {
+          // Create a child workbook if needed(如果有需要,则建立子工作薄)
+          if (workbookBuildingListener != null && stubWorkbook == null) {
+            stubWorkbook = workbookBuildingListener.getStubHSSFWorkbook();
+          }
+
+          sheetIndex++;
+          if (orderedBSRs == null) {
+            orderedBSRs = BoundSheetRecord.orderByBofPosition(boundSheetRecords);
+          }
+          sheetName = orderedBSRs[sheetIndex].getSheetname();
+        }
+        break;
+
+      case SSTRecord.sid:
+        sstRecord = (SSTRecord) record;
+        break;
+
+      case BlankRecord.sid:
+        BlankRecord brec = (BlankRecord) record;
+        thisRow = brec.getRow();
+        thisColumn = brec.getColumn();
+        thisStr = "";
+        rowlist.add(thisColumn, thisStr);
+        break;
+      case BoolErrRecord.sid: // Cell is boolean(单元格为布尔类型)
+        BoolErrRecord berec = (BoolErrRecord) record;
+        thisRow = berec.getRow();
+        thisColumn = berec.getColumn();
+        thisStr = berec.getBooleanValue() + "";
+        rowlist.add(thisColumn, thisStr);
+        break;
+
+      case FormulaRecord.sid: // Cell is a formula type(单元格为公式类型)
+        FormulaRecord frec = (FormulaRecord) record;
+        thisRow = frec.getRow();
+        thisColumn = frec.getColumn();
+        if (outputFormulaValues) {
+          if (Double.isNaN(frec.getValue())) {
+            // Formula result is a string
+            // This is stored in the next record
+            outputNextStringRecord = true;
+            nextRow = frec.getRow();
+            nextColumn = frec.getColumn();
+          } else {
+            thisStr = formatListener.formatNumberDateCell(frec);
+          }
+        } else {
+          thisStr =
+              '"'
+                  + HSSFFormulaParser.toFormulaString(stubWorkbook, frec.getParsedExpression())
+                  + '"';
+        }
+        rowlist.add(thisColumn, thisStr);
+        break;
+      case StringRecord.sid: // a string of formulas in a cell(单元格中公式的字符串)
+        if (outputNextStringRecord) {
+          // String for formula
+          StringRecord srec = (StringRecord) record;
+          thisStr = srec.getString();
+          thisRow = nextRow;
+          thisColumn = nextColumn;
+          outputNextStringRecord = false;
+        }
+        break;
+      case LabelRecord.sid:
+        LabelRecord lrec = (LabelRecord) record;
+        curRow = thisRow = lrec.getRow();
+        thisColumn = lrec.getColumn();
+        value = lrec.getValue().trim();
+        value = value.equals("") ? " " : value;
+        this.rowlist.add(thisColumn, value);
+        break;
+      case LabelSSTRecord.sid: // Cell is a string type(单元格为字符串类型)
+        LabelSSTRecord lsrec = (LabelSSTRecord) record;
+        curRow = thisRow = lsrec.getRow();
+        thisColumn = lsrec.getColumn();
+        if (sstRecord == null) {
+          rowlist.add(thisColumn, " ");
+        } else {
+          value = sstRecord.getString(lsrec.getSSTIndex()).toString().trim();
+          value = value.equals("") ? " " : value;
+          rowlist.add(thisColumn, value);
+        }
+        break;
+      case NumberRecord.sid: // Cell is a numeric type(单元格为数字类型)
+        NumberRecord numrec = (NumberRecord) record;
+        curRow = thisRow = numrec.getRow();
+        thisColumn = numrec.getColumn();
+        value = formatListener.formatNumberDateCell(numrec).trim();
+        value = value.equals("") ? "0" : value;
+        // Add column values to the container(向容器加入列值)
+        rowlist.add(thisColumn, value);
+        break;
+      default:
+        break;
+    }
+
+    // Encountered a new line of operations(遇到新行的操作)(
+    if (thisRow != -1 && thisRow != lastRowNumber) {
+      lastColumnNumber = -1;
+    }
+
+    // Null operation(空值的操作)
+    if (record instanceof MissingCellDummyRecord) {
+      MissingCellDummyRecord mc = (MissingCellDummyRecord) record;
+      curRow = thisRow = mc.getRow();
+      thisColumn = mc.getColumn();
+      rowlist.add(thisColumn, " ");
+    }
+
+    // Update row and column values(更新行和列的值)
+    if (thisRow > -1) lastRowNumber = thisRow;
+    if (thisColumn > -1) lastColumnNumber = thisColumn;
+
+    // End of line operation(行结束时的操作)
+    if (record instanceof LastCellOfRowDummyRecord) {
+      if (minColumns > 0) {
+        // Column value is re-empted(列值重新置空)
+        if (lastColumnNumber == -1) {
+          lastColumnNumber = 0;
+        }
+      }
+      lastColumnNumber = -1;
+
+      // At the end of each line, the dealRow() method(每行结束时, dealRow() 方法)
+      excelRowDeal.dealRow(orderedBSRs, sheetIndex, curRow, rowlist);
+      // Empty container(清空容器)
+      rowlist.clear();
+    }
+  }
+
+  public void close() {
+    try {
+      if (fs != null) {
+        fs.close();
+      }
+    } catch (Exception e) {
+      logger.info("ExcelXlsReader fs closed failed", e);
+    }
+
+    try {
+      if (inputStream != null) {
+        inputStream.close();
+      }
+    } catch (IOException e) {
+      logger.info("ExcelXlsReader inputStream closed failed", e);
+    }
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/FirstRowDeal.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/FirstRowDeal.java
new file mode 100644
index 000000000..6924a3ebb
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/FirstRowDeal.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.excel;
+
+import org.apache.poi.hssf.record.BoundSheetRecord;
+
+import java.util.ArrayList;
+import java.util.List;
+
+class FirstRowDeal implements IExcelRowDeal {
+
+  private List<String> sheetNames = new ArrayList<>();
+  private List<String> row;
+
+  public List<String> getSheetNames() {
+    return sheetNames;
+  }
+
+  public void setSheetNames(List<String> sheetNames) {
+    this.sheetNames = sheetNames;
+  }
+
+  public List<String> getRow() {
+    return row;
+  }
+
+  public void setRow(List<String> row) {
+    this.row = row;
+  }
+
+  @Override
+  public void dealRow(
+      BoundSheetRecord[] orderedBSRs, int sheetIndex, int curRow, List<String> rowlist) {
+    for (BoundSheetRecord record : orderedBSRs) {
+      sheetNames.add(record.getSheetname());
+    }
+    row = rowlist;
+    throw new ExcelAnalysisException("Finished to deal first row");
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultMetaData.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/IExcelRowDeal.java
similarity index 76%
copy from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultMetaData.scala
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/IExcelRowDeal.java
index 7871bb2ac..405104174 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultMetaData.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/IExcelRowDeal.java
@@ -15,8 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset
+package org.apache.linkis.storage.excel;
 
-import org.apache.linkis.common.io.MetaData
+import org.apache.poi.hssf.record.BoundSheetRecord;
 
-abstract class ResultMetaData extends MetaData {}
+import java.util.List;
+
+interface IExcelRowDeal {
+  void dealRow(BoundSheetRecord[] orderedBSRs, int sheetIndex, int curRow, List<String> rowlist);
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/RowToCsvDeal.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/RowToCsvDeal.java
new file mode 100644
index 000000000..7deccfb92
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/RowToCsvDeal.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.excel;
+
+import org.apache.poi.hssf.record.BoundSheetRecord;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Map;
+
+class RowToCsvDeal implements IExcelRowDeal {
+
+  private Map<String, Object> params;
+  private List<String> sheetNames;
+  private OutputStream outputStream;
+  private Boolean hasHeader;
+  private Boolean fisrtRow = true;
+
+  public void init(Boolean hasHeader, List<String> sheetNames, OutputStream outputStream) {
+    this.hasHeader = hasHeader;
+    this.sheetNames = sheetNames;
+    this.outputStream = outputStream;
+  }
+
+  @Override
+  public void dealRow(
+      BoundSheetRecord[] orderedBSRs, int sheetIndex, int curRow, List<String> rowlist) {
+    String sheetName = orderedBSRs[sheetIndex].getSheetname();
+    if (sheetNames == null || sheetNames.isEmpty() || sheetNames.contains(sheetName)) {
+      if (!(curRow == 0 && hasHeader)) {
+        try {
+          if (fisrtRow) {
+            fisrtRow = false;
+          } else {
+            outputStream.write("\n".getBytes());
+          }
+          int len = rowlist.size();
+          for (int i = 0; i < len; i++) {
+            outputStream.write(rowlist.get(i).replaceAll("\n|\t", " ").getBytes("utf-8"));
+            if (i < len - 1) {
+              outputStream.write("\t".getBytes());
+            }
+          }
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/StorageExcelWriter.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/StorageExcelWriter.java
new file mode 100644
index 000000000..05e1b68de
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/StorageExcelWriter.java
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.excel;
+
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.storage.domain.Column;
+import org.apache.linkis.storage.domain.DataType;
+import org.apache.linkis.storage.resultset.table.TableMetaData;
+import org.apache.linkis.storage.resultset.table.TableRecord;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.poi.ss.usermodel.*;
+import org.apache.poi.xssf.streaming.SXSSFSheet;
+import org.apache.poi.xssf.streaming.SXSSFWorkbook;
+
+import java.io.*;
+import java.math.BigDecimal;
+import java.util.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StorageExcelWriter extends ExcelFsWriter {
+
+  private static Logger logger = LoggerFactory.getLogger(StorageExcelWriter.class);
+
+  private String charset;
+  private String sheetName;
+  private String dateFormat;
+  private OutputStream outputStream;
+  private boolean autoFormat;
+  protected SXSSFWorkbook workBook;
+  protected SXSSFSheet sheet;
+  private DataFormat format;
+  protected DataType[] types;
+  protected int rowPoint;
+  protected int columnCounter;
+  protected Map<String, CellStyle> styles;
+  private boolean isFlush;
+  private ByteArrayOutputStream os;
+  private ByteArrayInputStream is;
+
+  public StorageExcelWriter(
+      String charset,
+      String sheetName,
+      String dateFormat,
+      OutputStream outputStream,
+      boolean autoFormat) {
+    this.charset = charset;
+    this.sheetName = sheetName;
+    this.dateFormat = dateFormat;
+    this.outputStream = outputStream;
+    this.autoFormat = autoFormat;
+  }
+
+  public void init() {
+    workBook = new SXSSFWorkbook();
+    sheet = workBook.createSheet(sheetName);
+  }
+
+  public CellStyle getDefaultHeadStyle() {
+    Font headerFont = workBook.createFont();
+    headerFont.setBold(true);
+    headerFont.setFontHeightInPoints((short) 14);
+    headerFont.setColor(IndexedColors.RED.getIndex());
+    CellStyle headerCellStyle = workBook.createCellStyle();
+    headerCellStyle.setFont(headerFont);
+    return headerCellStyle;
+  }
+
+  public Workbook getWorkBook() {
+    // 自适应列宽
+    sheet.trackAllColumnsForAutoSizing();
+    for (int elem = 0; elem <= columnCounter; elem++) {
+      sheet.autoSizeColumn(elem);
+    }
+    return workBook;
+  }
+
+  public CellStyle createCellStyle(DataType dataType) {
+    CellStyle style = workBook.createCellStyle();
+    format = workBook.createDataFormat();
+    style.setDataFormat(format.getFormat("@"));
+
+    if (autoFormat) {
+      switch (dataType) {
+        case StringType:
+        case CharType:
+        case VarcharType:
+          style.setDataFormat(format.getFormat("@"));
+          break;
+        case TinyIntType:
+        case ShortIntType:
+        case IntType:
+          style.setDataFormat(format.getFormat("#"));
+          break;
+        case LongType:
+        case BigIntType:
+          style.setDataFormat(format.getFormat("#.##E+00"));
+          break;
+        case FloatType:
+          style.setDataFormat(format.getFormat("#.0000000000"));
+          break;
+        case DoubleType:
+          style.setDataFormat(format.getFormat("#.0000000000"));
+          break;
+        case DateType:
+        case TimestampType:
+          style.setDataFormat(format.getFormat("m/d/yy h:mm"));
+          break;
+        case DecimalType:
+        case BigDecimalType:
+          style.setDataFormat(format.getFormat("#.000000000"));
+          break;
+        default:
+          style.setDataFormat(format.getFormat("@"));
+      }
+    }
+    return style;
+  }
+
+  public CellStyle getCellStyle(DataType dataType) {
+    CellStyle style = styles.get(dataType.getTypeName());
+    if (style == null) {
+      CellStyle newStyle = createCellStyle(dataType);
+      styles.put(dataType.getTypeName(), newStyle);
+      return newStyle;
+    } else {
+      return style;
+    }
+  }
+
+  @Override
+  public void addMetaData(MetaData metaData) throws IOException {
+    init();
+    Row tableHead = sheet.createRow(0);
+    Column[] columns = ((TableMetaData) metaData).getColumns();
+    List<DataType> columnType = new ArrayList<>();
+    for (int i = 0; i < columns.length; i++) {
+      Cell headCell = tableHead.createCell(columnCounter);
+      headCell.setCellValue(columns[i].getColumnName());
+      headCell.setCellStyle(getDefaultHeadStyle());
+      columnType.add(columns[i].getDataType());
+      columnCounter++;
+    }
+    types = columnType.toArray(new DataType[0]);
+    rowPoint++;
+  }
+
+  @Override
+  public void addRecord(Record record) throws IOException {
+    // TODO: 是否需要替换null值
+    Row tableBody = sheet.createRow(rowPoint);
+    int colunmPoint = 0;
+    Object[] excelRecord = ((TableRecord) record).row;
+    for (Object elem : excelRecord) {
+      Cell cell = tableBody.createCell(colunmPoint);
+      DataType dataType = types[colunmPoint];
+      if (autoFormat) {
+        setCellTypeValue(dataType, elem, cell);
+      } else {
+        cell.setCellValue(DataType.valueToString(elem));
+      }
+      cell.setCellStyle(getCellStyle(dataType));
+      colunmPoint++;
+    }
+    rowPoint++;
+  }
+
+  private void setCellTypeValue(DataType dataType, Object elem, Cell cell) {
+    if (null == elem) return;
+
+    try {
+      switch (dataType) {
+        case StringType:
+        case CharType:
+        case VarcharType:
+          cell.setCellValue(DataType.valueToString(elem));
+          break;
+        case TinyIntType:
+        case ShortIntType:
+        case IntType:
+          cell.setCellValue(Integer.valueOf(elem.toString()));
+          break;
+        case LongType:
+        case BigIntType:
+          cell.setCellValue(Long.valueOf(elem.toString()));
+          break;
+        case FloatType:
+          cell.setCellValue(Float.valueOf(elem.toString()));
+          break;
+        case DoubleType:
+          doubleCheck(elem.toString());
+          cell.setCellValue(Double.valueOf(elem.toString()));
+          break;
+        case DateType:
+        case TimestampType:
+          cell.setCellValue(getDate(elem));
+          break;
+        case DecimalType:
+        case BigDecimalType:
+          doubleCheck(DataType.valueToString(elem));
+          cell.setCellValue(Double.valueOf(DataType.valueToString(elem)));
+          break;
+        default:
+          cell.setCellValue(DataType.valueToString(elem));
+      }
+    } catch (Exception e) {
+      cell.setCellValue(DataType.valueToString(elem));
+    }
+  }
+
+  private Date getDate(Object value) {
+    if (value instanceof Date) {
+      return (Date) value;
+    } else {
+      throw new NumberFormatException(
+          "Value "
+              + value
+              + " with class : "
+              + value.getClass().getName()
+              + " is not a valid type of Date.");
+    }
+  }
+
+  /**
+   * Check whether the double exceeds the number of digits, which will affect the data accuracy
+   *
+   * @param elemValue
+   */
+  private void doubleCheck(String elemValue) {
+    BigDecimal value = new BigDecimal(elemValue).stripTrailingZeros();
+    if ((value.precision() - value.scale()) > 15) {
+      throw new NumberFormatException(
+          "Value " + elemValue + " error : This data exceeds 15 significant digits.");
+    }
+  }
+
+  @Override
+  public void flush() {
+    try {
+      getWorkBook().write(os);
+    } catch (IOException e) {
+      logger.warn("flush fail", e);
+    }
+    byte[] content = os.toByteArray();
+    is = new ByteArrayInputStream(content);
+    byte[] buffer = new byte[1024];
+    int bytesRead = 0;
+    while (isFlush) {
+      try {
+        bytesRead = is.read(buffer, 0, 1024);
+        if (bytesRead == -1) {
+          isFlush = false;
+        } else {
+          outputStream.write(buffer, 0, bytesRead);
+        }
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  @Override
+  public void close() {
+    if (isFlush) {
+      flush();
+    }
+    IOUtils.closeQuietly(outputStream);
+    IOUtils.closeQuietly(is);
+    IOUtils.closeQuietly(os);
+    IOUtils.closeQuietly(workBook);
+  }
+
+  @Override
+  public String getCharset() {
+    return this.charset;
+  }
+
+  @Override
+  public String getSheetName() {
+    return this.sheetName;
+  }
+
+  @Override
+  public String getDateFormat() {
+    return this.dateFormat;
+  }
+
+  @Override
+  public boolean isAutoFormat() {
+    return this.autoFormat;
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/StorageMultiExcelWriter.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/StorageMultiExcelWriter.java
similarity index 58%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/StorageMultiExcelWriter.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/StorageMultiExcelWriter.java
index b47e04394..a028397bd 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/StorageMultiExcelWriter.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/excel/StorageMultiExcelWriter.java
@@ -15,39 +15,44 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.excel
+package org.apache.linkis.storage.excel;
 
-import org.apache.poi.xssf.streaming.SXSSFWorkbook
+import org.apache.poi.xssf.streaming.SXSSFWorkbook;
 
-import java.io.OutputStream
+import java.io.OutputStream;
 
-class StorageMultiExcelWriter(
-    override val outputStream: OutputStream,
-    override val autoFormat: Boolean
-) extends StorageExcelWriter(null, null, null, outputStream, autoFormat) {
+public class StorageMultiExcelWriter extends StorageExcelWriter {
 
-  private var sheetIndex = 0
+  private int sheetIndex = 0;
 
-  override def init: Unit = {
-    if (workBook == null) workBook = new SXSSFWorkbook()
+  public StorageMultiExcelWriter(OutputStream outputStream, boolean autoFormat) {
+    super(null, null, null, outputStream, autoFormat);
+  }
+
+  @Override
+  public void init() {
+    if (workBook == null) {
+      workBook = new SXSSFWorkbook();
+    }
     // 1.让表自适应列宽
     if (sheet != null) {
-      sheet.trackAllColumnsForAutoSizing()
-      0 to columnCounter foreach (sheet.autoSizeColumn)
+      sheet.trackAllColumnsForAutoSizing();
+      for (int i = 0; i <= columnCounter; i++) {
+        sheet.autoSizeColumn(i);
+      }
     }
     // 2.重置参数
     // 2.1 columnCounter 归0
-    columnCounter = 0
+    columnCounter = 0;
     // 2.2 创建新sheet
-    sheet = workBook.createSheet(s"resultset$sheetIndex")
+    sheet = workBook.createSheet("resultset" + sheetIndex);
     // 2.3 sheetIndex自增
-    sheetIndex += 1
+    sheetIndex++;
     // 2.4 types 置空
-    types = null
+    types = null;
     // 2.5 rowPoint 归0 记录行数
-    rowPoint = 0
+    rowPoint = 0;
     // 2.6 styles 清空
-    styles.clear()
+    styles.clear();
   }
-
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildHDFSFileSystem.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildHDFSFileSystem.java
index 9f53a6249..8103c6f3d 100644
--- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildHDFSFileSystem.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildHDFSFileSystem.java
@@ -21,7 +21,7 @@ import org.apache.linkis.common.io.Fs;
 import org.apache.linkis.storage.factory.BuildFactory;
 import org.apache.linkis.storage.fs.FileSystem;
 import org.apache.linkis.storage.fs.impl.HDFSFileSystem;
-import org.apache.linkis.storage.io.IOMethodInterceptorCreator$;
+import org.apache.linkis.storage.io.IOMethodInterceptorFactory;
 import org.apache.linkis.storage.utils.StorageUtils;
 
 import org.springframework.cglib.proxy.Enhancer;
@@ -46,7 +46,7 @@ public class BuildHDFSFileSystem implements BuildFactory {
       // TODO Agent user(代理的用户)
       Enhancer enhancer = new Enhancer();
       enhancer.setSuperclass(HDFSFileSystem.class.getSuperclass());
-      enhancer.setCallback(IOMethodInterceptorCreator$.MODULE$.getIOMethodInterceptor(fsName()));
+      enhancer.setCallback(IOMethodInterceptorFactory.getIOMethodInterceptor(fsName()));
       fs = (FileSystem) enhancer.create();
     }
     fs.setUser(proxyUser);
@@ -63,6 +63,6 @@ public class BuildHDFSFileSystem implements BuildFactory {
 
   @Override
   public String fsName() {
-    return StorageUtils.HDFS();
+    return StorageUtils.HDFS;
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildLocalFileSystem.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildLocalFileSystem.java
index ef88cec36..bcd61c573 100644
--- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildLocalFileSystem.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildLocalFileSystem.java
@@ -21,7 +21,7 @@ import org.apache.linkis.common.io.Fs;
 import org.apache.linkis.storage.factory.BuildFactory;
 import org.apache.linkis.storage.fs.FileSystem;
 import org.apache.linkis.storage.fs.impl.LocalFileSystem;
-import org.apache.linkis.storage.io.IOMethodInterceptorCreator$;
+import org.apache.linkis.storage.io.IOMethodInterceptorFactory;
 import org.apache.linkis.storage.utils.StorageConfiguration;
 import org.apache.linkis.storage.utils.StorageUtils;
 
@@ -34,7 +34,7 @@ public class BuildLocalFileSystem implements BuildFactory {
     FileSystem fs = null;
     if (StorageUtils.isIOProxy()) {
       if (user.equals(proxyUser)) {
-        if ((Boolean) StorageConfiguration.IS_SHARE_NODE().getValue()) {
+        if ((Boolean) StorageConfiguration.IS_SHARE_NODE.getValue()) {
           fs = new LocalFileSystem();
         } else {
           fs = getProxyFs();
@@ -58,12 +58,12 @@ public class BuildLocalFileSystem implements BuildFactory {
   private FileSystem getProxyFs() {
     Enhancer enhancer = new Enhancer();
     enhancer.setSuperclass(LocalFileSystem.class.getSuperclass());
-    enhancer.setCallback(IOMethodInterceptorCreator$.MODULE$.getIOMethodInterceptor(fsName()));
+    enhancer.setCallback(IOMethodInterceptorFactory.getIOMethodInterceptor(fsName()));
     return (FileSystem) enhancer.create();
   }
 
   @Override
   public String fsName() {
-    return StorageUtils.FILE();
+    return StorageUtils.FILE;
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildOSSSystem.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildOSSSystem.java
index 1c3161251..ba1bd7abe 100644
--- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildOSSSystem.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildOSSSystem.java
@@ -65,6 +65,6 @@ public class BuildOSSSystem implements BuildFactory {
 
   @Override
   public String fsName() {
-    return StorageUtils.OSS();
+    return StorageUtils.OSS;
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildS3FileSystem.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildS3FileSystem.java
index 1818941fa..44082e589 100644
--- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildS3FileSystem.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/factory/impl/BuildS3FileSystem.java
@@ -57,6 +57,6 @@ public class BuildS3FileSystem implements BuildFactory {
 
   @Override
   public String fsName() {
-    return StorageUtils.S3();
+    return StorageUtils.S3;
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/HDFSFileSystem.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/HDFSFileSystem.java
index 7b52d8709..871d35350 100644
--- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/HDFSFileSystem.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/HDFSFileSystem.java
@@ -162,8 +162,7 @@ public class HDFSFileSystem extends FileSystem {
     List<FsPath> fsPaths = new ArrayList<FsPath>();
     for (FileStatus f : stat) {
       fsPaths.add(
-          fillStorageFile(
-              new FsPath(StorageUtils.HDFS_SCHEMA() + f.getPath().toUri().getPath()), f));
+          fillStorageFile(new FsPath(StorageUtils.HDFS_SCHEMA + f.getPath().toUri().getPath()), f));
     }
     if (fsPaths.isEmpty()) {
       return null;
@@ -175,8 +174,8 @@ public class HDFSFileSystem extends FileSystem {
   @Override
   public void init(Map<String, String> properties) throws IOException {
     if (MapUtils.isNotEmpty(properties)
-        && properties.containsKey(StorageConfiguration.PROXY_USER().key())) {
-      user = StorageConfiguration.PROXY_USER().getValue(properties);
+        && properties.containsKey(StorageConfiguration.PROXY_USER.key())) {
+      user = StorageConfiguration.PROXY_USER.getValue(properties);
     }
 
     if (user == null) {
@@ -193,14 +192,14 @@ public class HDFSFileSystem extends FileSystem {
         }
       }
     }
-    if (StorageConfiguration.FS_CACHE_DISABLE().getValue()) {
+    if (StorageConfiguration.FS_CACHE_DISABLE.getValue()) {
       conf.set("fs.hdfs.impl.disable.cache", "true");
     }
     fs = HDFSUtils.getHDFSUserFileSystem(user, conf);
     if (fs == null) {
       throw new IOException("init HDFS FileSystem failed!");
     }
-    if (StorageConfiguration.FS_CHECKSUM_DISBALE().getValue()) {
+    if (StorageConfiguration.FS_CHECKSUM_DISBALE.getValue()) {
       fs.setVerifyChecksum(false);
       fs.setWriteChecksum(false);
     }
@@ -213,7 +212,7 @@ public class HDFSFileSystem extends FileSystem {
 
   @Override
   public String rootUserName() {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+    return StorageConfiguration.HDFS_ROOT_USER.getValue();
   }
 
   @Override
@@ -306,9 +305,9 @@ public class HDFSFileSystem extends FileSystem {
     } catch (IOException e) {
       String message = e.getMessage();
       String rootCauseMessage = ExceptionUtils.getRootCauseMessage(e);
-      if ((message != null && message.matches(LinkisStorageConf.HDFS_FILE_SYSTEM_REST_ERRS()))
+      if ((message != null && message.matches(LinkisStorageConf.HDFS_FILE_SYSTEM_REST_ERRS))
           || (rootCauseMessage != null
-              && rootCauseMessage.matches(LinkisStorageConf.HDFS_FILE_SYSTEM_REST_ERRS()))) {
+              && rootCauseMessage.matches(LinkisStorageConf.HDFS_FILE_SYSTEM_REST_ERRS))) {
         logger.info("Failed to execute exists, retry", e);
         resetRootHdfs();
         return fs.exists(new Path(checkHDFSPath(dest.getPath())));
@@ -426,9 +425,9 @@ public class HDFSFileSystem extends FileSystem {
 
   private String checkHDFSPath(String path) {
     try {
-      boolean checkHdfsPath = (boolean) StorageConfiguration.HDFS_PATH_PREFIX_CHECK_ON().getValue();
+      boolean checkHdfsPath = (boolean) StorageConfiguration.HDFS_PATH_PREFIX_CHECK_ON.getValue();
       if (checkHdfsPath) {
-        boolean rmHdfsPrefix = (boolean) StorageConfiguration.HDFS_PATH_PREFIX_REMOVE().getValue();
+        boolean rmHdfsPrefix = (boolean) StorageConfiguration.HDFS_PATH_PREFIX_REMOVE.getValue();
         if (rmHdfsPrefix) {
           if (StringUtils.isBlank(path)) {
             return path;
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/LocalFileSystem.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/LocalFileSystem.java
index 2df547f10..0e3066489 100644
--- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/LocalFileSystem.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/LocalFileSystem.java
@@ -113,7 +113,7 @@ public class LocalFileSystem extends FileSystem {
     if (group != null) {
       setGroup(dest, group);
     }
-    setGroup(dest, StorageConfiguration.STORAGE_USER_GROUP().getValue());
+    setGroup(dest, StorageConfiguration.STORAGE_USER_GROUP.getValue());
     return true;
   }
 
@@ -289,15 +289,15 @@ public class LocalFileSystem extends FileSystem {
 
     if (MapUtils.isNotEmpty(properties)) {
       this.properties = properties;
-      if (properties.containsKey(StorageConfiguration.PROXY_USER().key())) {
-        user = StorageConfiguration.PROXY_USER().getValue(properties);
+      if (properties.containsKey(StorageConfiguration.PROXY_USER.key())) {
+        user = StorageConfiguration.PROXY_USER.getValue(properties);
       }
-      group = StorageConfiguration.STORAGE_USER_GROUP().getValue(properties);
+      group = StorageConfiguration.STORAGE_USER_GROUP.getValue(properties);
     } else {
       this.properties = new HashMap<String, String>();
     }
     if (FsPath.WINDOWS) {
-      group = StorageConfiguration.STORAGE_USER_GROUP().getValue(properties);
+      group = StorageConfiguration.STORAGE_USER_GROUP.getValue(properties);
     }
     if (StringUtils.isEmpty(group)) {
       String groupInfo;
@@ -320,7 +320,7 @@ public class LocalFileSystem extends FileSystem {
 
   @Override
   public String rootUserName() {
-    return StorageConfiguration.LOCAL_ROOT_USER().getValue();
+    return StorageConfiguration.LOCAL_ROOT_USER.getValue();
   }
 
   @Override
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/OSSFileSystem.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/OSSFileSystem.java
index d2e0e357d..0a4181c13 100644
--- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/OSSFileSystem.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/OSSFileSystem.java
@@ -135,8 +135,8 @@ public class OSSFileSystem extends FileSystem {
       fsPaths.add(
           fillStorageFile(
               new FsPath(
-                  StorageUtils.OSS_SCHEMA()
-                      + StorageConfiguration.OSS_ACCESS_BUCKET_NAME().getValue()
+                  StorageUtils.OSS_SCHEMA
+                      + StorageConfiguration.OSS_ACCESS_BUCKET_NAME.getValue()
                       + "/"
                       + f.getPath().toUri().getPath()),
               f));
@@ -155,10 +155,10 @@ public class OSSFileSystem extends FileSystem {
 
     // origin configs
     Map<String, String> originProperties = Maps.newHashMap();
-    originProperties.put("fs.oss.endpoint", StorageConfiguration.OSS_ENDPOINT().getValue());
-    originProperties.put("fs.oss.accessKeyId", StorageConfiguration.OSS_ACCESS_KEY_ID().getValue());
+    originProperties.put("fs.oss.endpoint", StorageConfiguration.OSS_ENDPOINT.getValue());
+    originProperties.put("fs.oss.accessKeyId", StorageConfiguration.OSS_ACCESS_KEY_ID.getValue());
     originProperties.put(
-        "fs.oss.accessKeySecret", StorageConfiguration.OSS_ACCESS_KEY_SECRET().getValue());
+        "fs.oss.accessKeySecret", StorageConfiguration.OSS_ACCESS_KEY_SECRET.getValue());
     for (String key : originProperties.keySet()) {
       String value = originProperties.get(key);
       if (StringUtils.isNotBlank(value)) {
@@ -178,8 +178,7 @@ public class OSSFileSystem extends FileSystem {
     fs = new AliyunOSSFileSystem();
     try {
       fs.initialize(
-          new URI(
-              StorageUtils.OSS_SCHEMA() + StorageConfiguration.OSS_ACCESS_BUCKET_NAME().getValue()),
+          new URI(StorageUtils.OSS_SCHEMA + StorageConfiguration.OSS_ACCESS_BUCKET_NAME.getValue()),
           conf);
     } catch (URISyntaxException e) {
       throw new IOException("init OSS FileSystem failed!");
@@ -191,7 +190,7 @@ public class OSSFileSystem extends FileSystem {
 
   @Override
   public String fsName() {
-    return StorageUtils.OSS();
+    return StorageUtils.OSS;
   }
 
   @Override
@@ -294,9 +293,9 @@ public class OSSFileSystem extends FileSystem {
     } catch (IOException e) {
       String message = e.getMessage();
       String rootCauseMessage = ExceptionUtils.getRootCauseMessage(e);
-      if ((message != null && message.matches(LinkisStorageConf.HDFS_FILE_SYSTEM_REST_ERRS()))
+      if ((message != null && message.matches(LinkisStorageConf.HDFS_FILE_SYSTEM_REST_ERRS))
           || (rootCauseMessage != null
-              && rootCauseMessage.matches(LinkisStorageConf.HDFS_FILE_SYSTEM_REST_ERRS()))) {
+              && rootCauseMessage.matches(LinkisStorageConf.HDFS_FILE_SYSTEM_REST_ERRS))) {
         logger.info("Failed to execute exists, retry", e);
         resetRootOSS();
         return fs.exists(new Path(checkOSSPath(dest.getPath())));
@@ -359,9 +358,9 @@ public class OSSFileSystem extends FileSystem {
 
   private static String checkOSSPath(String path) {
     try {
-      boolean checkOSSPath = (boolean) StorageConfiguration.OSS_PATH_PREFIX_CHECK_ON().getValue();
+      boolean checkOSSPath = (boolean) StorageConfiguration.OSS_PATH_PREFIX_CHECK_ON.getValue();
       if (checkOSSPath) {
-        boolean rmOSSPrefix = (boolean) StorageConfiguration.OSS_PATH_PREFIX_REMOVE().getValue();
+        boolean rmOSSPrefix = (boolean) StorageConfiguration.OSS_PATH_PREFIX_REMOVE.getValue();
         if (rmOSSPrefix) {
           if (StringUtils.isBlank(path)) {
             return path;
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/S3FileSystem.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/S3FileSystem.java
index 0bd669fa3..e10737c91 100644
--- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/S3FileSystem.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/fs/impl/S3FileSystem.java
@@ -66,11 +66,11 @@ public class S3FileSystem extends FileSystem {
 
   @Override
   public void init(Map<String, String> properties) throws IOException {
-    accessKey = StorageConfiguration.S3_ACCESS_KEY().getValue(properties);
-    secretKey = StorageConfiguration.S3_SECRET_KEY().getValue(properties);
-    endPoint = StorageConfiguration.S3_ENDPOINT().getValue(properties);
-    bucket = StorageConfiguration.S3_BUCKET().getValue(properties);
-    region = StorageConfiguration.S3_REGION().getValue(properties);
+    accessKey = StorageConfiguration.S3_ACCESS_KEY.getValue(properties);
+    secretKey = StorageConfiguration.S3_SECRET_KEY.getValue(properties);
+    endPoint = StorageConfiguration.S3_ENDPOINT.getValue(properties);
+    bucket = StorageConfiguration.S3_BUCKET.getValue(properties);
+    region = StorageConfiguration.S3_REGION.getValue(properties);
 
     AwsClientBuilder.EndpointConfiguration endpointConfiguration =
         new AwsClientBuilder.EndpointConfiguration(endPoint, region);
@@ -90,7 +90,7 @@ public class S3FileSystem extends FileSystem {
 
   @Override
   public String fsName() {
-    return StorageUtils.S3();
+    return StorageUtils.S3;
   }
 
   @Override
@@ -340,9 +340,9 @@ public class S3FileSystem extends FileSystem {
   public String buildPath(String path) {
     if (path == null || "".equals(path)) return "";
     if (path.startsWith("/")) {
-      return StorageUtils.S3_SCHEMA() + path;
+      return StorageUtils.S3_SCHEMA + path;
     }
-    return StorageUtils.S3_SCHEMA() + "/" + path;
+    return StorageUtils.S3_SCHEMA + "/" + path;
   }
 }
 
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOMetaData.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOClient.java
similarity index 71%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOMetaData.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOClient.java
index 32b578c5a..294f9957c 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOMetaData.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOClient.java
@@ -15,11 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset.io
+package org.apache.linkis.storage.io;
 
-import org.apache.linkis.common.io.MetaData
-import org.apache.linkis.storage.resultset.ResultMetaData
+import org.apache.linkis.storage.domain.MethodEntity;
 
-class IOMetaData(val off: Int, val len: Int) extends ResultMetaData {
-  override def cloneMeta(): MetaData = new IOMetaData(off, len)
+import java.util.Map;
+
+public interface IOClient {
+
+  String execute(String user, MethodEntity methodEntity, Map<String, Object> params);
+
+  String[] executeWithEngine(String user, MethodEntity methodEntity, Map<String, Object> params);
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/io/IOMethodInterceptorCreator.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOClientFactory.java
similarity index 53%
copy from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/io/IOMethodInterceptorCreator.scala
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOClientFactory.java
index 51e1589eb..244ad5947 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/io/IOMethodInterceptorCreator.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOClientFactory.java
@@ -15,39 +15,43 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.io
+package org.apache.linkis.storage.io;
 
-import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.MUST_REGISTER_TOM
-import org.apache.linkis.storage.exception.StorageErrorException
+import org.apache.linkis.storage.exception.StorageErrorException;
 
-import org.springframework.cglib.proxy.MethodInterceptor
+import java.util.UUID;
 
-trait IOMethodInterceptorCreator {
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-  def createIOMethodInterceptor(fsName: String): MethodInterceptor
-}
+import static org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.MUST_REGISTER_TOC;
+
+public class IOClientFactory {
+  private static final Logger logger = LoggerFactory.getLogger(IOClientFactory.class);
+  private static IOClient ioClient = null;
 
-object IOMethodInterceptorCreator {
+  private static final String SUCCESS = "SUCCESS";
+  private static final String FAILED = "FAILED";
 
-  var interceptorCreator: IOMethodInterceptorCreator = null
+  public static IOClient getIOClient() throws StorageErrorException {
+    if (ioClient == null) {
+      throw new StorageErrorException(
+          MUST_REGISTER_TOC.getErrorCode(), MUST_REGISTER_TOC.getErrorDesc());
+    }
+    return ioClient;
+  }
 
   /**
    * This method is called when ioClient is initialized. ioClient初始化时会调用该方法
    *
-   * @param interceptorCreator
+   * @param client IOClient
    */
-  def register(interceptorCreator: IOMethodInterceptorCreator): Unit = {
-    this.interceptorCreator = interceptorCreator
+  public static void register(IOClient client) {
+    ioClient = client;
+    logger.debug("IOClient: {} registered", ioClient.toString());
   }
 
-  def getIOMethodInterceptor(fsName: String): MethodInterceptor = {
-    if (interceptorCreator == null) {
-      throw new StorageErrorException(
-        MUST_REGISTER_TOM.getErrorCode,
-        MUST_REGISTER_TOM.getErrorDesc
-      )
-    }
-    interceptorCreator.createIOMethodInterceptor(fsName)
+  public static String getFSId() {
+    return UUID.randomUUID().toString();
   }
-
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/CSVFsReader.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOMethodInterceptorCreator.java
similarity index 79%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/CSVFsReader.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOMethodInterceptorCreator.java
index d40d041a3..26a8ceea1 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/CSVFsReader.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOMethodInterceptorCreator.java
@@ -15,8 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.csv
+package org.apache.linkis.storage.io;
 
-import org.apache.linkis.common.io.FsReader
+import org.springframework.cglib.proxy.MethodInterceptor;
 
-abstract class CSVFsReader extends FsReader {}
+public interface IOMethodInterceptorCreator {
+
+  MethodInterceptor createIOMethodInterceptor(String fsName);
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/io/IOMethodInterceptorCreator.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOMethodInterceptorFactory.java
similarity index 52%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/io/IOMethodInterceptorCreator.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOMethodInterceptorFactory.java
index 51e1589eb..8f0c4016d 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/io/IOMethodInterceptorCreator.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/io/IOMethodInterceptorFactory.java
@@ -15,39 +15,35 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.io
+package org.apache.linkis.storage.io;
 
-import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.MUST_REGISTER_TOM
-import org.apache.linkis.storage.exception.StorageErrorException
+import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary;
+import org.apache.linkis.storage.exception.StorageWarnException;
 
-import org.springframework.cglib.proxy.MethodInterceptor
+import org.springframework.cglib.proxy.MethodInterceptor;
 
-trait IOMethodInterceptorCreator {
+public class IOMethodInterceptorFactory {
 
-  def createIOMethodInterceptor(fsName: String): MethodInterceptor
-}
-
-object IOMethodInterceptorCreator {
+  private static IOMethodInterceptorCreator interceptorCreator = null;
 
-  var interceptorCreator: IOMethodInterceptorCreator = null
+  private IOMethodInterceptorFactory() {}
 
   /**
    * This method is called when ioClient is initialized. ioClient初始化时会调用该方法
    *
    * @param interceptorCreator
    */
-  def register(interceptorCreator: IOMethodInterceptorCreator): Unit = {
-    this.interceptorCreator = interceptorCreator
+  public static void register(IOMethodInterceptorCreator interceptorCreator) {
+    IOMethodInterceptorFactory.interceptorCreator = interceptorCreator;
   }
 
-  def getIOMethodInterceptor(fsName: String): MethodInterceptor = {
-    if (interceptorCreator == null) {
-      throw new StorageErrorException(
-        MUST_REGISTER_TOM.getErrorCode,
-        MUST_REGISTER_TOM.getErrorDesc
-      )
+  public static MethodInterceptor getIOMethodInterceptor(String fsName)
+      throws StorageWarnException {
+    if (IOMethodInterceptorFactory.interceptorCreator == null) {
+      throw new StorageWarnException(
+          LinkisStorageErrorCodeSummary.MUST_REGISTER_TOM.getErrorCode(),
+          LinkisStorageErrorCodeSummary.MUST_REGISTER_TOM.getErrorDesc());
     }
-    interceptorCreator.createIOMethodInterceptor(fsName)
+    return IOMethodInterceptorFactory.interceptorCreator.createIOMethodInterceptor(fsName);
   }
-
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/DefaultResultSetFactory.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/DefaultResultSetFactory.java
new file mode 100644
index 000000000..db78afac2
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/DefaultResultSetFactory.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset;
+
+import org.apache.linkis.common.io.Fs;
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.common.io.resultset.ResultSet;
+import org.apache.linkis.storage.FSFactory;
+import org.apache.linkis.storage.domain.Dolphin;
+import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary;
+import org.apache.linkis.storage.exception.StorageWarnException;
+import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.storage.utils.StorageUtils;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.MessageFormat;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.THE_FILE_IS_EMPTY;
+
+public class DefaultResultSetFactory implements ResultSetFactory {
+
+  private static final Logger logger = LoggerFactory.getLogger(DefaultResultSetFactory.class);
+
+  private final Map<String, Class<ResultSet<ResultMetaData, ResultRecord>>> resultClasses;
+
+  private final String[] resultTypes;
+
+  public DefaultResultSetFactory() {
+    resultClasses =
+        StorageUtils.loadClasses(
+            StorageConfiguration.STORAGE_RESULT_SET_CLASSES.getValue(),
+            StorageConfiguration.STORAGE_RESULT_SET_PACKAGE.getValue(),
+            t -> {
+              try {
+                return t.newInstance().resultSetType().toLowerCase(Locale.getDefault());
+              } catch (InstantiationException e) {
+                logger.warn("DefaultResultSetFactory init failed", e);
+              } catch (IllegalAccessException e) {
+                logger.warn("DefaultResultSetFactory init failed", e);
+              }
+              return null;
+            });
+    resultTypes = ResultSetFactory.resultSetType.keySet().toArray(new String[0]);
+  }
+
+  @Override
+  public ResultSet<? extends MetaData, ? extends Record> getResultSetByType(String resultSetType) {
+    if (!resultClasses.containsKey(resultSetType)) {
+      throw new StorageWarnException(
+          LinkisStorageErrorCodeSummary.UNSUPPORTED_RESULT.getErrorCode(),
+          MessageFormat.format(
+              LinkisStorageErrorCodeSummary.UNSUPPORTED_RESULT.getErrorDesc(), resultSetType));
+    }
+    try {
+      return resultClasses.get(resultSetType).newInstance();
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new StorageWarnException(
+          LinkisStorageErrorCodeSummary.UNSUPPORTED_RESULT.getErrorCode(),
+          MessageFormat.format(
+              LinkisStorageErrorCodeSummary.UNSUPPORTED_RESULT.getErrorDesc(), resultSetType),
+          e);
+    }
+  }
+
+  @Override
+  public ResultSet<? extends MetaData, ? extends Record> getResultSetByPath(FsPath fsPath)
+      throws StorageWarnException {
+    return getResultSetByPath(fsPath, StorageUtils.getJvmUser());
+  }
+
+  @Override
+  public ResultSet<? extends MetaData, ? extends Record> getResultSetByContent(String content) {
+    return getResultSetByType(Dolphin.getType(content));
+  }
+
+  @Override
+  public boolean exists(String resultSetType) {
+    return resultClasses.containsKey(resultSetType);
+  }
+
+  @Override
+  public boolean isResultSetPath(String path) {
+    return path.endsWith(Dolphin.DOLPHIN_FILE_SUFFIX);
+  }
+
+  @Override
+  public boolean isResultSet(String content) {
+    try {
+      return resultClasses.containsKey(Dolphin.getType(content));
+    } catch (Exception e) {
+      logger.info("Wrong result Set: " + e.getMessage());
+      return false;
+    }
+  }
+
+  @Override
+  public ResultSet<? extends MetaData, ? extends Record> getResultSet(String output)
+      throws StorageWarnException {
+    return getResultSet(output, StorageUtils.getJvmUser());
+  }
+
+  @Override
+  public String[] getResultSetType() {
+    return Arrays.copyOf(resultTypes, resultTypes.length);
+  }
+
+  @Override
+  public ResultSet<? extends MetaData, ? extends Record> getResultSetByPath(FsPath fsPath, Fs fs) {
+    try (InputStream inputStream = fs.read(fsPath)) {
+      String resultSetType = Dolphin.getType(inputStream);
+      if (StringUtils.isEmpty(resultSetType)) {
+        throw new StorageWarnException(
+            THE_FILE_IS_EMPTY.getErrorCode(),
+            MessageFormat.format(THE_FILE_IS_EMPTY.getErrorDesc(), fsPath.getPath()));
+      }
+      // Utils.tryQuietly(fs::close);
+      return getResultSetByType(resultSetType);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public ResultSet<? extends MetaData, ? extends Record> getResultSetByPath(
+      FsPath fsPath, String proxyUser) {
+    if (fsPath == null) {
+      return null;
+    }
+    logger.info("Get Result Set By Path:" + fsPath.getPath());
+    try (Fs fs = FSFactory.getFsByProxyUser(fsPath, proxyUser)) {
+      fs.init(new HashMap<>());
+      try (InputStream inputStream = fs.read(fsPath)) {
+        String resultSetType = Dolphin.getType(inputStream);
+        if (StringUtils.isEmpty(resultSetType)) {
+          throw new StorageWarnException(
+              THE_FILE_IS_EMPTY.getErrorCode(),
+              MessageFormat.format(THE_FILE_IS_EMPTY.getErrorDesc(), fsPath.getPath()));
+        }
+        IOUtils.closeQuietly(inputStream);
+        return getResultSetByType(resultSetType);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public ResultSet<? extends MetaData, ? extends Record> getResultSet(
+      String output, String proxyUser) {
+    if (isResultSetPath(output)) {
+      return getResultSetByPath(new FsPath(output), proxyUser);
+    } else if (isResultSet(output)) {
+      return getResultSetByContent(output);
+    } else {
+      return null;
+    }
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultMetaData.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultMetaData.java
similarity index 84%
copy from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultMetaData.scala
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultMetaData.java
index 7871bb2ac..04f835ac0 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultMetaData.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultMetaData.java
@@ -15,8 +15,8 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset
+package org.apache.linkis.storage.resultset;
 
-import org.apache.linkis.common.io.MetaData
+import org.apache.linkis.common.io.MetaData;
 
-abstract class ResultMetaData extends MetaData {}
+public interface ResultMetaData extends MetaData {}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/ExcelFsReader.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultRecord.java
similarity index 85%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/ExcelFsReader.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultRecord.java
index 621145cb4..ce2686099 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/ExcelFsReader.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultRecord.java
@@ -15,8 +15,8 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.excel
+package org.apache.linkis.storage.resultset;
 
-import org.apache.linkis.common.io.FsReader
+import org.apache.linkis.common.io.Record;
 
-abstract class ExcelFsReader extends FsReader {}
+public interface ResultRecord extends Record {}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetFactory.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetFactory.java
new file mode 100644
index 000000000..ed65cea16
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetFactory.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset;
+
+import org.apache.linkis.common.io.*;
+import org.apache.linkis.common.io.resultset.ResultSet;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+public interface ResultSetFactory {
+
+  String TEXT_TYPE = "1";
+  String TABLE_TYPE = "2";
+  String IO_TYPE = "3";
+  String PICTURE_TYPE = "4";
+  String HTML_TYPE = "5";
+
+  /** TODO 修改为注册形式,并修改ResultSet的getResultType逻辑 Result set corresponding type record(结果集对应类型记录) */
+  Map<String, String> resultSetType =
+      new LinkedHashMap<String, String>() {
+        {
+          put(TEXT_TYPE, "TEXT");
+          put(TABLE_TYPE, "TABLE");
+          put(IO_TYPE, "IO");
+          put(PICTURE_TYPE, "PICTURE");
+          put(HTML_TYPE, "HTML");
+        }
+      };
+
+  DefaultResultSetFactory factory = new DefaultResultSetFactory();
+
+  static ResultSetFactory getInstance() {
+    return factory;
+  }
+
+  ResultSet<? extends MetaData, ? extends Record> getResultSetByType(String resultSetType);
+
+  ResultSet<? extends MetaData, ? extends Record> getResultSetByPath(FsPath fsPath);
+
+  ResultSet<? extends MetaData, ? extends Record> getResultSetByPath(FsPath fsPath, Fs fs);
+
+  ResultSet<? extends MetaData, ? extends Record> getResultSetByContent(String content);
+
+  boolean exists(String resultSetType);
+
+  boolean isResultSetPath(String path);
+
+  boolean isResultSet(String content);
+
+  ResultSet<? extends MetaData, ? extends Record> getResultSet(String output);
+
+  ResultSet<? extends MetaData, ? extends Record> getResultSetByPath(
+      FsPath fsPath, String proxyUser);
+
+  ResultSet<? extends MetaData, ? extends Record> getResultSet(String output, String proxyUser);
+
+  String[] getResultSetType();
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetReaderFactory.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetReaderFactory.java
new file mode 100644
index 000000000..749ec9a24
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetReaderFactory.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset;
+
+import org.apache.linkis.common.io.Fs;
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.common.io.resultset.ResultSet;
+import org.apache.linkis.common.io.resultset.ResultSetReader;
+import org.apache.linkis.storage.FSFactory;
+import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary;
+import org.apache.linkis.storage.exception.StorageWarnException;
+import org.apache.linkis.storage.resultset.table.TableMetaData;
+import org.apache.linkis.storage.resultset.table.TableRecord;
+import org.apache.linkis.storage.resultset.table.TableResultSet;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ResultSetReaderFactory {
+  private static final Logger logger = LoggerFactory.getLogger(ResultSetReaderFactory.class);
+
+  public static <K extends MetaData, V extends Record> ResultSetReader getResultSetReader(
+      ResultSet<K, V> resultSet, InputStream inputStream) {
+    return new StorageResultSetReader<>(resultSet, inputStream);
+  }
+
+  public static <K extends MetaData, V extends Record> ResultSetReader getResultSetReader(
+      ResultSet<K, V> resultSet, String value) {
+    return new StorageResultSetReader<>(resultSet, value);
+  }
+
+  public static ResultSetReader getResultSetReader(String res) {
+    ResultSetFactory rsFactory = ResultSetFactory.getInstance();
+    if (rsFactory.isResultSet(res)) {
+      ResultSet<? extends MetaData, ? extends Record> resultSet = rsFactory.getResultSet(res);
+      return ResultSetReaderFactory.getResultSetReader(resultSet, res);
+    } else {
+      FsPath resPath = new FsPath(res);
+      ResultSet<? extends MetaData, ? extends Record> resultSet =
+          rsFactory.getResultSetByPath(resPath);
+      try {
+        FSFactory.getFs(resPath).init(null);
+      } catch (IOException e) {
+        logger.warn("ResultSetReaderFactory fs init failed", e);
+      }
+      ResultSetReader reader = null;
+      try {
+        reader =
+            ResultSetReaderFactory.getResultSetReader(
+                resultSet, FSFactory.getFs(resPath).read(resPath));
+      } catch (IOException e) {
+        logger.warn("ResultSetReaderFactory fs read failed", e);
+      }
+      if (reader instanceof StorageResultSetReader) {
+        ((StorageResultSetReader<?, ?>) reader).setFs(FSFactory.getFs(resPath));
+      }
+      return (StorageResultSetReader<?, ?>) reader;
+    }
+  }
+
+  public static ResultSetReader getTableResultReader(String res) {
+    ResultSetFactory rsFactory = ResultSetFactory.getInstance();
+    if (rsFactory.isResultSet(res)) {
+      ResultSet<?, ?> resultSet = rsFactory.getResultSet(res);
+      if (!ResultSetFactory.TABLE_TYPE.equals(resultSet.resultSetType())) {
+        throw new StorageWarnException(
+            LinkisStorageErrorCodeSummary.TABLE_ARE_NOT_SUPPORTED.getErrorCode(),
+            LinkisStorageErrorCodeSummary.TABLE_ARE_NOT_SUPPORTED.getErrorDesc());
+      }
+      return ResultSetReaderFactory.<TableMetaData, TableRecord>getResultSetReader(
+          (TableResultSet) resultSet, res);
+    } else {
+      FsPath resPath = new FsPath(res);
+      ResultSet<?, ?> resultSet = rsFactory.getResultSetByPath(resPath);
+      if (!ResultSetFactory.TABLE_TYPE.equals(resultSet.resultSetType())) {
+        throw new StorageWarnException(
+            LinkisStorageErrorCodeSummary.TABLE_ARE_NOT_SUPPORTED.getErrorCode(),
+            LinkisStorageErrorCodeSummary.TABLE_ARE_NOT_SUPPORTED.getErrorDesc());
+      }
+
+      Fs fs = FSFactory.getFs(resPath);
+      try {
+        fs.init(null);
+        InputStream read = fs.read(resPath);
+
+        return ResultSetReaderFactory.<TableMetaData, TableRecord>getResultSetReader(
+            (TableResultSet) resultSet, read);
+      } catch (IOException e) {
+        throw new StorageWarnException(
+            LinkisStorageErrorCodeSummary.TABLE_ARE_NOT_SUPPORTED.getErrorCode(),
+            LinkisStorageErrorCodeSummary.TABLE_ARE_NOT_SUPPORTED.getErrorDesc());
+      }
+    }
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetWriterFactory.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetWriterFactory.java
new file mode 100644
index 000000000..1abeaf093
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetWriterFactory.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset;
+
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.common.io.resultset.ResultSet;
+import org.apache.linkis.common.io.resultset.ResultSetReader;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ResultSetWriterFactory {
+  private static final Logger logger = LoggerFactory.getLogger(ResultSetWriterFactory.class);
+
+  public static <K extends MetaData, V extends Record>
+      org.apache.linkis.common.io.resultset.ResultSetWriter<K, V> getResultSetWriter(
+          ResultSet<K, V> resultSet, long maxCacheSize, FsPath storePath) {
+    return new StorageResultSetWriter<>(resultSet, maxCacheSize, storePath);
+  }
+
+  public static <K extends MetaData, V extends Record>
+      org.apache.linkis.common.io.resultset.ResultSetWriter<K, V> getResultSetWriter(
+          ResultSet<K, V> resultSet, long maxCacheSize, FsPath storePath, String proxyUser) {
+    StorageResultSetWriter<K, V> writer =
+        new StorageResultSetWriter<>(resultSet, maxCacheSize, storePath);
+    writer.setProxyUser(proxyUser);
+    return writer;
+  }
+
+  public static Record[] getRecordByWriter(
+      org.apache.linkis.common.io.resultset.ResultSetWriter<? extends MetaData, ? extends Record>
+          writer,
+      long limit) {
+    String res = writer.toString();
+    return getRecordByRes(res, limit);
+  }
+
+  public static Record[] getRecordByRes(String res, long limit) {
+    ResultSetReader reader = ResultSetReaderFactory.getResultSetReader(res);
+    int count = 0;
+    List<Record> records = new ArrayList<>();
+    try {
+      reader.getMetaData();
+      while (reader.hasNext() && count < limit) {
+        records.add(reader.getRecord());
+        count++;
+      }
+    } catch (IOException e) {
+      logger.warn("ResultSetWriter getRecordByRes failed", e);
+    }
+    return records.toArray(new Record[0]);
+  }
+
+  public static Record getLastRecordByRes(String res) {
+    ResultSetReader reader = ResultSetReaderFactory.getResultSetReader(res);
+    Record record = null;
+    try {
+      reader.getMetaData();
+      while (reader.hasNext()) {
+        record = reader.getRecord();
+      }
+    } catch (IOException e) {
+      logger.warn("ResultSetWriter getLastRecordByRes failed", e);
+    }
+    return record;
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSet.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSet.java
new file mode 100644
index 000000000..c83661de2
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSet.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset;
+
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.common.io.resultset.ResultSet;
+import org.apache.linkis.storage.domain.Dolphin;
+import org.apache.linkis.storage.utils.StorageConfiguration;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class StorageResultSet<K extends MetaData, V extends Record>
+    implements ResultSet<K, V> {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageResultSet.class);
+
+  private byte[] resultHeaderBytes = null;
+
+  {
+    byte[] arr2 = Dolphin.getIntBytes(Integer.parseInt(resultSetType()));
+    byte[] mergedArray = new byte[Dolphin.MAGIC_BYTES.length + arr2.length];
+    System.arraycopy(Dolphin.MAGIC_BYTES, 0, mergedArray, 0, Dolphin.MAGIC_BYTES.length);
+    System.arraycopy(arr2, 0, mergedArray, Dolphin.MAGIC_BYTES.length, arr2.length);
+    resultHeaderBytes = mergedArray;
+  }
+
+  @Override
+  public String charset() {
+    return StorageConfiguration.STORAGE_RS_FILE_TYPE.getValue();
+  }
+
+  @Override
+  public FsPath getResultSetPath(FsPath parentDir, String fileName) {
+    final String path =
+        parentDir.getPath().endsWith("/")
+            ? parentDir.getUriString() + fileName + Dolphin.DOLPHIN_FILE_SUFFIX
+            : parentDir.getUriString() + "/" + fileName + Dolphin.DOLPHIN_FILE_SUFFIX;
+    logger.info("Get result set path: {}", path);
+    return new FsPath(path);
+  }
+
+  @Override
+  public byte[] getResultSetHeader() {
+    return resultHeaderBytes;
+  }
+
+  @Override
+  public boolean belongToPath(String path) {
+    return path.endsWith(Dolphin.DOLPHIN_FILE_SUFFIX);
+  }
+
+  @Override
+  public boolean belongToResultSet(String content) {
+    try {
+      return Dolphin.getType(content).equals(resultSetType());
+    } catch (Exception e) {
+      logger.info("Wrong result set: ", e);
+      return false;
+    }
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetReader.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetReader.java
new file mode 100644
index 000000000..35f7483c8
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetReader.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset;
+
+import org.apache.linkis.common.io.Fs;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.common.io.resultset.ResultDeserializer;
+import org.apache.linkis.common.io.resultset.ResultSet;
+import org.apache.linkis.common.io.resultset.ResultSetReader;
+import org.apache.linkis.storage.domain.Dolphin;
+import org.apache.linkis.storage.exception.StorageWarnException;
+import org.apache.linkis.storage.utils.StorageUtils;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StorageResultSetReader<K extends MetaData, V extends Record>
+    extends ResultSetReader<K, V> {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageResultSetReader.class);
+
+  private final ResultSet<K, V> resultSet;
+  private final InputStream inputStream;
+  private final ResultDeserializer<K, V> deserializer;
+  private K metaData;
+  private Record row;
+  private int colCount = 0;
+  private int rowCount = 0;
+  private Fs fs;
+
+  private final int READ_CACHE = 1024;
+  private final byte[] bytes = new byte[READ_CACHE];
+
+  public StorageResultSetReader(ResultSet<K, V> resultSet, InputStream inputStream) {
+    super(resultSet, inputStream);
+    this.resultSet = resultSet;
+    this.inputStream = inputStream;
+    this.deserializer = resultSet.createResultSetDeserializer();
+  }
+
+  public StorageResultSetReader(ResultSet<K, V> resultSet, String value) {
+    this(resultSet, new ByteArrayInputStream(value.getBytes(Dolphin.CHAR_SET)));
+  }
+
+  public void init() throws IOException {
+    String resType = Dolphin.getType(inputStream);
+    if (!StringUtils.equals(resultSet.resultSetType(), resType)) {
+      throw new RuntimeException(
+          "File type does not match(文件类型不匹配): "
+              + ResultSetFactory.resultSetType.getOrDefault(resType, "TABLE"));
+    }
+  }
+
+  public byte[] readLine() {
+    int rowLen = 0;
+    try {
+      rowLen = Dolphin.readInt(inputStream);
+    } catch (StorageWarnException | IOException e) {
+      logger.info("Read finished(读取完毕)");
+      return null;
+    }
+
+    byte[] rowBuffer = new byte[0];
+    int len = 0;
+
+    while (rowLen > 0 && len >= 0) {
+      if (rowLen > READ_CACHE) {
+        len = StorageUtils.readBytes(inputStream, bytes, READ_CACHE);
+      } else {
+        len = StorageUtils.readBytes(inputStream, bytes, rowLen);
+      }
+
+      if (len > 0) {
+        rowLen -= len;
+        rowBuffer = Arrays.copyOf(rowBuffer, rowBuffer.length + len);
+        System.arraycopy(bytes, 0, rowBuffer, rowBuffer.length - len, len);
+      }
+    }
+    rowCount++;
+    return rowBuffer;
+  }
+
+  @Override
+  public Record getRecord() {
+    if (metaData == null) throw new RuntimeException("Must read metadata first(必须先读取metadata)");
+    if (row == null) {
+      throw new RuntimeException(
+          "Can't get the value of the field, maybe the IO stream has been read or has been closed!(拿不到字段的值,也许IO流已读取完毕或已被关闭!)");
+    }
+    return row;
+  }
+
+  public void setFs(Fs fs) {
+    this.fs = fs;
+  }
+
+  public Fs getFs() {
+    return fs;
+  }
+
+  @Override
+  public MetaData getMetaData() {
+    if (metaData == null) {
+      try {
+        init();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    metaData = deserializer.createMetaData(readLine());
+    return metaData;
+  }
+
+  @Override
+  public int skip(int recordNum) throws IOException {
+    if (recordNum < 0) return -1;
+
+    if (metaData == null) getMetaData();
+    for (int i = recordNum; i > 0; i--) {
+      try {
+        inputStream.skip(Dolphin.readInt(inputStream));
+      } catch (Throwable t) {
+        return recordNum - i;
+      }
+    }
+    return recordNum;
+  }
+
+  @Override
+  public long getPosition() throws IOException {
+    return rowCount;
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (metaData == null) getMetaData();
+    byte[] line = readLine();
+    if (line == null) return false;
+    row = deserializer.createRecord(line);
+    if (row == null) return false;
+    return true;
+  }
+
+  @Override
+  public long available() throws IOException {
+    return inputStream.available();
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.closeQuietly(inputStream);
+    if (this.fs != null) {
+      this.fs.close();
+    }
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetWriter.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetWriter.java
new file mode 100644
index 000000000..5109ed44d
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetWriter.java
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset;
+
+import org.apache.linkis.common.io.*;
+import org.apache.linkis.common.io.resultset.*;
+import org.apache.linkis.common.io.resultset.ResultSetWriter;
+import org.apache.linkis.common.utils.*;
+import org.apache.linkis.storage.*;
+import org.apache.linkis.storage.conf.*;
+import org.apache.linkis.storage.domain.*;
+import org.apache.linkis.storage.utils.*;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StorageResultSetWriter<K extends MetaData, V extends Record>
+    extends ResultSetWriter<K, V> {
+  private static final Logger logger = LoggerFactory.getLogger(StorageResultSetWriter.class);
+
+  private final ResultSet<K, V> resultSet;
+  private final long maxCacheSize;
+  private final FsPath storePath;
+
+  private final ResultSerializer serializer;
+  private boolean moveToWriteRow = false;
+  private OutputStream outputStream = null;
+  private int rowCount = 0;
+  private final List<Byte> buffer = new ArrayList<Byte>();
+  private Fs fs = null;
+  private MetaData rMetaData = null;
+  private String proxyUser = StorageUtils.getJvmUser();
+  private boolean fileCreated = false;
+  private boolean closed = false;
+  private final Object WRITER_LOCK_CREATE = new Object();
+  private final Object WRITER_LOCK_CLOSE = new Object();
+
+  public StorageResultSetWriter(ResultSet<K, V> resultSet, long maxCacheSize, FsPath storePath) {
+    super(resultSet, maxCacheSize, storePath);
+    this.resultSet = resultSet;
+    this.maxCacheSize = maxCacheSize;
+    this.storePath = storePath;
+
+    this.serializer = resultSet.createResultSetSerializer();
+  }
+
+  public MetaData getMetaData() {
+    return rMetaData;
+  }
+
+  public void setProxyUser(String proxyUser) {
+    this.proxyUser = proxyUser;
+  }
+
+  public boolean isEmpty() {
+    return rMetaData == null && buffer.size() <= Dolphin.FILE_EMPTY;
+  }
+
+  public void init() {
+    try {
+      writeLine(resultSet.getResultSetHeader(), true);
+    } catch (IOException e) {
+      logger.warn("StorageResultSetWriter init failed", e);
+    }
+  }
+
+  public void createNewFile() {
+    if (!fileCreated) {
+      synchronized (WRITER_LOCK_CREATE) {
+        if (!fileCreated) {
+          if (storePath != null && outputStream == null) {
+            logger.info("Try to create a new file:{}, with proxy user:{}", storePath, proxyUser);
+            fs = FSFactory.getFsByProxyUser(storePath, proxyUser);
+            try {
+              fs.init(null);
+              FileSystemUtils.createNewFile(storePath, proxyUser, true);
+              outputStream = fs.write(storePath, true);
+            } catch (IOException e) {
+              logger.warn("StorageResultSetWriter createNewFile failed", e);
+            }
+            logger.info("Succeed to create a new file:{}", storePath);
+            fileCreated = true;
+          }
+        }
+      }
+    } else if (storePath != null && outputStream == null) {
+      logger.warn("outputStream had been set null, but createNewFile() was called again.");
+    }
+  }
+
+  public void writeLine(byte[] bytes, boolean cache) throws IOException {
+    if (closed) {
+      logger.warn("the writer had been closed, but writeLine() was still called.");
+      return;
+    }
+    if (bytes.length > LinkisStorageConf.ROW_BYTE_MAX_LEN) {
+      throw new IOException(
+          String.format(
+              "A single row of data cannot exceed %s", LinkisStorageConf.ROW_BYTE_MAX_LEN_STR));
+    }
+    if (buffer.size() > maxCacheSize && !cache) {
+      if (outputStream == null) {
+        createNewFile();
+      }
+      flush();
+      outputStream.write(bytes);
+    } else {
+      for (byte b : bytes) {
+        buffer.add(b);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    if (outputStream == null) {
+      if (isEmpty()) {
+        return "";
+      }
+
+      byte[] byteArray = getBytes();
+      return new String(byteArray, Dolphin.CHAR_SET);
+    }
+    return storePath.getSchemaPath();
+  }
+
+  private byte[] getBytes() {
+    byte[] byteArray = new byte[buffer.size()];
+    for (int i = 0; i < buffer.size(); i++) {
+      byteArray[i] = buffer.get(i);
+    }
+    return byteArray;
+  }
+
+  @Override
+  public FsPath toFSPath() {
+    return storePath;
+  }
+
+  @Override
+  public void addMetaDataAndRecordString(String content) {
+    if (!moveToWriteRow) {
+      byte[] bytes = content.getBytes(Dolphin.CHAR_SET);
+      try {
+        writeLine(bytes, false);
+      } catch (IOException e) {
+        logger.warn("addMetaDataAndRecordString failed", e);
+      }
+    }
+    moveToWriteRow = true;
+  }
+
+  @Override
+  public void addRecordString(String content) {}
+
+  @Override
+  public void addMetaData(MetaData metaData) throws IOException {
+    if (!moveToWriteRow) {
+      rMetaData = metaData;
+      init();
+      if (metaData == null) {
+        writeLine(serializer.metaDataToBytes(metaData), true);
+      } else {
+        writeLine(serializer.metaDataToBytes(metaData), false);
+      }
+      moveToWriteRow = true;
+    }
+  }
+
+  @Override
+  public void addRecord(Record record) {
+    if (moveToWriteRow) {
+      rowCount++;
+      try {
+        writeLine(serializer.recordToBytes(record), false);
+      } catch (IOException e) {
+        logger.warn("addMetaDataAndRecordString failed", e);
+      }
+    }
+  }
+
+  public void closeFs() {
+    if (fs != null) {
+      IOUtils.closeQuietly(fs);
+      fs = null;
+    }
+  }
+
+  @Override
+  public void close() {
+    if (closed) {
+      logger.warn("the writer had been closed, but close() was still called.");
+      return;
+    }
+    synchronized (WRITER_LOCK_CLOSE) {
+      if (!closed) {
+        closed = true;
+      } else {
+        return;
+      }
+    }
+    try {
+      if (outputStream != null) {
+        flush();
+      }
+    } finally {
+      if (outputStream != null) {
+        IOUtils.closeQuietly(outputStream);
+        outputStream = null;
+      }
+      closeFs();
+    }
+  }
+
+  @Override
+  public void flush() {
+    createNewFile();
+    if (outputStream != null) {
+      try {
+        if (!buffer.isEmpty()) {
+          outputStream.write(getBytes());
+          buffer.clear();
+        }
+        if (outputStream instanceof HdfsDataOutputStream) {
+          ((HdfsDataOutputStream) outputStream).hflush();
+        } else {
+          outputStream.flush();
+        }
+      } catch (IOException e) {
+        logger.warn("Error encountered when flush result set", e);
+      }
+    }
+    if (closed && logger.isDebugEnabled()) {
+      logger.debug("the writer had been closed, but flush() was still called.");
+    }
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/html/HtmlResultSet.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/html/HtmlResultSet.java
new file mode 100644
index 000000000..00c0e7b2a
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/html/HtmlResultSet.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset.html;
+
+import org.apache.linkis.common.io.resultset.ResultDeserializer;
+import org.apache.linkis.common.io.resultset.ResultSerializer;
+import org.apache.linkis.storage.LineMetaData;
+import org.apache.linkis.storage.LineRecord;
+import org.apache.linkis.storage.resultset.ResultSetFactory;
+import org.apache.linkis.storage.resultset.StorageResultSet;
+import org.apache.linkis.storage.resultset.txt.TextResultDeserializer;
+import org.apache.linkis.storage.resultset.txt.TextResultSerializer;
+
+import java.io.Serializable;
+
+public class HtmlResultSet extends StorageResultSet<LineMetaData, LineRecord>
+    implements Serializable {
+
+  @Override
+  public String resultSetType() {
+    return ResultSetFactory.HTML_TYPE;
+  }
+
+  @Override
+  public ResultSerializer createResultSetSerializer() {
+    return new TextResultSerializer();
+  }
+
+  @Override
+  public ResultDeserializer<LineMetaData, LineRecord> createResultSetDeserializer() {
+    return new TextResultDeserializer();
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineMetaData.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOMetaData.java
similarity index 66%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineMetaData.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOMetaData.java
index e6c9df454..bc450b5ea 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineMetaData.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOMetaData.java
@@ -15,18 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage
+package org.apache.linkis.storage.resultset.io;
 
-import org.apache.linkis.common.io.MetaData
-import org.apache.linkis.storage.resultset.ResultMetaData
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.storage.resultset.ResultMetaData;
 
-class LineMetaData(private var metaData: String = null) extends ResultMetaData {
+public class IOMetaData implements ResultMetaData {
 
-  def getMetaData: String = metaData
+  public int off;
+  public int len;
 
-  def setMetaData(metaData: String): Unit = {
-    this.metaData = metaData
+  public IOMetaData(int off, int len) {
+    this.off = off;
+    this.len = len;
   }
 
-  override def cloneMeta(): MetaData = new LineMetaData(metaData)
+  @Override
+  public MetaData cloneMeta() {
+    return new IOMetaData(off, len);
+  }
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableMetaData.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IORecord.java
similarity index 69%
copy from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableMetaData.scala
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IORecord.java
index 526078d6c..90bcbe9da 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableMetaData.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IORecord.java
@@ -15,16 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset.table
+package org.apache.linkis.storage.resultset.io;
 
-import org.apache.linkis.common.io.MetaData
-import org.apache.linkis.storage.domain.Column
-import org.apache.linkis.storage.resultset.ResultMetaData
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.storage.resultset.ResultRecord;
 
-class TableMetaData(val columns: Array[Column]) extends ResultMetaData {
+public class IORecord implements ResultRecord {
 
-  override def cloneMeta(): MetaData = {
-    new TableMetaData(columns)
+  public byte[] value;
+
+  public IORecord(byte[] value) {
+    this.value = value;
   }
 
+  @Override
+  public Record cloneRecord() {
+    return new IORecord(value);
+  }
 }
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOResultDeserializer.java
similarity index 54%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOResultDeserializer.java
index babeb1f6c..02d83aa7b 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOResultDeserializer.java
@@ -15,16 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.resultset.io;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.io.resultset.ResultDeserializer;
+import org.apache.linkis.storage.domain.Dolphin;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+import org.apache.commons.codec.binary.Base64;
+
+public class IOResultDeserializer extends ResultDeserializer<IOMetaData, IORecord> {
+
+  @Override
+  public IOMetaData createMetaData(byte[] bytes) {
+    String[] values = Dolphin.getString(bytes, 0, bytes.length).split(Dolphin.COL_SPLIT);
+    return new IOMetaData(Integer.parseInt(values[0]), Integer.parseInt(values[1]));
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public IORecord createRecord(byte[] bytes) {
+    return new IORecord(Base64.decodeBase64(Dolphin.getString(bytes, 0, bytes.length)));
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOResultSerializer.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOResultSerializer.java
new file mode 100644
index 000000000..2401e361a
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOResultSerializer.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset.io;
+
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.common.io.resultset.ResultSerializer;
+import org.apache.linkis.storage.domain.Dolphin;
+import org.apache.linkis.storage.utils.StorageUtils;
+
+import org.apache.commons.codec.binary.Base64;
+
+public class IOResultSerializer extends ResultSerializer {
+
+  @Override
+  public byte[] metaDataToBytes(MetaData metaData) {
+    IOMetaData ioMetaData = (IOMetaData) metaData;
+    return lineToBytes(ioMetaData.off + Dolphin.COL_SPLIT + ioMetaData.len);
+  }
+
+  @Override
+  public byte[] recordToBytes(Record record) {
+    IORecord ioRecord = (IORecord) record;
+    return lineToBytes(Base64.encodeBase64String(ioRecord.value));
+  }
+
+  private byte[] lineToBytes(String value) {
+    byte[] bytes = value == null ? Dolphin.NULL_BYTES : Dolphin.getBytes(value);
+    byte[] intBytes = Dolphin.getIntBytes(bytes.length);
+    return StorageUtils.mergeByteArrays(intBytes, bytes);
+  }
+}
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOResultSet.java
similarity index 52%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOResultSet.java
index babeb1f6c..67f8e7690 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/io/IOResultSet.java
@@ -15,16 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.resultset.io;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.io.resultset.ResultDeserializer;
+import org.apache.linkis.common.io.resultset.ResultSerializer;
+import org.apache.linkis.storage.resultset.ResultSetFactory;
+import org.apache.linkis.storage.resultset.StorageResultSet;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+import java.io.Serializable;
+
+public class IOResultSet extends StorageResultSet<IOMetaData, IORecord> implements Serializable {
+
+  @Override
+  public String resultSetType() {
+    return ResultSetFactory.IO_TYPE;
+  }
+
+  @Override
+  public ResultSerializer createResultSetSerializer() {
+    return new IOResultSerializer();
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public ResultDeserializer<IOMetaData, IORecord> createResultSetDeserializer() {
+    return new IOResultDeserializer();
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/picture/PictureResultSet.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/picture/PictureResultSet.java
new file mode 100644
index 000000000..5e73592a7
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/picture/PictureResultSet.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset.picture;
+
+import org.apache.linkis.common.io.resultset.ResultDeserializer;
+import org.apache.linkis.common.io.resultset.ResultSerializer;
+import org.apache.linkis.storage.LineMetaData;
+import org.apache.linkis.storage.LineRecord;
+import org.apache.linkis.storage.resultset.ResultSetFactory;
+import org.apache.linkis.storage.resultset.StorageResultSet;
+import org.apache.linkis.storage.resultset.txt.TextResultDeserializer;
+import org.apache.linkis.storage.resultset.txt.TextResultSerializer;
+
+import java.io.Serializable;
+
+public class PictureResultSet extends StorageResultSet<LineMetaData, LineRecord>
+    implements Serializable {
+
+  @Override
+  public String resultSetType() {
+    return ResultSetFactory.PICTURE_TYPE;
+  }
+
+  @Override
+  public ResultSerializer createResultSetSerializer() {
+    return new TextResultSerializer();
+  }
+
+  @Override
+  public ResultDeserializer<LineMetaData, LineRecord> createResultSetDeserializer() {
+    return new TextResultDeserializer();
+  }
+}
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableMetaData.java
similarity index 58%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableMetaData.java
index babeb1f6c..429ab33c8 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableMetaData.java
@@ -15,16 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.resultset.table;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.storage.domain.Column;
+import org.apache.linkis.storage.resultset.ResultMetaData;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+public class TableMetaData implements ResultMetaData {
+
+  public Column[] columns;
+
+  public TableMetaData(Column[] columns) {
+    this.columns = columns;
+  }
+
+  public Column[] getColumns() {
+    return columns;
+  }
+
+  public void setColumns(Column[] columns) {
+    this.columns = columns;
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public MetaData cloneMeta() {
+    return new TableMetaData(columns.clone());
   }
 }
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableRecord.java
similarity index 56%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableRecord.java
index babeb1f6c..f13d42b0b 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableRecord.java
@@ -15,16 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.resultset.table;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.storage.resultset.ResultRecord;
+import org.apache.linkis.storage.utils.StorageUtils;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+import java.util.Arrays;
+
+public class TableRecord implements ResultRecord {
+
+  public final Object[] row;
+
+  public TableRecord(Object[] row) {
+    this.row = row;
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public Record cloneRecord() {
+    return new TableRecord(row.clone());
+  }
+
+  public String[] tableRecordToString(String nullValue) {
+    return Arrays.stream(row)
+        .map(col -> StorageUtils.colToString(col, nullValue))
+        .toArray(String[]::new);
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableResultDeserializer.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableResultDeserializer.java
new file mode 100644
index 000000000..7b7838fd2
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableResultDeserializer.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset.table;
+
+import org.apache.linkis.common.io.resultset.ResultDeserializer;
+import org.apache.linkis.storage.domain.Column;
+import org.apache.linkis.storage.domain.DataType;
+import org.apache.linkis.storage.domain.Dolphin;
+import org.apache.linkis.storage.exception.StorageWarnException;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.PARSING_METADATA_FAILED;
+
+public class TableResultDeserializer extends ResultDeserializer<TableMetaData, TableRecord> {
+  private static final Logger logger = LoggerFactory.getLogger(TableResultDeserializer.class);
+
+  private TableMetaData metaData;
+
+  @Override
+  public TableMetaData createMetaData(byte[] bytes) {
+    int colByteLen = Integer.parseInt(Dolphin.getString(bytes, 0, Dolphin.INT_LEN));
+    String colString = Dolphin.getString(bytes, Dolphin.INT_LEN, colByteLen);
+    String[] colArray =
+        colString.endsWith(Dolphin.COL_SPLIT)
+            ? colString.substring(0, colString.length() - 1).split(Dolphin.COL_SPLIT)
+            : colString.split(Dolphin.COL_SPLIT);
+    int index = Dolphin.INT_LEN + colByteLen;
+    if (colArray.length % 3 != 0) {
+      throw new StorageWarnException(
+          PARSING_METADATA_FAILED.getErrorCode(), PARSING_METADATA_FAILED.getErrorDesc());
+    }
+    List<Column> columns = new ArrayList<>();
+    for (int i = 0; i < colArray.length; i += 3) {
+      int len = Integer.parseInt(colArray[i]);
+      String colName = Dolphin.getString(bytes, index, len);
+      index += len;
+      len = Integer.parseInt(colArray[i + 1]);
+      String colType = Dolphin.getString(bytes, index, len);
+      index += len;
+      len = Integer.parseInt(colArray[i + 2]);
+      String colComment = Dolphin.getString(bytes, index, len);
+      index += len;
+      columns.add(new Column(colName, DataType.toDataType(colType), colComment));
+    }
+    metaData = new TableMetaData(columns.toArray(new Column[0]));
+    return metaData;
+  }
+
+  /**
+   * colByteLen:All column fields are long(所有列字段长 记录的长度) colString:Obtain column
+   * length(获得列长):10,20,21 colArray:Column length array(列长数组) Get data by column length(通过列长获得数据)
+   *
+   * @param bytes
+   * @return
+   */
+  @Override
+  public TableRecord createRecord(byte[] bytes) {
+    int colByteLen = Integer.parseInt(Dolphin.getString(bytes, 0, Dolphin.INT_LEN));
+    String colString = Dolphin.getString(bytes, Dolphin.INT_LEN, colByteLen);
+    String[] colArray;
+    if (colString.endsWith(Dolphin.COL_SPLIT)) {
+      colArray = colString.substring(0, colString.length() - 1).split(Dolphin.COL_SPLIT);
+    } else {
+      colArray = colString.split(Dolphin.COL_SPLIT);
+    }
+    int index = Dolphin.INT_LEN + colByteLen;
+    Object[] data = new Object[colArray.length];
+    for (int i = 0; i < colArray.length; i++) {
+      int len = Integer.parseInt(colArray[i]);
+      String res = Dolphin.getString(bytes, index, len);
+      index += len;
+      if (i >= metaData.columns.length) {
+        data[i] = res;
+      } else {
+        data[i] = DataType.toValue(metaData.columns[i].getDataType(), res);
+      }
+    }
+    return new TableRecord(data);
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableResultSerializer.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableResultSerializer.java
new file mode 100644
index 000000000..6abe4c56d
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableResultSerializer.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset.table;
+
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.common.io.resultset.ResultSerializer;
+import org.apache.linkis.storage.domain.Column;
+import org.apache.linkis.storage.domain.Dolphin;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class TableResultSerializer extends ResultSerializer {
+
+  @Override
+  public byte[] metaDataToBytes(MetaData metaData) {
+    TableMetaData tableMetaData = (TableMetaData) metaData;
+    Object[] objects =
+        Arrays.stream(tableMetaData.columns).map(Column::toArray).flatMap(Arrays::stream).toArray();
+    return lineToBytes(objects);
+  }
+
+  @Override
+  public byte[] recordToBytes(Record record) {
+    TableRecord tableRecord = (TableRecord) record;
+    return lineToBytes(tableRecord.row);
+  }
+
+  /**
+   * Convert a row of data to an array of Bytes Convert the data to byte and get the corresponding
+   * total byte length to write to the file Data write format: line length (fixed length) column
+   * length (fixed length) field index comma segmentation real data For example:
+   * 000000004900000000116,10,3,4,5,peace1johnnwang1101true11.51 The length of the line does not
+   * include its own length 将一行数据转换为Bytes的数组 对数据转换为byte,并获取相应的总byte长度写入文件 数据写入格式:行长(固定长度) 列长(固定长度)
+   * 字段索引逗号分割 真实数据 如:000000004900000000116,10,3,4,5,peace1johnnwang1101true11.51 其中行长不包括自身长度
+   *
+   * @param line
+   */
+  private byte[] lineToBytes(Object[] line) {
+    // Data cache(数据缓存)
+    List<byte[]> dataBytes = new ArrayList<>();
+    // Column cache(列缓存)
+    List<byte[]> colIndex = new ArrayList<>();
+    int colByteLen = 0;
+    int length = 0;
+    for (Object data : line) {
+      byte[] bytes = data == null ? Dolphin.NULL_BYTES : Dolphin.getBytes(data);
+      dataBytes.add(bytes);
+      byte[] colBytes = Dolphin.getBytes(bytes.length);
+      colIndex.add(colBytes);
+      colIndex.add(Dolphin.COL_SPLIT_BYTES);
+      colByteLen += colBytes.length + Dolphin.COL_SPLIT_LEN;
+      length += bytes.length;
+    }
+    length += colByteLen + Dolphin.INT_LEN;
+    return toByteArray(length, colByteLen, colIndex, dataBytes);
+  }
+
+  /**
+   * Splice a row of data into a byte array(将一行的数据拼接成byte数组)
+   *
+   * @param length The total length of the line data byte, excluding its own
+   *     length(行数据byte总长度,不包括自身的长度)
+   * @param colByteLen Record field index byte column length(记录字段索引byte的列长)
+   * @param colIndex Field index, including separator comma(字段索引,包括分割符逗号)
+   * @param dataBytes Byte of real data(真实数据的byte)
+   * @return
+   */
+  public static byte[] toByteArray(
+      int length, int colByteLen, List<byte[]> colIndex, List<byte[]> dataBytes) {
+    List<Byte> row = new ArrayList<>();
+    colIndex.addAll(dataBytes);
+
+    for (byte intByte : Dolphin.getIntBytes(length)) {
+      row.add(intByte);
+    }
+
+    for (byte colByte : Dolphin.getIntBytes(colByteLen)) {
+      row.add(colByte);
+    }
+
+    colIndex.forEach(
+        bytes -> {
+          for (byte b : bytes) {
+            row.add(b);
+          }
+        });
+    byte[] result = new byte[row.size()];
+    for (int i = 0; i < result.length; i++) {
+      result[i] = row.get(i);
+    }
+    return result;
+  }
+}
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableResultSet.java
similarity index 51%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableResultSet.java
index babeb1f6c..5f356af0c 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/table/TableResultSet.java
@@ -15,16 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.resultset.table;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.io.resultset.ResultDeserializer;
+import org.apache.linkis.common.io.resultset.ResultSerializer;
+import org.apache.linkis.storage.resultset.ResultSetFactory;
+import org.apache.linkis.storage.resultset.StorageResultSet;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+import java.io.Serializable;
+
+public class TableResultSet extends StorageResultSet<TableMetaData, TableRecord>
+    implements Serializable {
+
+  @Override
+  public String resultSetType() {
+    return ResultSetFactory.TABLE_TYPE;
+  }
+
+  @Override
+  public ResultSerializer createResultSetSerializer() {
+    return new TableResultSerializer();
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public ResultDeserializer<TableMetaData, TableRecord> createResultSetDeserializer() {
+    return new TableResultDeserializer();
   }
 }
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/txt/TextResultDeserializer.java
similarity index 57%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/txt/TextResultDeserializer.java
index babeb1f6c..3165e5af8 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/txt/TextResultDeserializer.java
@@ -15,16 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.resultset.txt;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.io.resultset.ResultDeserializer;
+import org.apache.linkis.storage.LineMetaData;
+import org.apache.linkis.storage.LineRecord;
+import org.apache.linkis.storage.domain.Dolphin;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+public class TextResultDeserializer extends ResultDeserializer<LineMetaData, LineRecord> {
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public LineMetaData createMetaData(byte[] bytes) {
+    return new LineMetaData(Dolphin.getString(bytes, 0, bytes.length));
+  }
+
+  @Override
+  public LineRecord createRecord(byte[] bytes) {
+    return new LineRecord(Dolphin.getString(bytes, 0, bytes.length));
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/txt/TextResultSerializer.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/txt/TextResultSerializer.java
new file mode 100644
index 000000000..5555ad9eb
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/txt/TextResultSerializer.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset.txt;
+
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.common.io.resultset.ResultSerializer;
+import org.apache.linkis.storage.LineMetaData;
+import org.apache.linkis.storage.LineRecord;
+import org.apache.linkis.storage.domain.Dolphin;
+import org.apache.linkis.storage.utils.StorageUtils;
+
+public class TextResultSerializer extends ResultSerializer {
+
+  @Override
+  public byte[] metaDataToBytes(MetaData metaData) {
+    if (metaData == null) {
+      return lineToBytes(null);
+    } else {
+      LineMetaData textMetaData = (LineMetaData) metaData;
+      return lineToBytes(textMetaData.getMetaData());
+    }
+  }
+
+  @Override
+  public byte[] recordToBytes(Record record) {
+    LineRecord textRecord = (LineRecord) record;
+    return lineToBytes(textRecord.getLine());
+  }
+
+  private byte[] lineToBytes(String value) {
+    byte[] bytes = (value == null) ? Dolphin.NULL_BYTES : Dolphin.getBytes(value);
+    return StorageUtils.mergeByteArrays(Dolphin.getIntBytes(bytes.length), bytes);
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/txt/TextResultSet.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/txt/TextResultSet.java
new file mode 100644
index 000000000..19fd8f9db
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/txt/TextResultSet.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.resultset.txt;
+
+import org.apache.linkis.common.io.resultset.ResultDeserializer;
+import org.apache.linkis.common.io.resultset.ResultSerializer;
+import org.apache.linkis.storage.LineMetaData;
+import org.apache.linkis.storage.LineRecord;
+import org.apache.linkis.storage.resultset.ResultSetFactory;
+import org.apache.linkis.storage.resultset.StorageResultSet;
+
+import java.io.Serializable;
+
+public class TextResultSet extends StorageResultSet<LineMetaData, LineRecord>
+    implements Serializable {
+
+  @Override
+  public String resultSetType() {
+    return ResultSetFactory.TEXT_TYPE;
+  }
+
+  @Override
+  public ResultSerializer createResultSetSerializer() {
+    return new TextResultSerializer();
+  }
+
+  @Override
+  public ResultDeserializer<LineMetaData, LineRecord> createResultSetDeserializer() {
+    return new TextResultDeserializer();
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultDeserializer.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/Compaction.java
similarity index 52%
copy from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultDeserializer.scala
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/Compaction.java
index 4e8199da0..abfbae9c9 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultDeserializer.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/Compaction.java
@@ -15,22 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset.io
+package org.apache.linkis.storage.script;
 
-import org.apache.linkis.common.io.resultset.ResultDeserializer
-import org.apache.linkis.storage.domain.Dolphin
+import org.apache.linkis.storage.script.compaction.PYScriptCompaction;
+import org.apache.linkis.storage.script.compaction.QLScriptCompaction;
+import org.apache.linkis.storage.script.compaction.ScalaScriptCompaction;
+import org.apache.linkis.storage.script.compaction.ShellScriptCompaction;
 
-import org.apache.commons.codec.binary.Base64
+public interface Compaction {
+  String prefixConf();
 
-class IOResultDeserializer extends ResultDeserializer[IOMetaData, IORecord] {
+  String prefix();
 
-  override def createMetaData(bytes: Array[Byte]): IOMetaData = {
-    val values = Dolphin.getString(bytes, 0, bytes.length).split(Dolphin.COL_SPLIT)
-    new IOMetaData(values(0).toInt, values(1).toInt)
-  }
+  boolean belongTo(String suffix);
+
+  String compact(Variable variable);
 
-  override def createRecord(bytes: Array[Byte]): IORecord = {
-    new IORecord(Base64.decodeBase64(Dolphin.getString(bytes, 0, bytes.length)))
+  public static Compaction[] listCompactions() {
+    return new Compaction[] {
+      new PYScriptCompaction(),
+      new QLScriptCompaction(),
+      new ScalaScriptCompaction(),
+      new ShellScriptCompaction()
+    };
   }
 
+  String getAnnotationSymbol();
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultMetaData.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/Parser.java
similarity index 79%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultMetaData.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/Parser.java
index 7871bb2ac..ee1ec1901 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultMetaData.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/Parser.java
@@ -15,8 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset
+package org.apache.linkis.storage.script;
 
-import org.apache.linkis.common.io.MetaData
+public interface Parser {
+  String prefixConf();
 
-abstract class ResultMetaData extends MetaData {}
+  String prefix();
+
+  boolean belongTo(String suffix);
+
+  Variable parse(String line);
+
+  String getAnnotationSymbol();
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineRecord.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ParserFactory.java
similarity index 61%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineRecord.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ParserFactory.java
index d6e3220cf..58edf3dfe 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/LineRecord.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ParserFactory.java
@@ -15,20 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage
+package org.apache.linkis.storage.script;
 
-import org.apache.linkis.common.io.Record
-import org.apache.linkis.storage.resultset.ResultRecord
+import org.apache.linkis.storage.script.parser.PYScriptParser;
+import org.apache.linkis.storage.script.parser.QLScriptParser;
+import org.apache.linkis.storage.script.parser.ScalaScriptParser;
+import org.apache.linkis.storage.script.parser.ShellScriptParser;
 
-class LineRecord(private var line: String) extends ResultRecord {
-
-  def getLine: String = line
-
-  def setLine(line: String): Unit = {
-    this.line = line
+public class ParserFactory {
+  public static Parser[] listParsers() {
+    return new Parser[] {
+      new PYScriptParser(), new QLScriptParser(), new ScalaScriptParser(), new ShellScriptParser()
+    };
   }
-
-  override def cloneRecord(): Record = new LineRecord(line)
-
-  override def toString: String = line
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/ScriptFsReader.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptFsReader.java
similarity index 60%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/ScriptFsReader.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptFsReader.java
index d89074b97..00b79eb7e 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/ScriptFsReader.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptFsReader.java
@@ -15,23 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.script
+package org.apache.linkis.storage.script;
 
-import org.apache.linkis.common.io.{FsPath, FsReader}
-import org.apache.linkis.storage.script.reader.StorageScriptFsReader
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.common.io.FsReader;
+import org.apache.linkis.storage.script.reader.StorageScriptFsReader;
 
-import java.io.InputStream
+import java.io.InputStream;
 
-abstract class ScriptFsReader extends FsReader {
+public abstract class ScriptFsReader extends FsReader {
 
-  val path: FsPath
-  val charset: String
+  protected FsPath path;
+  protected String charset;
 
-}
-
-object ScriptFsReader {
-
-  def getScriptFsReader(path: FsPath, charset: String, inputStream: InputStream): ScriptFsReader =
-    new StorageScriptFsReader(path, charset, inputStream)
+  public ScriptFsReader(FsPath path, String charset) {
+    this.path = path;
+    this.charset = charset;
+  }
 
+  public static ScriptFsReader getScriptFsReader(
+      FsPath path, String charset, InputStream inputStream) {
+    return new StorageScriptFsReader(path, charset, inputStream);
+  }
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/txt/TextResultDeserializer.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptFsWriter.java
similarity index 58%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/txt/TextResultDeserializer.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptFsWriter.java
index b306b1f29..835e1bfb2 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/txt/TextResultDeserializer.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptFsWriter.java
@@ -15,20 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset.txt
+package org.apache.linkis.storage.script;
 
-import org.apache.linkis.common.io.resultset.ResultDeserializer
-import org.apache.linkis.storage.{LineMetaData, LineRecord}
-import org.apache.linkis.storage.domain.Dolphin
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.common.io.FsWriter;
+import org.apache.linkis.storage.script.writer.StorageScriptFsWriter;
 
-class TextResultDeserializer extends ResultDeserializer[LineMetaData, LineRecord] {
+import java.io.InputStream;
+import java.io.OutputStream;
 
-  override def createMetaData(bytes: Array[Byte]): LineMetaData = {
-    new LineMetaData(Dolphin.getString(bytes, 0, bytes.length))
-  }
+public abstract class ScriptFsWriter extends FsWriter {
+  FsPath path;
+  String charset;
 
-  override def createRecord(bytes: Array[Byte]): LineRecord = {
-    new LineRecord(Dolphin.getString(bytes, 0, bytes.length))
-  }
+  public abstract InputStream getInputStream();
 
+  public static ScriptFsWriter getScriptFsWriter(
+      FsPath path, String charset, OutputStream outputStream) {
+    return new StorageScriptFsWriter(path, charset, outputStream);
+  }
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/CSVFsWriter.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptMetaData.java
similarity index 64%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/CSVFsWriter.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptMetaData.java
index c8ae290f2..28999aae7 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/CSVFsWriter.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptMetaData.java
@@ -15,25 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.csv
+package org.apache.linkis.storage.script;
 
-import org.apache.linkis.common.io.FsWriter
+import org.apache.linkis.common.io.MetaData;
 
-import java.io.OutputStream
+public class ScriptMetaData implements MetaData {
+  private Variable[] variables;
 
-abstract class CSVFsWriter extends FsWriter {
-  val charset: String
-  val separator: String
-  val quoteRetouchEnable: Boolean
-}
+  public ScriptMetaData(Variable[] variables) {
+    this.variables = variables;
+  }
 
-object CSVFsWriter {
+  @Override
+  public MetaData cloneMeta() {
+    return new ScriptMetaData(variables.clone());
+  }
 
-  def getCSVFSWriter(
-      charset: String,
-      separator: String,
-      quoteRetouchEnable: Boolean,
-      outputStream: OutputStream
-  ): CSVFsWriter = new StorageCSVWriter(charset, separator, quoteRetouchEnable, outputStream)
+  public Variable[] getMetaData() {
+    return variables;
+  }
 
+  public void setMetaData(Variable[] variables) {
+    this.variables = variables;
+  }
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultRecord.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptRecord.java
similarity index 80%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultRecord.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptRecord.java
index ab9244ca9..c948b9558 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultRecord.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/ScriptRecord.java
@@ -15,8 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset
+package org.apache.linkis.storage.script;
 
-import org.apache.linkis.common.io.Record
+import org.apache.linkis.storage.LineRecord;
 
-abstract class ResultRecord extends Record {}
+public class ScriptRecord extends LineRecord {
+  public ScriptRecord(String line) {
+    super(line);
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableRecord.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/Variable.java
similarity index 59%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableRecord.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/Variable.java
index d8e056096..8c726671b 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableRecord.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/Variable.java
@@ -15,22 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset.table
+package org.apache.linkis.storage.script;
 
-import org.apache.linkis.common.io.Record
-import org.apache.linkis.storage.resultset.ResultRecord
-import org.apache.linkis.storage.utils.StorageUtils
+public class Variable {
+  protected String sortParent;
+  protected String sort;
+  protected String key;
+  protected String value;
 
-class TableRecord(val row: Array[Any]) extends ResultRecord {
+  public Variable(String sortParent, String sort, String key, String value) {
+    this.sortParent = sortParent;
+    this.sort = sort;
+    this.key = key;
+    this.value = value;
+  }
+
+  public String getSortParent() {
+    return sortParent;
+  }
 
-  override def cloneRecord(): Record = {
-    new TableRecord(row)
+  public String getSort() {
+    return sort;
   }
 
-  def tableRecordToString(nullValue: String = "NULL"): Array[String] = {
-    row.map { col =>
-      StorageUtils.colToString(col, nullValue)
-    }
+  public String getKey() {
+    return key;
   }
 
+  public String getValue() {
+    return value;
+  }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/VariableParser.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/VariableParser.java
new file mode 100644
index 000000000..cd1074645
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/VariableParser.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.script;
+
+import java.util.*;
+
+public class VariableParser {
+
+  public static final String CONFIGURATION = "configuration";
+  public static final String VARIABLE = "variable";
+  public static final String RUNTIME = "runtime";
+  public static final String STARTUP = "startup";
+  public static final String SPECIAL = "special";
+
+  public static Variable[] getVariables(Map<String, Object> params) {
+    List<Variable> variables = new ArrayList<>();
+    Map<String, Object> variableMap =
+        (Map<String, Object>) params.getOrDefault(VARIABLE, new HashMap<String, Object>());
+    for (Map.Entry<String, Object> entry : variableMap.entrySet()) {
+      variables.add(new Variable(VARIABLE, null, entry.getKey(), entry.getValue().toString()));
+    }
+
+    Map<String, Object> configurationMap =
+        (Map<String, Object>) params.getOrDefault(CONFIGURATION, new HashMap<String, Object>());
+    for (Map.Entry<String, Object> entry : configurationMap.entrySet()) {
+      Map<String, Object> subMap = (Map<String, Object>) entry.getValue();
+      for (Map.Entry<String, Object> subEntry : subMap.entrySet()) {
+        if (!isContextIDINFO(subEntry.getKey())) {
+          Object value = subEntry.getValue();
+          if (value instanceof Map) {
+            Map<String, Object> innerMap = (Map<String, Object>) value;
+            for (Map.Entry<String, Object> innerEntry : innerMap.entrySet()) {
+              if (!isContextIDINFO(innerEntry.getKey())) {
+                variables.add(
+                    new Variable(
+                        entry.getKey(),
+                        subEntry.getKey(),
+                        innerEntry.getKey(),
+                        innerEntry.getValue().toString()));
+              }
+            }
+          } else {
+            if (value == null) {
+              variables.add(new Variable(CONFIGURATION, entry.getKey(), subEntry.getKey(), ""));
+            } else {
+              variables.add(
+                  new Variable(CONFIGURATION, entry.getKey(), subEntry.getKey(), value.toString()));
+            }
+          }
+        }
+      }
+    }
+    return variables.toArray(new Variable[variables.size()]);
+  }
+
+  private static boolean isContextIDINFO(String key) {
+    return "contextID".equalsIgnoreCase(key) || "nodeName".equalsIgnoreCase(key);
+  }
+
+  public static Map<String, Object> getMap(Variable[] variables) {
+    Map<String, String> variableKey2Value = new HashMap<>();
+    Map<String, Object> confs = new HashMap<>();
+
+    Arrays.stream(variables)
+        .filter(variable -> variable.sort == null)
+        .forEach(v -> variableKey2Value.put(v.key, v.value));
+
+    Arrays.stream(variables)
+        .filter(variable -> variable.sort != null)
+        .forEach(
+            v -> {
+              switch (v.getSort()) {
+                case STARTUP:
+                case RUNTIME:
+                case SPECIAL:
+                  if (!confs.containsKey(v.getSort())) {
+                    confs.put(v.getSort(), createMap(v));
+                  } else {
+                    Map<String, Object> subMap = (Map<String, Object>) confs.get(v.getSort());
+                    subMap.put(v.getKey(), v.getValue());
+                  }
+                  break;
+                default:
+                  if (!confs.containsKey(v.getSortParent())) {
+                    Map<String, Object> subMap = new HashMap<>();
+                    subMap.put(v.getSort(), createMap(v));
+                    confs.put(v.getSortParent(), subMap);
+                  } else {
+                    Map<String, Object> subMap = (Map<String, Object>) confs.get(v.getSortParent());
+                    if (!subMap.containsKey(v.getSort())) {
+                      subMap.put(v.getSort(), createMap(v));
+                    } else {
+                      Map<String, Object> innerMap = (Map<String, Object>) subMap.get(v.getSort());
+                      innerMap.put(v.getKey(), v.getValue());
+                    }
+                  }
+                  break;
+              }
+            });
+
+    Map<String, Object> params = new HashMap<>();
+    if (!variableKey2Value.isEmpty()) {
+      params.put(VARIABLE, variableKey2Value);
+    }
+    if (!confs.isEmpty()) {
+      params.put(CONFIGURATION, confs);
+    }
+    return params;
+  }
+
+  private static Map<String, Object> createMap(Variable variable) {
+    Map<String, Object> map = new HashMap<>();
+    map.put(variable.getKey(), variable.getValue());
+    return map;
+  }
+}
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/CommonScriptCompaction.java
similarity index 51%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/CommonScriptCompaction.java
index babeb1f6c..f8e01a682 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/CommonScriptCompaction.java
@@ -15,16 +15,33 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.script.compaction;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.storage.script.Compaction;
+import org.apache.linkis.storage.script.Variable;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+public abstract class CommonScriptCompaction implements Compaction {
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public String compact(Variable variable) {
+    switch (variable.getSortParent()) {
+      case "variable":
+        return prefix() + " " + variable.getKey() + "=" + variable.getValue();
+      default:
+        return prefixConf()
+            + " "
+            + variable.getSortParent()
+            + " "
+            + variable.getSort()
+            + " "
+            + variable.getKey()
+            + "="
+            + variable.getValue();
+    }
+  }
+
+  @Override
+  public String getAnnotationSymbol() {
+    return prefix().split("@")[0];
   }
 }
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/PYScriptCompaction.java
similarity index 55%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/PYScriptCompaction.java
index babeb1f6c..4022e7d46 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/PYScriptCompaction.java
@@ -15,16 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.script.compaction;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+public class PYScriptCompaction extends CommonScriptCompaction {
+
+  private static final PYScriptCompaction pYScriptCompaction = new PYScriptCompaction();
+
+  public static CommonScriptCompaction apply() {
+    return pYScriptCompaction;
+  }
+
+  @Override
+  public boolean belongTo(String suffix) {
+    return CodeAndRunTypeUtils.getSuffixBelongToLanguageTypeOrNot(
+        suffix, CodeAndRunTypeUtils.LANGUAGE_TYPE_PYTHON());
+  }
+
+  @Override
+  public String prefix() {
+    return "#@set";
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public String prefixConf() {
+    return "#conf@set";
   }
 }
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/QLScriptCompaction.java
similarity index 55%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/QLScriptCompaction.java
index babeb1f6c..97fc29efa 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/QLScriptCompaction.java
@@ -15,16 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.script.compaction;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+public class QLScriptCompaction extends CommonScriptCompaction {
+
+  private static final QLScriptCompaction qLScriptCompaction = new QLScriptCompaction();
+
+  public static CommonScriptCompaction apply() {
+    return qLScriptCompaction;
+  }
+
+  @Override
+  public boolean belongTo(String suffix) {
+    return CodeAndRunTypeUtils.getSuffixBelongToLanguageTypeOrNot(
+        suffix, CodeAndRunTypeUtils.LANGUAGE_TYPE_SQL());
+  }
+
+  @Override
+  public String prefix() {
+    return "--@set";
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public String prefixConf() {
+    return "--conf@set";
   }
 }
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/ScalaScriptCompaction.java
similarity index 55%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/ScalaScriptCompaction.java
index babeb1f6c..18d5d2f53 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/ScalaScriptCompaction.java
@@ -15,16 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.script.compaction;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+public class ScalaScriptCompaction extends CommonScriptCompaction {
+
+  private static final ScalaScriptCompaction compaction = new ScalaScriptCompaction();
+
+  public static CommonScriptCompaction apply() {
+    return compaction;
+  }
+
+  @Override
+  public String prefix() {
+    return "//@set";
+  }
+
+  @Override
+  public boolean belongTo(String suffix) {
+    return CodeAndRunTypeUtils.getSuffixBelongToLanguageTypeOrNot(
+        suffix, CodeAndRunTypeUtils.LANGUAGE_TYPE_SCALA());
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public String prefixConf() {
+    return "//conf@set";
   }
 }
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/ShellScriptCompaction.java
similarity index 55%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/ShellScriptCompaction.java
index babeb1f6c..085815b55 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/compaction/ShellScriptCompaction.java
@@ -15,16 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.script.compaction;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+public class ShellScriptCompaction extends CommonScriptCompaction {
+
+  private static final ShellScriptCompaction shellScriptCompaction = new ShellScriptCompaction();
+
+  public static CommonScriptCompaction apply() {
+    return shellScriptCompaction;
+  }
+
+  @Override
+  public String prefix() {
+    return "#@set";
+  }
+
+  @Override
+  public String prefixConf() {
+    return "#conf@set";
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public boolean belongTo(String suffix) {
+    return CodeAndRunTypeUtils.getSuffixBelongToLanguageTypeOrNot(
+        suffix, CodeAndRunTypeUtils.LANGUAGE_TYPE_SHELL());
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/CommonScriptParser.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/CommonScriptParser.java
new file mode 100644
index 000000000..0d7e9f58e
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/CommonScriptParser.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.script.parser;
+
+import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary;
+import org.apache.linkis.storage.exception.StorageWarnException;
+import org.apache.linkis.storage.script.Parser;
+import org.apache.linkis.storage.script.Variable;
+import org.apache.linkis.storage.script.VariableParser;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public abstract class CommonScriptParser implements Parser {
+
+  @Override
+  public Variable parse(String line) {
+    String variableReg = "\\s*" + prefix() + "\\s*(.+)\\s*" + "=" + "\\s*(.+)\\s*";
+    Pattern pattern = Pattern.compile(variableReg);
+    Matcher matcher = pattern.matcher(line);
+    if (matcher.matches()) {
+      String key = matcher.group(1).trim();
+      String value = matcher.group(2).trim();
+      return new Variable(VariableParser.VARIABLE, null, key, value);
+
+    } else {
+      String[] splitLine = line.split("=");
+      if (splitLine.length != 2) {
+        throw new StorageWarnException(
+            LinkisStorageErrorCodeSummary.INVALID_CUSTOM_PARAMETER.getErrorCode(),
+            LinkisStorageErrorCodeSummary.INVALID_CUSTOM_PARAMETER.getErrorDesc());
+      }
+      String[] subSplit =
+          Arrays.stream(splitLine[0].split(" "))
+              .filter(str -> !"".equals(str))
+              .toArray(String[]::new);
+      if (subSplit.length != 4) {
+        throw new StorageWarnException(
+            LinkisStorageErrorCodeSummary.INVALID_CUSTOM_PARAMETER.getErrorCode(),
+            LinkisStorageErrorCodeSummary.INVALID_CUSTOM_PARAMETER.getErrorDesc());
+      }
+      if (!subSplit[0].trim().equals(prefixConf())) {
+        throw new StorageWarnException(
+            LinkisStorageErrorCodeSummary.INVALID_CUSTOM_PARAMETER.getErrorCode(),
+            LinkisStorageErrorCodeSummary.INVALID_CUSTOM_PARAMETER.getErrorDesc());
+      }
+      String sortParent = subSplit[1].trim();
+      String sort = subSplit[2].trim();
+      String key = subSplit[3].trim();
+      String value = splitLine[1].trim();
+      return new Variable(sortParent, sort, key, value);
+    }
+  }
+
+  @Override
+  public String getAnnotationSymbol() {
+    return prefix().split("@")[0];
+  }
+}
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/PYScriptParser.java
similarity index 56%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/PYScriptParser.java
index babeb1f6c..3d5edcfac 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/PYScriptParser.java
@@ -15,16 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.script.parser;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+public class PYScriptParser extends CommonScriptParser {
+
+  private static final PYScriptParser pYScriptParser = new PYScriptParser();
+
+  public static CommonScriptParser apply() {
+    return pYScriptParser;
+  }
+
+  @Override
+  public String prefix() {
+    return "#@set";
+  }
+
+  @Override
+  public boolean belongTo(String suffix) {
+    return CodeAndRunTypeUtils.getSuffixBelongToLanguageTypeOrNot(
+        suffix, CodeAndRunTypeUtils.LANGUAGE_TYPE_PYTHON());
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public String prefixConf() {
+    return "#conf@set";
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultDeserializer.scala b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/QLScriptParser.java
similarity index 56%
rename from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultDeserializer.scala
rename to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/QLScriptParser.java
index 4e8199da0..806a84760 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultDeserializer.scala
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/QLScriptParser.java
@@ -15,22 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.storage.resultset.io
+package org.apache.linkis.storage.script.parser;
 
-import org.apache.linkis.common.io.resultset.ResultDeserializer
-import org.apache.linkis.storage.domain.Dolphin
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils;
 
-import org.apache.commons.codec.binary.Base64
+public class QLScriptParser extends CommonScriptParser {
 
-class IOResultDeserializer extends ResultDeserializer[IOMetaData, IORecord] {
+  private static final QLScriptParser qLScriptParser = new QLScriptParser();
 
-  override def createMetaData(bytes: Array[Byte]): IOMetaData = {
-    val values = Dolphin.getString(bytes, 0, bytes.length).split(Dolphin.COL_SPLIT)
-    new IOMetaData(values(0).toInt, values(1).toInt)
+  public static CommonScriptParser apply() {
+    return qLScriptParser;
   }
 
-  override def createRecord(bytes: Array[Byte]): IORecord = {
-    new IORecord(Base64.decodeBase64(Dolphin.getString(bytes, 0, bytes.length)))
+  @Override
+  public String prefix() {
+    return "--@set";
   }
 
+  @Override
+  public boolean belongTo(String suffix) {
+    return CodeAndRunTypeUtils.getSuffixBelongToLanguageTypeOrNot(
+        suffix, CodeAndRunTypeUtils.LANGUAGE_TYPE_SQL());
+  }
+
+  @Override
+  public String prefixConf() {
+    return "--conf@set";
+  }
 }
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/ScalaScriptParser.java
similarity index 56%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/ScalaScriptParser.java
index babeb1f6c..982538d8e 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/ScalaScriptParser.java
@@ -15,16 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.script.parser;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+public class ScalaScriptParser extends CommonScriptParser {
+
+  private static final ScalaScriptParser scalaScriptParser = new ScalaScriptParser();
+
+  public static CommonScriptParser apply() {
+    return scalaScriptParser;
+  }
+
+  @Override
+  public String prefix() {
+    return "//@set";
+  }
+
+  @Override
+  public boolean belongTo(String suffix) {
+    return CodeAndRunTypeUtils.getSuffixBelongToLanguageTypeOrNot(
+        suffix, CodeAndRunTypeUtils.LANGUAGE_TYPE_SCALA());
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public String prefixConf() {
+    return "//conf@set";
   }
 }
diff --git a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/ShellScriptParser.java
similarity index 56%
copy from linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
copy to linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/ShellScriptParser.java
index babeb1f6c..e1eebe3e5 100644
--- a/linkis-engineconn-plugins/openlookeng/src/main/java/org/apache/linkis/engineplugin/openlookeng/builder/OpenLooKengProcessECLaunchBuilder.java
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/parser/ShellScriptParser.java
@@ -15,16 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.linkis.engineplugin.openlookeng.builder;
+package org.apache.linkis.storage.script.parser;
 
-import org.apache.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils;
 
-public class OpenLooKengProcessECLaunchBuilder extends JavaProcessEngineConnLaunchBuilder {
+public class ShellScriptParser extends CommonScriptParser {
+
+  private static final ShellScriptParser shellScriptParser = new ShellScriptParser();
+
+  public static CommonScriptParser create() {
+    return shellScriptParser;
+  }
+
+  @Override
+  public String prefix() {
+    return "#@set";
+  }
+
+  @Override
+  public boolean belongTo(String suffix) {
+    return CodeAndRunTypeUtils.getSuffixBelongToLanguageTypeOrNot(
+        suffix, CodeAndRunTypeUtils.LANGUAGE_TYPE_SHELL());
+  }
 
   @Override
-  public String getEngineStartUser(UserCreatorLabel label) {
-    return StorageConfiguration.HDFS_ROOT_USER().getValue();
+  public String prefixConf() {
+    return "#conf@set";
   }
 }
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/reader/StorageScriptFsReader.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/reader/StorageScriptFsReader.java
new file mode 100644
index 000000000..1f862fa6b
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/reader/StorageScriptFsReader.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.script.reader;
+
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.storage.script.*;
+import org.apache.linkis.storage.utils.StorageUtils;
+
+import org.apache.commons.io.IOUtils;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.*;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class StorageScriptFsReader extends ScriptFsReader {
+  private final FsPath path;
+  private final String charset;
+  private final InputStream inputStream;
+
+  private InputStreamReader inputStreamReader;
+  private BufferedReader bufferedReader;
+
+  private ScriptMetaData metadata;
+
+  private List<Variable> variables = new ArrayList<>();
+  private String lineText;
+
+  public StorageScriptFsReader(FsPath path, String charset, InputStream inputStream) {
+    super(path, charset);
+    this.path = path;
+    this.charset = charset;
+    this.inputStream = inputStream;
+  }
+
+  @Override
+  public Record getRecord() throws IOException {
+    if (metadata == null) throw new IOException("Must read metadata first(必须先读取metadata)");
+    ScriptRecord record = new ScriptRecord(lineText);
+    lineText = bufferedReader.readLine();
+    return record;
+  }
+
+  @Override
+  public MetaData getMetaData() throws IOException {
+    if (metadata == null) init();
+    Parser parser = getScriptParser();
+    lineText = bufferedReader.readLine();
+    while (hasNext()
+        && Objects.nonNull(parser)
+        && isMetadata(lineText, parser.prefix(), parser.prefixConf())) {
+      variables.add(parser.parse(lineText));
+      lineText = bufferedReader.readLine();
+    }
+    metadata = new ScriptMetaData(variables.toArray(new Variable[0]));
+    return metadata;
+  }
+
+  public void init() {
+    inputStreamReader = new InputStreamReader(inputStream);
+    bufferedReader = new BufferedReader(inputStreamReader);
+  }
+
+  @Override
+  public int skip(int recordNum) throws IOException {
+    if (recordNum < 0) return -1;
+    if (metadata == null) getMetaData();
+    try {
+      return (int) bufferedReader.skip(recordNum);
+    } catch (Throwable t) {
+      return recordNum;
+    }
+  }
+
+  @Override
+  public long getPosition() throws IOException {
+    return -1L;
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    return lineText != null;
+  }
+
+  @Override
+  public long available() throws IOException {
+    return inputStream != null ? inputStream.available() : 0L;
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.closeQuietly(bufferedReader);
+    IOUtils.closeQuietly(inputStreamReader);
+    IOUtils.closeQuietly(inputStream);
+  }
+
+  /**
+   * Determine if the read line is metadata(判断读的行是否是metadata)
+   *
+   * @param line
+   * @return
+   */
+  public boolean isMetadata(String line, String prefix, String prefixConf) {
+    String regex = "\\s*" + prefix + "\\s*(.+)\\s*=\\s*(.+)\\s*";
+    if (line.matches(regex)) {
+      return true;
+    } else {
+      String[] split = line.split("=");
+      if (split.length != 2) {
+        return false;
+      }
+      if (Stream.of(split[0].split(" ")).filter(str -> !"".equals(str)).count() != 4) {
+        return false;
+      }
+
+      Optional<String> optional =
+          Stream.of(split[0].split(" ")).filter(str -> !"".equals(str)).findFirst();
+      if (optional.isPresent() && !optional.get().equals(prefixConf)) {
+        return false;
+      }
+      return true;
+    }
+  }
+
+  /**
+   * get the script parser according to the path(根据文件路径 获取对应的script parser )
+   *
+   * @return Scripts Parser
+   */
+  public Parser getScriptParser() {
+    List<Parser> parsers =
+        Arrays.stream(ParserFactory.listParsers())
+            .filter(p -> p.belongTo(StorageUtils.pathToSuffix(path.getPath())))
+            .collect(Collectors.toList());
+    if (parsers.size() > 0) {
+      return parsers.get(0);
+    } else {
+      return null;
+    }
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/writer/StorageScriptFsWriter.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/writer/StorageScriptFsWriter.java
new file mode 100644
index 000000000..84dd6abb8
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/script/writer/StorageScriptFsWriter.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.script.writer;
+
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.storage.LineRecord;
+import org.apache.linkis.storage.script.Compaction;
+import org.apache.linkis.storage.script.ScriptFsWriter;
+import org.apache.linkis.storage.script.ScriptMetaData;
+import org.apache.linkis.storage.script.Variable;
+import org.apache.linkis.storage.utils.StorageConfiguration;
+import org.apache.linkis.storage.utils.StorageUtils;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+
+import java.io.*;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StorageScriptFsWriter extends ScriptFsWriter {
+  private static final Logger log = LoggerFactory.getLogger(StorageScriptFsWriter.class);
+
+  private final FsPath path;
+  private final String charset;
+  private final OutputStream outputStream;
+  private final StringBuilder stringBuilder = new StringBuilder();
+
+  public StorageScriptFsWriter(FsPath path, String charset, OutputStream outputStream) {
+    this.path = path;
+    this.charset = charset;
+    this.outputStream = outputStream;
+  }
+
+  @Override
+  public void addMetaData(MetaData metaData) throws IOException {
+    String suffix = StorageUtils.pathToSuffix(path.getPath());
+    List<Compaction> compactions =
+        Stream.of(Compaction.listCompactions())
+            .filter(compaction -> compaction.belongTo(suffix))
+            .collect(Collectors.toList());
+    List<String> metadataLine = new ArrayList<>();
+    if (!compactions.isEmpty()) {
+      Variable[] metaData1 = ((ScriptMetaData) metaData).getMetaData();
+      Stream.of(metaData1).map(compactions.get(0)::compact).forEach(metadataLine::add);
+
+      // add annotition symbol
+      if (metadataLine.size() > 0) {
+        metadataLine.add(compactions.get(0).getAnnotationSymbol());
+      }
+      if (outputStream != null) {
+        IOUtils.writeLines(metadataLine, "\n", outputStream, charset);
+      } else {
+        metadataLine.forEach(m -> stringBuilder.append(m).append("\n"));
+      }
+    }
+  }
+
+  @Override
+  public void addRecord(Record record) throws IOException {
+    LineRecord scriptRecord = (LineRecord) record;
+    if (outputStream != null) {
+      IOUtils.write(scriptRecord.getLine(), outputStream, charset);
+    } else {
+      stringBuilder.append(scriptRecord.getLine());
+    }
+  }
+
+  @Override
+  public void close() {
+    IOUtils.closeQuietly(outputStream);
+  }
+
+  @Override
+  public void flush() {
+    if (outputStream instanceof HdfsDataOutputStream) {
+      try {
+        ((HdfsDataOutputStream) outputStream).hflush();
+      } catch (IOException t) {
+        log.warn("Error encountered when flush script", t);
+      }
+    } else if (outputStream != null) {
+      try {
+        outputStream.flush();
+      } catch (IOException t) {
+        log.warn("Error encountered when flush script", t);
+      }
+    }
+  }
+
+  @Override
+  public InputStream getInputStream() {
+    byte[] bytes = null;
+    try {
+      bytes =
+          stringBuilder.toString().getBytes(StorageConfiguration.STORAGE_RS_FILE_TYPE.getValue());
+    } catch (UnsupportedEncodingException e) {
+      log.warn("StorageScriptFsWriter getInputStream failed", e);
+    }
+    return new ByteArrayInputStream(bytes);
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/AbstractFileSource.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/AbstractFileSource.java
new file mode 100644
index 000000000..b3421f840
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/AbstractFileSource.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.source;
+
+import org.apache.linkis.common.io.FsWriter;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.math3.util.Pair;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public abstract class AbstractFileSource implements FileSource {
+
+  private FileSplit[] fileSplits;
+
+  public AbstractFileSource(FileSplit[] fileSplits) {
+    this.fileSplits = fileSplits;
+  }
+
+  @Override
+  public FileSource shuffle(Function<Record, Record> function) {
+    Arrays.stream(fileSplits).forEach(fileSplit -> fileSplit.shuffler = function);
+    return this;
+  }
+
+  @Override
+  public FileSource page(int page, int pageSize) {
+    Arrays.stream(fileSplits).forEach(fileSplit -> fileSplit.page(page, pageSize));
+    return this;
+  }
+
+  @Override
+  public FileSource addParams(Map<String, String> params) {
+    Arrays.stream(fileSplits).forEach(fileSplit -> fileSplit.addParams(params));
+    return this;
+  }
+
+  @Override
+  public FileSource addParams(String key, String value) {
+    Arrays.stream(fileSplits).forEach(fileSplit -> fileSplit.addParams(key, value));
+    return this;
+  }
+
+  @Override
+  public FileSplit[] getFileSplits() {
+    return this.fileSplits;
+  }
+
+  @Override
+  public Map<String, String> getParams() {
+    return Arrays.stream(fileSplits)
+        .map(FileSplit::getParams)
+        .flatMap(map -> map.entrySet().stream())
+        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+  }
+
+  @Override
+  public <K extends MetaData, V extends Record> void write(FsWriter<K, V> fsWriter) {
+    Arrays.stream(fileSplits).forEach(fileSplit -> fileSplit.write(fsWriter));
+  }
+
+  @Override
+  public void close() {
+    Arrays.stream(fileSplits).forEach(IOUtils::closeQuietly);
+  }
+
+  @Override
+  public Pair<Object, List<String[]>>[] collect() {
+    return Arrays.stream(fileSplits).map(FileSplit::collect).toArray(Pair[]::new);
+  }
+
+  @Override
+  public int getTotalLine() {
+    return Arrays.stream(fileSplits).mapToInt(FileSplit::getTotalLine).sum();
+  }
+
+  @Override
+  public String[] getTypes() {
+    return Arrays.stream(fileSplits).map(FileSplit::getType).toArray(String[]::new);
+  }
+
+  @Override
+  public Pair<Integer, Integer>[] getFileInfo(int needToCountRowNumber) {
+    return Arrays.stream(fileSplits)
+        .map(fileSplit -> fileSplit.getFileInfo(needToCountRowNumber))
+        .toArray(Pair[]::new);
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/FileSource.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/FileSource.java
new file mode 100644
index 000000000..b7bcc8c84
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/FileSource.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.source;
+
+import org.apache.linkis.common.io.*;
+import org.apache.linkis.common.io.resultset.ResultSet;
+import org.apache.linkis.common.io.resultset.ResultSetReader;
+import org.apache.linkis.storage.conf.LinkisStorageConf;
+import org.apache.linkis.storage.exception.StorageWarnException;
+import org.apache.linkis.storage.resultset.ResultSetFactory;
+import org.apache.linkis.storage.resultset.ResultSetReaderFactory;
+import org.apache.linkis.storage.script.ScriptFsReader;
+import org.apache.linkis.storage.utils.StorageConfiguration;
+
+import org.apache.commons.math3.util.Pair;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE;
+
+public interface FileSource extends Closeable {
+  Logger logger = LoggerFactory.getLogger(FileSource.class);
+
+  FileSource shuffle(Function<Record, Record> s);
+
+  FileSource page(int page, int pageSize);
+
+  Pair<Object, List<String[]>>[] collect();
+
+  Pair<Integer, Integer>[] getFileInfo(int needToCountRowNumber);
+
+  <K extends MetaData, V extends Record> void write(FsWriter<K, V> fsWriter);
+
+  FileSource addParams(Map<String, String> params);
+
+  FileSource addParams(String key, String value);
+
+  Map<String, String> getParams();
+
+  int getTotalLine();
+
+  String[] getTypes();
+
+  FileSplit[] getFileSplits();
+
+  String[] fileType = LinkisStorageConf.getFileTypeArr();
+  BiFunction<String, String, Boolean> suffixPredicate =
+      (path, suffix) -> path.endsWith("." + suffix);
+
+  static boolean isResultSet(String path) {
+    return suffixPredicate.apply(path, fileType[0]);
+  }
+
+  static boolean isResultSet(FsPath fsPath) {
+    return isResultSet(fsPath.getPath());
+  }
+
+  /**
+   * Currently only supports table multi-result sets
+   *
+   * @param fsPaths
+   * @param fs
+   * @return
+   */
+  static FileSource create(FsPath[] fsPaths, Fs fs) {
+    // Filter non-table result sets
+    FileSplit[] fileSplits =
+        Arrays.stream(fsPaths)
+            .map(fsPath -> createResultSetFileSplit(fsPath, fs))
+            .filter(FileSource::isTableResultSet)
+            .toArray(FileSplit[]::new);
+    return new ResultsetFileSource(fileSplits);
+  }
+
+  static boolean isTableResultSet(FileSplit fileSplit) {
+    return fileSplit.type.equals(ResultSetFactory.TABLE_TYPE);
+  }
+
+  static boolean isTableResultSet(FileSource fileSource) {
+    // Return true only if all splits are table result sets
+    return Arrays.stream(fileSource.getFileSplits()).allMatch(FileSource::isTableResultSet);
+  }
+
+  static FileSource create(FsPath fsPath, Fs fs) {
+    if (!canRead(fsPath.getPath())) {
+      throw new StorageWarnException(
+          UNSUPPORTED_OPEN_FILE_TYPE.getErrorCode(), UNSUPPORTED_OPEN_FILE_TYPE.getErrorDesc());
+    }
+    if (isResultSet(fsPath)) {
+      return new ResultsetFileSource(new FileSplit[] {createResultSetFileSplit(fsPath, fs)});
+    } else {
+      return new TextFileSource(new FileSplit[] {createTextFileSplit(fsPath, fs)});
+    }
+  }
+
+  static FileSource create(FsPath fsPath, InputStream is) {
+    if (!canRead(fsPath.getPath())) {
+      throw new StorageWarnException(
+          UNSUPPORTED_OPEN_FILE_TYPE.getErrorCode(), UNSUPPORTED_OPEN_FILE_TYPE.getErrorDesc());
+    }
+    if (isResultSet(fsPath)) {
+      return new ResultsetFileSource(new FileSplit[] {createResultSetFileSplit(fsPath, is)});
+    } else {
+      return new TextFileSource(new FileSplit[] {createTextFileSplit(fsPath, is)});
+    }
+  }
+
+  static FileSplit createResultSetFileSplit(FsPath fsPath, InputStream is) {
+    ResultSet resultset = ResultSetFactory.getInstance().getResultSetByPath(fsPath);
+    ResultSetReader resultsetReader = ResultSetReaderFactory.getResultSetReader(resultset, is);
+    return new FileSplit(resultsetReader, resultset.resultSetType());
+  }
+
+  static FileSplit createResultSetFileSplit(FsPath fsPath, Fs fs) {
+    ResultSet resultset = ResultSetFactory.getInstance().getResultSetByPath(fsPath, fs);
+    ResultSetReader resultsetReader = null;
+    try {
+      resultsetReader = ResultSetReaderFactory.getResultSetReader(resultset, fs.read(fsPath));
+    } catch (IOException e) {
+      logger.warn("FileSource createResultSetFileSplit failed", e);
+    }
+    return new FileSplit(resultsetReader, resultset.resultSetType());
+  }
+
+  static FileSplit createTextFileSplit(FsPath fsPath, InputStream is) {
+    ScriptFsReader scriptFsReader =
+        ScriptFsReader.getScriptFsReader(
+            fsPath, StorageConfiguration.STORAGE_RS_FILE_TYPE.getValue(), is);
+    return new FileSplit(scriptFsReader);
+  }
+
+  static FileSplit createTextFileSplit(FsPath fsPath, Fs fs) {
+    ScriptFsReader scriptFsReader = null;
+    try {
+      scriptFsReader =
+          ScriptFsReader.getScriptFsReader(
+              fsPath, StorageConfiguration.STORAGE_RS_FILE_TYPE.getValue(), fs.read(fsPath));
+    } catch (IOException e) {
+      logger.warn("FileSource createTextFileSplit failed", e);
+    }
+    return new FileSplit(scriptFsReader);
+  }
+
+  static boolean canRead(String path) {
+    return Arrays.stream(fileType).anyMatch(suffix -> path.endsWith("." + suffix));
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/FileSplit.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/FileSplit.java
new file mode 100644
index 000000000..a43b7feb0
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/FileSplit.java
@@ -0,0 +1,281 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.source;
+
+import org.apache.linkis.common.io.FsReader;
+import org.apache.linkis.common.io.FsWriter;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.storage.LineMetaData;
+import org.apache.linkis.storage.LineRecord;
+import org.apache.linkis.storage.domain.Column;
+import org.apache.linkis.storage.domain.DataType;
+import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary;
+import org.apache.linkis.storage.exception.StorageWarnException;
+import org.apache.linkis.storage.resultset.table.TableMetaData;
+import org.apache.linkis.storage.resultset.table.TableRecord;
+import org.apache.linkis.storage.script.Parser;
+import org.apache.linkis.storage.script.ScriptMetaData;
+import org.apache.linkis.storage.script.Variable;
+import org.apache.linkis.storage.script.VariableParser;
+import org.apache.linkis.storage.script.reader.StorageScriptFsReader;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.math3.util.Pair;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.*;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileSplit implements Closeable {
+  private static final Logger logger = LoggerFactory.getLogger(FileSplit.class);
+
+  private FsReader<? extends MetaData, ? extends Record> fsReader;
+  protected String type = "script/text";
+  private int start = 0;
+  private int end = -1;
+  private int count = 0;
+  private int totalLine = 0;
+  protected Function<Record, Record> shuffler;
+  private boolean pageTrigger = false;
+  protected Map<String, String> params = new HashMap<>();
+
+  public FileSplit(FsReader<? extends MetaData, ? extends Record> fsReader) {
+    this.fsReader = fsReader;
+  }
+
+  public FileSplit(FsReader<? extends MetaData, ? extends Record> fsReader, String type) {
+    this.fsReader = fsReader;
+    this.type = type;
+  }
+
+  public void page(int page, int pageSize) {
+    if (!pageTrigger) {
+      start = (page - 1) * pageSize;
+      end = pageSize * page - 1;
+      pageTrigger = true;
+    }
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void addParams(Map<String, String> params) {
+    this.params.putAll(params);
+  }
+
+  public void addParams(String key, String value) {
+    this.params.put(key, value);
+  }
+
+  public Map<String, String> getParams() {
+    return params;
+  }
+
+  public int getTotalLine() {
+    return totalLine;
+  }
+
+  public <M> M whileLoop(Function<MetaData, M> metaDataFunction, Consumer<Record> recordConsumer) {
+    M m = null;
+    try {
+      MetaData metaData = fsReader.getMetaData();
+      m = metaDataFunction.apply(metaData);
+      if (pageTrigger) {
+        fsReader.skip(start);
+      }
+      count = start;
+      boolean hasRemovedFlag = false;
+      while (fsReader.hasNext() && ifContinueRead()) {
+        Record record = fsReader.getRecord();
+        boolean needRemoveFlag = false;
+        if (!hasRemovedFlag && fsReader instanceof StorageScriptFsReader) {
+          Parser parser = ((StorageScriptFsReader) fsReader).getScriptParser();
+          Variable[] meta = ((ScriptMetaData) metaData).getMetaData();
+          if (meta != null
+              && meta.length > 0
+              && parser != null
+              && parser.getAnnotationSymbol().equals(record.toString())) {
+            needRemoveFlag = true;
+            hasRemovedFlag = true;
+          }
+        }
+        if (!needRemoveFlag) {
+          recordConsumer.accept(shuffler.apply(record));
+          totalLine++;
+          count++;
+        }
+      }
+    } catch (IOException e) {
+      logger.warn("FileSplit forEach failed", e);
+      throw new StorageWarnException(
+          LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE.getErrorCode(),
+          LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE.getErrorMessage());
+    }
+    return m;
+  }
+
+  public void biConsumerWhileLoop(
+      Consumer<MetaData> metaDataFunction, Consumer<Record> recordConsumer) {
+    try {
+      MetaData metaData = fsReader.getMetaData();
+      metaDataFunction.accept(metaData);
+      if (pageTrigger) {
+        fsReader.skip(start);
+      }
+      count = start;
+      boolean hasRemovedFlag = false;
+      while (fsReader.hasNext() && ifContinueRead()) {
+        Record record = fsReader.getRecord();
+        boolean needRemoveFlag = false;
+        if (!hasRemovedFlag && fsReader instanceof StorageScriptFsReader) {
+          Parser parser = ((StorageScriptFsReader) fsReader).getScriptParser();
+          Variable[] meta = ((ScriptMetaData) metaData).getMetaData();
+          if (meta != null
+              && meta.length > 0
+              && parser != null
+              && parser.getAnnotationSymbol().equals(record.toString())) {
+            needRemoveFlag = true;
+            hasRemovedFlag = true;
+          }
+        }
+        if (!needRemoveFlag) {
+          recordConsumer.accept(shuffler.apply(record));
+          totalLine++;
+          count++;
+        }
+      }
+    } catch (IOException e) {
+      logger.warn("FileSplit forEach failed", e);
+      throw new StorageWarnException(
+          LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE.getErrorCode(),
+          LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE.getErrorMessage());
+    }
+  }
+
+  public Pair<Integer, Integer> getFileInfo(int needToCountRowNumber) {
+    int colNumber = 0;
+    int rowNumber = 0;
+    MetaData metaData = null;
+    try {
+      metaData = fsReader.getMetaData();
+      colNumber =
+          metaData instanceof TableMetaData ? ((TableMetaData) metaData).getColumns().length : 1;
+      rowNumber =
+          needToCountRowNumber == -1
+              ? fsReader.skip(Integer.MAX_VALUE)
+              : fsReader.skip(needToCountRowNumber);
+    } catch (IOException e) {
+      logger.warn("FileSplit getFileInfo failed", e);
+      throw new StorageWarnException(
+          LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE.getErrorCode(),
+          LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE.getErrorMessage());
+    }
+    return new Pair<>(colNumber, rowNumber);
+  }
+
+  public <K extends MetaData, V extends Record> void write(FsWriter<K, V> fsWriter) {
+    biConsumerWhileLoop(
+        metaData -> {
+          try {
+            fsWriter.addMetaData(metaData);
+          } catch (IOException e) {
+            logger.warn("FileSplit addMetaData failed", e);
+            throw new StorageWarnException(
+                LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE.getErrorCode(),
+                LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE.getErrorMessage());
+          }
+        },
+        record -> {
+          try {
+            fsWriter.addRecord(record);
+          } catch (IOException e) {
+            logger.warn("FileSplit addRecord failed", e);
+            throw new StorageWarnException(
+                LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE.getErrorCode(),
+                LinkisStorageErrorCodeSummary.UNSUPPORTED_OPEN_FILE_TYPE.getErrorMessage());
+          }
+        });
+  }
+
+  public Pair<Object, List<String[]>> collect() {
+    List<String[]> recordList = new ArrayList<>();
+    Object metaData =
+        whileLoop(
+            collectMetaData -> collectMetaData(collectMetaData),
+            r -> recordList.add(collectRecord(r)));
+    return new Pair<>(metaData, recordList);
+  }
+
+  public String[] collectRecord(Record record) {
+    if (record instanceof TableRecord) {
+      TableRecord tableRecord = (TableRecord) record;
+      return Arrays.stream(tableRecord.row).map(DataType::valueToString).toArray(String[]::new);
+    } else if (record instanceof LineRecord) {
+      LineRecord lineRecord = (LineRecord) record;
+      return new String[] {lineRecord.getLine()};
+    } else {
+      throw new IllegalArgumentException("Unknown record type");
+    }
+  }
+
+  public Object collectMetaData(MetaData metaData) {
+    if (metaData instanceof ScriptMetaData) {
+      ScriptMetaData scriptMetaData = (ScriptMetaData) metaData;
+      return VariableParser.getMap(scriptMetaData.getMetaData());
+    } else if (metaData instanceof LineMetaData) {
+      LineMetaData lineMetaData = (LineMetaData) metaData;
+      return lineMetaData.getMetaData();
+    } else if (metaData instanceof TableMetaData) {
+      TableMetaData tableMetaData = (TableMetaData) metaData;
+      return Arrays.stream(tableMetaData.getColumns())
+          .map(this::columnToMap)
+          .collect(Collectors.toList());
+    } else {
+      throw new IllegalArgumentException("Unknown metadata type");
+    }
+  }
+
+  private Map<String, String> columnToMap(Column column) {
+    Map<String, String> stringMap = new HashMap<>();
+    stringMap.put("columnName", column.getColumnName());
+    stringMap.put("comment", column.getComment());
+    stringMap.put("dataType", column.getDataType().getTypeName());
+    return stringMap;
+  }
+
+  public boolean ifContinueRead() {
+    return !pageTrigger || count <= end;
+  }
+
+  public boolean ifStartRead() {
+    return !pageTrigger || count >= start;
+  }
+
+  @Override
+  public void close() {
+    IOUtils.closeQuietly(fsReader);
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/ResultsetFileSource.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/ResultsetFileSource.java
new file mode 100644
index 000000000..d8562b7c5
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/ResultsetFileSource.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.source;
+
+import org.apache.linkis.storage.resultset.table.TableRecord;
+import org.apache.linkis.storage.utils.StorageUtils;
+
+import java.util.Arrays;
+
+public class ResultsetFileSource extends AbstractFileSource {
+
+  public ResultsetFileSource(FileSplit[] fileSplits) {
+    super(fileSplits);
+    shuffle(
+        record -> {
+          if (record instanceof TableRecord) {
+            TableRecord tableRecord = (TableRecord) record;
+            String nullValue = getParams().getOrDefault("nullValue", "NULL");
+            return new TableRecord(
+                Arrays.stream(tableRecord.row)
+                    .map(
+                        r -> {
+                          if (r == null || r.equals("NULL")) {
+                            return nullValue;
+                          } else if (r.equals("")) {
+                            return getParams().getOrDefault("nullValue", "");
+                          } else if (r instanceof Double) {
+                            return StorageUtils.doubleToString((Double) r);
+                          } else {
+                            return r;
+                          }
+                        })
+                    .toArray());
+          } else {
+            return record;
+          }
+        });
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/TextFileSource.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/TextFileSource.java
new file mode 100644
index 000000000..7e5396bf7
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/TextFileSource.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.source;
+
+import org.apache.linkis.storage.LineRecord;
+import org.apache.linkis.storage.script.ScriptRecord;
+
+import org.apache.commons.math3.util.Pair;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+class TextFileSource extends AbstractFileSource {
+  public static final String[] LINE_BREAKER = new String[] {"\n"};
+
+  public TextFileSource(FileSplit[] fileSplits) {
+    super(fileSplits);
+    shuffle(
+        record -> {
+          if (record instanceof ScriptRecord && "".equals(((ScriptRecord) record).getLine())) {
+            return new LineRecord("\n");
+          } else {
+            return record;
+          }
+        });
+  }
+
+  @Override
+  public Pair<Object, List<String[]>>[] collect() {
+    Pair<Object, List<String[]>>[] collects = super.collect();
+    if (!getParams().getOrDefault("ifMerge", "true").equals("true")) {
+      return collects;
+    }
+    ArrayList<List<String[]>> snds =
+        Arrays.stream(collects)
+            .map(Pair::getSecond)
+            .collect(Collectors.toCollection(ArrayList::new));
+    snds.forEach(
+        snd -> {
+          StringBuilder str = new StringBuilder();
+          snd.forEach(
+              arr -> {
+                if (Arrays.equals(arr, LINE_BREAKER)) {
+                  str.append("\n");
+                } else {
+                  str.append(arr[0]).append("\n");
+                }
+              });
+          snd.clear();
+          snd.add(new String[] {str.toString()});
+        });
+    return collects;
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/FileSystemUtils.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/FileSystemUtils.java
new file mode 100644
index 000000000..0f93cdb6a
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/FileSystemUtils.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.utils;
+
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.storage.FSFactory;
+import org.apache.linkis.storage.fs.FileSystem;
+import org.apache.linkis.storage.fs.impl.LocalFileSystem;
+
+import org.apache.commons.io.IOUtils;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.Stack;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileSystemUtils {
+  private static final Logger logger = LoggerFactory.getLogger(FileSystemUtils.class);
+
+  public static void copyFile(FsPath filePath, FsPath origin, String user) throws Exception {
+    FileSystem fileSystem = (FileSystem) FSFactory.getFsByProxyUser(filePath, user);
+    try {
+      fileSystem.init(null);
+      if (!fileSystem.exists(filePath)) {
+        if (!fileSystem.exists(filePath.getParent())) {
+          fileSystem.mkdirs(filePath.getParent());
+        }
+        fileSystem.createNewFile(filePath);
+      }
+      fileSystem.copyFile(origin, filePath);
+    } finally {
+      IOUtils.closeQuietly(fileSystem);
+    }
+  }
+
+  /**
+   * Create a new file
+   *
+   * @param filePath
+   * @param createParentWhenNotExists Whether to recursively create a directory
+   */
+  public static void createNewFile(FsPath filePath, boolean createParentWhenNotExists)
+      throws Exception {
+    createNewFile(filePath, StorageUtils.getJvmUser(), createParentWhenNotExists);
+  }
+
+  public static void createNewFile(
+      FsPath filePath, String user, boolean createParentWhenNotExists) {
+    FileSystem fileSystem = (FileSystem) FSFactory.getFsByProxyUser(filePath, user);
+    try {
+      fileSystem.init(null);
+      createNewFileWithFileSystem(fileSystem, filePath, user, createParentWhenNotExists);
+    } catch (IOException e) {
+      logger.warn("FileSystemUtils createNewFile failed", e);
+    } catch (Exception e) {
+      logger.warn("FileSystemUtils createNewFile failed", e);
+    } finally {
+      IOUtils.closeQuietly(fileSystem);
+    }
+  }
+
+  public static void createNewFileWithFileSystem(
+      FileSystem fileSystem, FsPath filePath, String user, boolean createParentWhenNotExists)
+      throws Exception {
+    if (!fileSystem.exists(filePath)) {
+      if (!fileSystem.exists(filePath.getParent())) {
+        if (!createParentWhenNotExists) {
+          throw new IOException(
+              "parent dir " + filePath.getParent().getPath() + " dose not exists.");
+        }
+        mkdirs(fileSystem, filePath.getParent(), user);
+      }
+      fileSystem.createNewFile(filePath);
+      if (fileSystem instanceof LocalFileSystem) {
+        fileSystem.setOwner(filePath, user);
+      } else {
+        logger.info("doesn't need to call setOwner");
+      }
+    }
+  }
+
+  /**
+   * Recursively create a directory
+   *
+   * @param fileSystem
+   * @param dest
+   * @param user
+   * @throws IOException
+   * @return
+   */
+  public static boolean mkdirs(FileSystem fileSystem, FsPath dest, String user) throws IOException {
+    FsPath parentPath = dest.getParent();
+    Stack<FsPath> dirsToMake = new Stack<>();
+    dirsToMake.push(dest);
+    while (!fileSystem.exists(parentPath)) {
+      dirsToMake.push(parentPath);
+
+      if (Objects.isNull(parentPath.getParent())) {
+        // parent path of root is null
+        break;
+      }
+
+      parentPath = parentPath.getParent();
+    }
+    if (!fileSystem.canExecute(parentPath)) {
+      throw new IOException("You have not permission to access path " + dest.getPath());
+    }
+    while (!dirsToMake.empty()) {
+      FsPath path = dirsToMake.pop();
+      fileSystem.mkdir(path);
+      if (fileSystem instanceof LocalFileSystem) {
+        fileSystem.setOwner(path, user);
+      } else {
+        logger.info("doesn't need to call setOwner");
+      }
+    }
+    return true;
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageConfiguration.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageConfiguration.java
new file mode 100644
index 000000000..03ec71797
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageConfiguration.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.utils;
+
+import org.apache.linkis.common.conf.ByteType;
+import org.apache.linkis.common.conf.CommonVars;
+
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+public class StorageConfiguration {
+
+  public static CommonVars<String> PROXY_USER =
+      new CommonVars<>("wds.linkis.storage.proxy.user", "${UM}", null, null);
+
+  public static CommonVars<String> STORAGE_ROOT_USER =
+      new CommonVars<>("wds.linkis.storage.root.user", "hadoop", null, null);
+
+  public static CommonVars<String> HDFS_ROOT_USER =
+      new CommonVars<>("wds.linkis.storage.hdfs.root.user", "hadoop", null, null);
+
+  public static CommonVars<String> LOCAL_ROOT_USER =
+      new CommonVars<>("wds.linkis.storage.local.root.user", "root", null, null);
+
+  public static CommonVars<String> STORAGE_USER_GROUP =
+      new CommonVars<>("wds.linkis.storage.fileSystem.group", "bdap", null, null);
+
+  public static CommonVars<String> STORAGE_RS_FILE_TYPE =
+      new CommonVars<>("wds.linkis.storage.rs.file.type", "utf-8", null, null);
+
+  public static CommonVars<String> STORAGE_RS_FILE_SUFFIX =
+      new CommonVars<>("wds.linkis.storage.rs.file.suffix", ".dolphin", null, null);
+
+  public static List<String> ResultTypes =
+      Lists.newArrayList("%TEXT", "%TABLE", "%HTML", "%IMG", "%ANGULAR", "%SVG");
+
+  public static CommonVars<String> STORAGE_RESULT_SET_PACKAGE =
+      new CommonVars<>(
+          "wds.linkis.storage.result.set.package",
+          "org.apache.linkis.storage.resultset",
+          null,
+          null);
+
+  public static CommonVars<String> STORAGE_RESULT_SET_CLASSES =
+      new CommonVars<>(
+          "wds.linkis.storage.result.set.classes",
+          "txt.TextResultSet,table.TableResultSet,io.IOResultSet,html.HtmlResultSet,picture.PictureResultSet",
+          null,
+          null);
+
+  public static CommonVars<String> STORAGE_BUILD_FS_CLASSES =
+      new CommonVars<>(
+          "wds.linkis.storage.build.fs.classes",
+          "org.apache.linkis.storage.factory.impl.BuildHDFSFileSystem,org.apache.linkis.storage.factory.impl.BuildLocalFileSystem,"
+              + "org.apache.linkis.storage.factory.impl.BuildOSSSystem,org.apache.linkis.storage.factory.impl.BuildS3FileSystem",
+          null,
+          null);
+
+  public static CommonVars<Boolean> IS_SHARE_NODE =
+      new CommonVars<>("wds.linkis.storage.is.share.node", true, null, null);
+
+  public static CommonVars<Boolean> ENABLE_IO_PROXY =
+      new CommonVars<>("wds.linkis.storage.enable.io.proxy", false, null, null);
+
+  public static CommonVars<String> IO_USER =
+      new CommonVars<>("wds.linkis.storage.io.user", "root", null, null);
+  public static CommonVars<Integer> IO_FS_EXPIRE_TIME =
+      new CommonVars<>("wds.linkis.storage.io.fs.num", 1000 * 60 * 10, null, null);
+
+  public static CommonVars<ByteType> IO_PROXY_READ_FETCH_SIZE =
+      new CommonVars<>("wds.linkis.storage.io.read.fetch.size", new ByteType("100k"), null, null);
+
+  public static CommonVars<ByteType> IO_PROXY_WRITE_CACHE_SIZE =
+      new CommonVars<>("wds.linkis.storage.io.write.cache.size", new ByteType("64k"), null, null);
+
+  public static CommonVars<String> IO_DEFAULT_CREATOR =
+      new CommonVars<>("wds.linkis.storage.io.default.creator", "IDE", null, null);
+  public static CommonVars<String> IO_FS_RE_INIT =
+      new CommonVars<>("wds.linkis.storage.io.fs.re.init", "re-init", null, null);
+
+  public static CommonVars<Integer> IO_INIT_RETRY_LIMIT =
+      new CommonVars<>("wds.linkis.storage.io.init.retry.limit", 10, null, null);
+
+  public static CommonVars<String> STORAGE_HDFS_GROUP =
+      new CommonVars<>("wds.linkis.storage.fileSystem.hdfs.group", "hadoop", null, null);
+
+  public static CommonVars<Integer> DOUBLE_FRACTION_LEN =
+      new CommonVars<>("wds.linkis.double.fraction.length", 30, null, null);
+
+  public static CommonVars<Boolean> HDFS_PATH_PREFIX_CHECK_ON =
+      new CommonVars<>("wds.linkis.storage.hdfs.prefix_check.enable", true, null, null);
+
+  public static CommonVars<Boolean> HDFS_PATH_PREFIX_REMOVE =
+      new CommonVars<>("wds.linkis.storage.hdfs.prefxi.remove", true, null, null);
+
+  public static CommonVars<Boolean> FS_CACHE_DISABLE =
+      new CommonVars<>("wds.linkis.fs.hdfs.impl.disable.cache", false, null, null);
+
+  public static CommonVars<Boolean> FS_CHECKSUM_DISBALE =
+      new CommonVars<>("linkis.fs.hdfs.impl.disable.checksum", false, null, null);
+
+  /**
+   * more arguments please refer to:
+   * https://hadoop.apache.org/docs/stable/hadoop-aliyun/tools/hadoop-aliyun/index.html Aliyun OSS
+   * endpoint to connect to. eg: https://oss-cn-hangzhou.aliyuncs.com
+   */
+  public static CommonVars<String> OSS_ENDPOINT =
+      new CommonVars<String>("wds.linkis.fs.oss.endpoint", "", null, null);
+
+  /** Aliyun bucket name eg: benchmark2 */
+  public static CommonVars<String> OSS_ACCESS_BUCKET_NAME =
+      new CommonVars<String>("wds.linkis.fs.oss.bucket.name", "", null, null);
+
+  /** Aliyun access key ID */
+  public static CommonVars<String> OSS_ACCESS_KEY_ID =
+      new CommonVars<String>("wds.linkis.fs.oss.accessKeyId", "", null, null);
+
+  /** Aliyun access key secret */
+  public static CommonVars<String> OSS_ACCESS_KEY_SECRET =
+      new CommonVars<String>("wds.linkis.fs.oss.accessKeySecret", "", null, null);
+
+  public static CommonVars<Boolean> OSS_PATH_PREFIX_CHECK_ON =
+      new CommonVars<Boolean>("wds.linkis.storage.oss.prefix_check.enable", false, null, null);
+
+  public static CommonVars<Boolean> OSS_PATH_PREFIX_REMOVE =
+      new CommonVars<Boolean>("wds.linkis.storage.oss.prefix.remove", true, null, null);
+
+  public static CommonVars<String> S3_ACCESS_KEY =
+      new CommonVars("linkis.storage.s3.access.key", "", null, null);
+
+  public static CommonVars<String> S3_SECRET_KEY =
+      new CommonVars("linkis.storage.s3.secret.key", "", null, null);
+
+  public static CommonVars<String> S3_ENDPOINT =
+      new CommonVars("linkis.storage.s3.endpoint", "", null, null);
+
+  public static CommonVars<String> S3_REGION =
+      new CommonVars("linkis.storage.s3.region", "", null, null);
+
+  public static CommonVars<String> S3_BUCKET =
+      new CommonVars("linkis.storage.s3.bucket", "", null, null);
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageHelper.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageHelper.java
new file mode 100644
index 000000000..e1dee151c
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageHelper.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.utils;
+
+import org.apache.linkis.common.io.Fs;
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.common.io.MetaData;
+import org.apache.linkis.common.io.Record;
+import org.apache.linkis.common.io.resultset.ResultSet;
+import org.apache.linkis.common.io.resultset.ResultSetReader;
+import org.apache.linkis.storage.FSFactory;
+import org.apache.linkis.storage.resultset.ResultSetFactory;
+import org.apache.linkis.storage.resultset.ResultSetReaderFactory;
+import org.apache.linkis.storage.resultset.table.TableMetaData;
+import org.apache.linkis.storage.resultset.table.TableRecord;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * 工具类,用于做storage jar包打出来做测试用 Tool class, which is used to print the storage jar package for testing
+ */
+public class StorageHelper {
+  private static final Log logger = LogFactory.getLog(StorageHelper.class);
+
+  public static void main(String[] args) {
+    if (args.length < 2) logger.info("Usage method params eg:getTableResLines path");
+    String method = args[0];
+    String[] params = Arrays.copyOfRange(args, 1, args.length);
+    try {
+      Thread.sleep(10000L);
+    } catch (InterruptedException e) {
+    }
+
+    switch (method) {
+      case "getTableResLines":
+        getTableResLines(params);
+        break;
+      case "getTableRes":
+        getTableRes(params);
+        break;
+      case "createNewFile":
+        createNewFile(params);
+        break;
+      default:
+        logger.info("There is no such method");
+    }
+  }
+
+  /**
+   * Get the number of table result set file lines(获得表格结果集文件行数)
+   *
+   * @param args
+   */
+  public static void getTableResLines(String[] args) {
+    ResultSetReader resultSetReader = null;
+    try {
+      FsPath resPath = StorageUtils.getFsPath(args[0]);
+      ResultSetFactory resultSetFactory = ResultSetFactory.getInstance();
+
+      ResultSet<? extends MetaData, ? extends Record> resultSet =
+          resultSetFactory.getResultSetByType(ResultSetFactory.TABLE_TYPE);
+      Fs fs = FSFactory.getFs(resPath);
+      fs.init(null);
+      resultSetReader = ResultSetReaderFactory.getResultSetReader(resultSet, fs.read(resPath));
+      TableMetaData metaData = (TableMetaData) resultSetReader.getMetaData();
+      Arrays.stream(metaData.getColumns()).forEach(column -> logger.info(column.toString()));
+      int num = 0;
+      Thread.sleep(10000L);
+      while (resultSetReader.hasNext()) {
+        resultSetReader.getRecord();
+        num++;
+      }
+      logger.info(Integer.toString(num));
+    } catch (Exception e) {
+      logger.error("getTableResLines error:", e);
+    } finally {
+      if (resultSetReader != null) {
+        try {
+          resultSetReader.close();
+        } catch (IOException e) {
+          logger.error("Failed to close ResultSetReader", e);
+        }
+      }
+    }
+  }
+
+  public static void getTableRes(String[] args) {
+    try {
+      int len = Integer.parseInt(args[1]);
+      int max = len + 10;
+      FsPath resPath = StorageUtils.getFsPath(args[0]);
+      ResultSetFactory resultSetFactory = ResultSetFactory.getInstance();
+      ResultSet<? extends MetaData, ? extends Record> resultSet =
+          resultSetFactory.getResultSetByType(ResultSetFactory.TABLE_TYPE);
+      Fs fs = FSFactory.getFs(resPath);
+
+      fs.init(null);
+
+      ResultSetReader reader =
+          ResultSetReaderFactory.getResultSetReader(resultSet, fs.read(resPath));
+      MetaData rmetaData = reader.getMetaData();
+      Arrays.stream(((TableMetaData) rmetaData).getColumns())
+          .forEach(column -> logger.info(column.toString()));
+      Arrays.stream(((TableMetaData) rmetaData).getColumns())
+          .map(column -> column.getColumnName() + ",")
+          .forEach(column -> logger.info(column));
+      int num = 0;
+      while (reader.hasNext()) {
+        num++;
+        if (num > max) return;
+        if (num > len) {
+          Record record = reader.getRecord();
+          Arrays.stream(((TableRecord) record).row)
+              .forEach(
+                  value -> {
+                    logger.info(value.toString());
+                    logger.info(",");
+                  });
+          logger.info("\n");
+        }
+      }
+    } catch (IOException e) {
+      logger.warn("StorageHelper getTableRes failed", e);
+    }
+  }
+
+  public static void createNewFile(String[] args) {
+    FsPath resPath = StorageUtils.getFsPath(args[0]);
+    String proxyUser = StorageUtils.getJvmUser();
+    try {
+      FileSystemUtils.createNewFile(resPath, proxyUser, true);
+    } catch (Exception e) {
+      logger.warn("StorageHelper createNewFile failed", e);
+    }
+    logger.info("success");
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageUtils.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageUtils.java
new file mode 100644
index 000000000..ed5099b67
--- /dev/null
+++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageUtils.java
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.linkis.storage.utils;
+
+import org.apache.linkis.common.io.Fs;
+import org.apache.linkis.common.io.FsPath;
+import org.apache.linkis.common.io.resultset.ResultSet;
+import org.apache.linkis.common.io.resultset.ResultSetReader;
+import org.apache.linkis.common.io.resultset.ResultSetWriter;
+import org.apache.linkis.common.utils.Utils;
+import org.apache.linkis.hadoop.common.conf.HadoopConf;
+import org.apache.linkis.storage.LineMetaData;
+import org.apache.linkis.storage.LineRecord;
+import org.apache.linkis.storage.exception.StorageWarnException;
+import org.apache.linkis.storage.resultset.ResultSetFactory;
+import org.apache.linkis.storage.resultset.ResultSetReaderFactory;
+import org.apache.linkis.storage.resultset.ResultSetWriterFactory;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.curator.utils.CloseableUtils;
+
+import java.io.*;
+import java.lang.reflect.Method;
+import java.text.NumberFormat;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.CONFIGURATION_NOT_READ;
+
+public class StorageUtils {
+  private static final Logger logger = LoggerFactory.getLogger(StorageUtils.class);
+
+  public static final String HDFS = "hdfs";
+  public static final String FILE = "file";
+  public static final String OSS = "oss";
+  public static final String S3 = "s3";
+
+  public static final String FILE_SCHEMA = "file://";
+  public static final String HDFS_SCHEMA = "hdfs://";
+  public static final String OSS_SCHEMA = "oss://";
+  public static final String S3_SCHEMA = "s3://";
+
+  private static final NumberFormat nf = NumberFormat.getInstance();
+
+  static {
+    nf.setGroupingUsed(false);
+    nf.setMaximumFractionDigits((int) StorageConfiguration.DOUBLE_FRACTION_LEN.getValue());
+  }
+
+  public static String doubleToString(double value) {
+    return nf.format(value);
+  }
+
+  public static <T> Map<String, T> loadClass(String classStr, Function<T, String> op) {
+    String[] _classes = classStr.split(",");
+    LinkedHashMap<String, T> classes = new LinkedHashMap<>();
+    for (String clazz : _classes) {
+      try {
+        T obj = Utils.getClassInstance(clazz.trim());
+        classes.put(op.apply(obj), obj);
+      } catch (Exception e) {
+        logger.warn("StorageUtils loadClass failed", e);
+      }
+    }
+    return classes;
+  }
+
+  public static <T> Map<String, Class<T>> loadClasses(
+      String classStr, String pge, Function<Class<T>, String> op) {
+    String[] _classes =
+        StringUtils.isEmpty(pge)
+            ? classStr.split(",")
+            : Stream.of(StringUtils.split(classStr, ','))
+                .map(value -> pge + "." + value)
+                .toArray(String[]::new);
+    Map<String, Class<T>> classes = new LinkedHashMap<>();
+    for (String clazz : _classes) {
+      try {
+        Class<T> _class =
+            (Class<T>) Thread.currentThread().getContextClassLoader().loadClass(clazz.trim());
+        classes.put(op.apply(_class), _class);
+      } catch (Exception e) {
+        logger.warn("StorageUtils loadClasses failed", e);
+      }
+    }
+    return classes;
+  }
+
+  public static String pathToSuffix(String path) {
+    String fileName = new File(path).getName();
+    if (fileName.length() > 0) {
+      int dot = fileName.lastIndexOf('.');
+      if (dot > -1 && dot < fileName.length() - 1) {
+        return fileName.substring(dot + 1);
+      }
+    }
+    return fileName;
+  }
+
+  public static Object invoke(Object obj, Method method, Object[] args) throws Exception {
+    return method.invoke(obj, args);
+  }
+
+  /**
+   * Serialized string is a result set of type Text(序列化字符串为Text类型的结果集)
+   *
+   * @param value
+   * @return
+   */
+  public static String serializerStringToResult(String value) throws IOException {
+    ResultSet resultSet =
+        ResultSetFactory.getInstance().getResultSetByType(ResultSetFactory.TEXT_TYPE);
+    ResultSetWriter writer =
+        ResultSetWriterFactory.getResultSetWriter(resultSet, Long.MAX_VALUE, null);
+    LineMetaData metaData = new LineMetaData(null);
+    LineRecord record = new LineRecord(value);
+    writer.addMetaData(metaData);
+    writer.addRecord(record);
+    String res = writer.toString();
+    IOUtils.closeQuietly(writer);
+    return res;
+  }
+
+  /**
+   * The result set of serialized text is a string(序列化text的结果集为字符串)
+   *
+   * @param result
+   * @return
+   */
+  public static String deserializerResultToString(String result) throws IOException {
+    ResultSet resultSet =
+        ResultSetFactory.getInstance().getResultSetByType(ResultSetFactory.TEXT_TYPE);
+    ResultSetReader reader = ResultSetReaderFactory.getResultSetReader(resultSet, result);
+    StringBuilder sb = new StringBuilder();
+    while (reader.hasNext()) {
+      LineRecord record = (LineRecord) reader.getRecord();
+      sb.append(record.getLine());
+    }
+    reader.close();
+    return sb.toString();
+  }
+
+  public static void close(OutputStream outputStream) {
+    close(outputStream, null, null);
+  }
+
+  public static void close(InputStream inputStream) {
+    close(null, inputStream, null);
+  }
+
+  public static void close(Fs fs) {
+    close(null, null, fs);
+  }
+
+  public static void close(OutputStream outputStream, InputStream inputStream, Fs fs) {
+    try {
+      if (outputStream != null) outputStream.close();
+    } catch (IOException e) {
+      // ignore exception
+    }
+    try {
+      if (inputStream != null) inputStream.close();
+    } catch (IOException e) {
+      // ignore exception
+    }
+    try {
+      if (fs != null) fs.close();
+    } catch (IOException e) {
+      // ignore exception
+    }
+  }
+
+  public static void close(Closeable closeable) {
+    CloseableUtils.closeQuietly(closeable);
+  }
+
+  public static String getJvmUser() {
+    return System.getProperty("user.name");
+  }
+
+  public static boolean isHDFSNode() {
+    File confPath = new File(HadoopConf.hadoopConfDir());
+    // TODO IO-client mode need return false
+    if (!confPath.exists() || confPath.isFile()) {
+      throw new StorageWarnException(
+          CONFIGURATION_NOT_READ.getErrorCode(), CONFIGURATION_NOT_READ.getErrorDesc());
+    } else return true;
+  }
+
+  /**
+   * Returns the FsPath by determining whether the path is a schema. By default, the FsPath of the
+   * file is returned.
+   *
+   * @param path
+   * @return
+   */
+  public static FsPath getFsPath(String path) {
+    if (path.startsWith(FILE_SCHEMA) || path.startsWith(HDFS_SCHEMA)) {
+      return new FsPath(path);
+    } else {
+      return new FsPath(FILE_SCHEMA + path);
+    }
+  }
+
+  public static int readBytes(InputStream inputStream, byte[] bytes, int len) {
+    int readLen = 0;
+    try {
+      int count = 0;
+      while (readLen < len) {
+        count = inputStream.read(bytes, readLen, len - readLen);
+
+        if (count == -1 && inputStream.available() < 1) {
+          return readLen;
+        }
+        readLen += count;
+      }
+    } catch (IOException e) {
+      logger.warn("FileSystemUtils readBytes failed", e);
+    }
+    return readLen;
+  }
+
+  public static String colToString(Object col, String nullValue) {
+    if (col == null) {
+      return nullValue;
+    } else if (col instanceof Double) {
+      return doubleToString((Double) col);
+    } else if ("NULL".equals(col) || "".equals(col)) {
+      return nullValue;
+    } else {
+      return col.toString();
+    }
+  }
+
+  public static String colToString(Object col) {
+    return colToString(col, "NULL");
+  }
+
+  public static boolean isIOProxy() {
+    return (boolean) StorageConfiguration.ENABLE_IO_PROXY.getValue();
+  }
+
+  public static byte[] mergeByteArrays(byte[] arr1, byte[] arr2) {
+    byte[] mergedArray = new byte[arr1.length + arr2.length];
+    System.arraycopy(arr1, 0, mergedArray, 0, arr1.length);
+    System.arraycopy(arr2, 0, mergedArray, arr1.length, arr2.length);
+    return mergedArray;
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/FSFactory.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/FSFactory.scala
deleted file mode 100644
index 5968332e1..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/FSFactory.scala
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage
-
-import org.apache.linkis.common.io.{Fs, FsPath}
-import org.apache.linkis.common.utils.Logging
-import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.UNSUPPORTED_FILE
-import org.apache.linkis.storage.exception.StorageFatalException
-import org.apache.linkis.storage.factory.BuildFactory
-import org.apache.linkis.storage.utils.{StorageConfiguration, StorageUtils}
-
-import java.text.MessageFormat
-
-object FSFactory extends Logging {
-
-  private val buildClasses: Map[String, BuildFactory] = StorageUtils.loadClass[BuildFactory](
-    StorageConfiguration.STORAGE_BUILD_FS_CLASSES.getValue,
-    t => t.fsName()
-  )
-
-  def getBuildFactory(fsName: String): BuildFactory = {
-    if (!buildClasses.contains(fsName)) {
-      throw new StorageFatalException(
-        UNSUPPORTED_FILE.getErrorCode,
-        MessageFormat.format(UNSUPPORTED_FILE.getErrorDesc, fsName)
-      )
-    }
-    buildClasses(fsName)
-  }
-
-  def getFs(fsType: String, proxyUser: String): Fs = {
-    val user = StorageUtils.getJvmUser
-    getBuildFactory(fsType).getFs(user, proxyUser)
-  }
-
-  def getFs(fsType: String): Fs = {
-    val user = StorageUtils.getJvmUser
-    getBuildFactory(fsType).getFs(user, user)
-  }
-
-  /**
-   *   1. If this machine has shared storage, the file:// type FS obtained here is the FS of the
-   *      process user. 2, if this machine does not have shared storage, then the file:// type FS
-   *      obtained is the proxy to the Remote (shared storage machine root) FS 3. If it is HDFS, it
-   *      returns the FS of the process user. 1、如果这台机器装有共享存储则这里获得的file://类型的FS为该进程用户的FS
-   *      2、如果这台机器没有共享存储则获得的file://类型的FS为代理到Remote(共享存储机器root)的FS 3、如果是HDFS则返回的就是该进程用户的FS
-   * @param fsPath
-   * @return
-   */
-  def getFs(fsPath: FsPath): Fs = {
-    getFs(fsPath.getFsType())
-  }
-
-  /**
-   *   1. If the process user is passed and the proxy user and the process user are consistent, the
-   *      file:// type FS is the FS of the process user (the shared storage exists) 2, if the
-   *      process user is passed and the proxy user and the process user are consistent and there is
-   *      no shared storage, the file:// type FS is the proxy to the remote (shared storage machine
-   *      root) FS 3. If the passed proxy user and process user are consistent, the hdfs type is the
-   *      FS of the process user. 4. If the proxy user and the process user are inconsistent, the
-   *      hdfs type is the FS after the proxy. 1、如果传了进程用户且代理用户和进程用户一致则file://类型的FS为该进程用户的FS(存在共享存储)
-   *      2、如果传了进程用户且代理用户和进程用户一致且没有共享存储则file://类型的FS为代理到Remote(共享存储机器root)的FS
-   *      3、如果传了的代理用户和进程用户一致则hdfs类型为该进程用户的FS 4、如果传了代理用户和进程用户不一致则hdfs类型为代理后的FS
-   *
-   * @param fsPath
-   * @param proxyUser
-   * @return
-   */
-  def getFsByProxyUser(fsPath: FsPath, proxyUser: String): Fs = {
-    getFs(fsPath.getFsType(), proxyUser)
-  }
-
-  def getFSByLabel(fs: String, label: String): Fs = {
-    val user = StorageUtils.getJvmUser
-    getBuildFactory(fs).getFs(user, user, label)
-  }
-
-  def getFSByLabelAndUser(fs: String, label: String, proxy: String): Fs = {
-    val user = StorageUtils.getJvmUser
-    getBuildFactory(fs).getFs(user, proxy, label)
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/conf/LinkisStorageConf.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/conf/LinkisStorageConf.scala
deleted file mode 100644
index bf03c7875..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/conf/LinkisStorageConf.scala
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.conf
-
-import org.apache.linkis.common.conf.CommonVars
-import org.apache.linkis.common.utils.ByteTimeUtils
-
-import org.apache.commons.lang3.StringUtils
-
-object LinkisStorageConf {
-  private val CONF_LOCK = new Object()
-
-  val HDFS_FILE_SYSTEM_REST_ERRS: String =
-    CommonVars
-      .apply(
-        "wds.linkis.hdfs.rest.errs",
-        ".*Filesystem closed.*|.*Failed to find any Kerberos tgt.*"
-      )
-      .getValue
-
-  val ROW_BYTE_MAX_LEN_STR = CommonVars("wds.linkis.resultset.row.max.str", "2m").getValue
-
-  val ROW_BYTE_MAX_LEN = ByteTimeUtils.byteStringAsBytes(ROW_BYTE_MAX_LEN_STR)
-
-  val FILE_TYPE = CommonVars(
-    "wds.linkis.storage.file.type",
-    "dolphin,sql,scala,py,hql,python,out,log,text,sh,jdbc,ngql,psql,fql,tsql"
-  ).getValue
-
-  private var fileTypeArr: Array[String] = null
-
-  private def fileTypeArrParser(fileType: String): Array[String] = {
-    if (StringUtils.isBlank(fileType)) Array()
-    else fileType.split(",")
-  }
-
-  def getFileTypeArr: Array[String] = {
-    if (fileTypeArr == null) {
-      CONF_LOCK.synchronized {
-        if (fileTypeArr == null) {
-          fileTypeArr = fileTypeArrParser(FILE_TYPE)
-        }
-      }
-    }
-    fileTypeArr
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/StorageCSVReader.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/StorageCSVReader.scala
deleted file mode 100644
index 6fc91c836..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/StorageCSVReader.scala
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.csv
-
-class StorageCSVReader {}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/StorageCSVWriter.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/StorageCSVWriter.scala
deleted file mode 100644
index f9b811b6a..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/csv/StorageCSVWriter.scala
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.csv
-
-import org.apache.linkis.common.io.{MetaData, Record}
-import org.apache.linkis.common.utils.Logging
-import org.apache.linkis.storage.domain.DataType
-import org.apache.linkis.storage.resultset.table.{TableMetaData, TableRecord}
-
-import org.apache.commons.io.IOUtils
-import org.apache.commons.lang3.StringUtils
-
-import java.io._
-
-class StorageCSVWriter(
-    val charset: String,
-    val separator: String,
-    val quoteRetouchEnable: Boolean,
-    val outputStream: OutputStream
-) extends CSVFsWriter
-    with Logging {
-
-  private val delimiter = separator match {
-    case separ if StringUtils.isNotEmpty(separ) => separ
-    case _ => '\t'
-  }
-
-  private val buffer: StringBuilder = new StringBuilder(50000)
-
-  @scala.throws[IOException]
-  override def addMetaData(metaData: MetaData): Unit = {
-    val head = metaData.asInstanceOf[TableMetaData].columns.map(_.columnName)
-    write(head)
-  }
-
-  private def compact(row: Array[String]): String = {
-    val quotationMarks: String = "\""
-    def decorateValue(v: String): String = {
-      if (StringUtils.isBlank(v)) v
-      else {
-        if (quoteRetouchEnable) {
-          s"$quotationMarks${v.replaceAll(quotationMarks, "")}$quotationMarks"
-        } else v
-      }
-    }
-    if (logger.isDebugEnabled()) {
-      logger.debug("delimiter:" + delimiter.toString)
-    }
-
-    row.map(x => decorateValue(x)).toList.mkString(delimiter.toString) + "\n"
-  }
-
-  private def write(row: Array[String]) = {
-    val content: String = compact(row)
-    if (buffer.length + content.length > 49500) {
-      IOUtils.write(buffer.toString().getBytes(charset), outputStream)
-      buffer.clear()
-    }
-    buffer.append(content)
-  }
-
-  @scala.throws[IOException]
-  override def addRecord(record: Record): Unit = {
-    val body = record.asInstanceOf[TableRecord].row.map(DataType.valueToString)
-    write(body)
-  }
-
-  override def flush(): Unit = {
-    IOUtils.write(buffer.toString().getBytes(charset), outputStream)
-    buffer.clear()
-  }
-
-  override def close(): Unit = {
-    flush()
-    IOUtils.closeQuietly(outputStream)
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/domain/DataType.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/domain/DataType.scala
deleted file mode 100644
index 55c82abd3..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/domain/DataType.scala
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.domain
-
-import org.apache.linkis.common.utils.{Logging, Utils}
-
-import java.math.{BigDecimal => JavaBigDecimal}
-import java.sql.{Date, Timestamp}
-
-object DataType extends Logging {
-
-  val NULL_VALUE = "NULL"
-  val LOWCASE_NULL_VALUE = "null"
-  // TODO Change to fine-grained regular expressions(改为精细化正则表达式)
-  val DECIMAL_REGEX = "^decimal\\(\\d*\\,\\d*\\)".r.unanchored
-
-  val SHORT_REGEX = "^short.*".r.unanchored
-  val INT_REGEX = "^int.*".r.unanchored
-  val LONG_REGEX = "^long.*".r.unanchored
-  val BIGINT_REGEX = "^bigint.*".r.unanchored
-  val FLOAT_REGEX = "^float.*".r.unanchored
-  val DOUBLE_REGEX = "^double.*".r.unanchored
-
-  val VARCHAR_REGEX = "^varchar.*".r.unanchored
-  val CHAR_REGEX = "^char.*".r.unanchored
-
-  val ARRAY_REGEX = "array.*".r.unanchored
-
-  val MAP_REGEX = "map.*".r.unanchored
-
-  val LIST_REGEX = "list.*".r.unanchored
-
-  val STRUCT_REGEX = "struct.*".r.unanchored
-
-  implicit def toDataType(dataType: String): DataType = dataType match {
-    case "void" | "null" => NullType
-    case "string" => StringType
-    case "boolean" => BooleanType
-    case SHORT_REGEX() => ShortIntType
-    case LONG_REGEX() => LongType
-    case BIGINT_REGEX() => BigIntType
-    case INT_REGEX() | "integer" | "smallint" => IntType
-    case FLOAT_REGEX() => FloatType
-    case DOUBLE_REGEX() => DoubleType
-    case VARCHAR_REGEX() => VarcharType
-    case CHAR_REGEX() => CharType
-    case "date" => DateType
-    case "timestamp" => TimestampType
-    case "binary" => BinaryType
-    case "decimal" | DECIMAL_REGEX() => DecimalType
-    case ARRAY_REGEX() => ArrayType
-    case MAP_REGEX() => MapType
-    case LIST_REGEX() => ListType
-    case STRUCT_REGEX() => StructType
-    case _ => StringType
-  }
-
-  def toValue(dataType: DataType, value: String): Any = Utils.tryCatch(dataType match {
-    case NullType => null
-    case StringType | CharType | VarcharType | StructType | ListType | ArrayType | MapType =>
-      value
-    case BooleanType => if (isNumberNull(value)) null else value.toBoolean
-    case ShortIntType => if (isNumberNull(value)) null else value.toShort
-    case IntType => if (isNumberNull(value)) null else value.toInt
-    case LongType | BigIntType => if (isNumberNull(value)) null else value.toLong
-    case FloatType => if (isNumberNull(value)) null else value.toFloat
-    case DoubleType => if (isNumberNull(value)) null else value.toDouble
-    case DecimalType => if (isNumberNull(value)) null else new JavaBigDecimal(value)
-    case DateType => if (isNumberNull(value)) null else Date.valueOf(value)
-    case TimestampType =>
-      if (isNumberNull(value)) null else Timestamp.valueOf(value).toString.stripSuffix(".0")
-    case BinaryType => if (isNull(value)) null else value.getBytes()
-    case _ => value
-  }) { t =>
-    logger.debug(s"Failed to  $value switch  to dataType:", t)
-    value
-  }
-
-  def isNull(value: String): Boolean =
-    if (value == null || value == NULL_VALUE || value.trim == "") true else false
-
-  def isNumberNull(value: String): Boolean =
-    if (null == value || NULL_VALUE.equalsIgnoreCase(value) || value.trim == "") {
-      true
-    } else {
-      false
-    }
-
-  def valueToString(value: Any): String = {
-    if (null == value) return LOWCASE_NULL_VALUE
-    value match {
-      case javaDecimal: JavaBigDecimal =>
-        javaDecimal.toPlainString
-      case _ => value.toString
-    }
-  }
-
-}
-
-abstract class DataType(val typeName: String, val javaSQLType: Int) {
-  override def toString: String = typeName
-}
-
-case object NullType extends DataType("void", 0)
-case object StringType extends DataType("string", 12)
-case object BooleanType extends DataType("boolean", 16)
-case object TinyIntType extends DataType("tinyint", -6)
-case object ShortIntType extends DataType("short", 5)
-case object IntType extends DataType("int", 4)
-case object LongType extends DataType("long", -5)
-case object BigIntType extends DataType("bigint", -5)
-case object FloatType extends DataType("float", 6)
-case object DoubleType extends DataType("double", 8)
-case object CharType extends DataType("char", 1)
-case object VarcharType extends DataType("varchar", 12)
-case object DateType extends DataType("date", 91)
-case object TimestampType extends DataType("timestamp", 93)
-case object BinaryType extends DataType("binary", -2)
-case object DecimalType extends DataType("decimal", 3)
-case object ArrayType extends DataType("array", 2003)
-case object MapType extends DataType("map", 2000)
-case object ListType extends DataType("list", 2001)
-case object StructType extends DataType("struct", 2002)
-case object BigDecimalType extends DataType("bigdecimal", 3)
-
-case class Column(columnName: String, dataType: DataType, comment: String) {
-
-  def toArray: Array[Any] = {
-    Array[Any](columnName, dataType, comment)
-  }
-
-  override def toString: String = s"columnName:$columnName,dataType:$dataType,comment:$comment"
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/domain/Dolphin.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/domain/Dolphin.scala
deleted file mode 100644
index 378c2c2ec..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/domain/Dolphin.scala
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.domain
-
-import org.apache.linkis.common.utils.Logging
-import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.FAILED_TO_READ_INTEGER
-import org.apache.linkis.storage.exception.StorageWarnException
-import org.apache.linkis.storage.utils.{StorageConfiguration, StorageUtils}
-
-import java.io.{InputStream, IOException}
-
-object Dolphin extends Logging {
-
-  val CHAR_SET = StorageConfiguration.STORAGE_RS_FILE_TYPE.getValue
-  val MAGIC = "dolphin"
-
-  val MAGIC_BYTES = MAGIC.getBytes(StorageConfiguration.STORAGE_RS_FILE_TYPE.getValue)
-  val MAGIC_LEN = MAGIC_BYTES.length
-
-  val DOLPHIN_FILE_SUFFIX = ".dolphin"
-
-  val COL_SPLIT = ","
-  val COL_SPLIT_BYTES = COL_SPLIT.getBytes("utf-8")
-  val COL_SPLIT_LEN = COL_SPLIT_BYTES.length
-
-  val NULL = "NULL"
-  val NULL_BYTES = "NULL".getBytes("utf-8")
-
-  val INT_LEN = 10
-
-  val FILE_EMPTY = 31
-
-  def getBytes(value: Any): Array[Byte] = {
-    value.toString.getBytes(CHAR_SET)
-  }
-
-  /**
-   * Convert a bytes array to a String content 将bytes数组转换为String内容
-   * @param bytes
-   * @param start
-   * @param len
-   * @return
-   */
-  def getString(bytes: Array[Byte], start: Int, len: Int): String =
-    new String(bytes, start, len, Dolphin.CHAR_SET)
-
-  /**
-   * Read an integer value that converts the array to a byte of length 10 bytes
-   * 读取整数值,该值为将数组转换为10字节长度的byte
-   * @param inputStream
-   * @return
-   */
-  def readInt(inputStream: InputStream): Int = {
-    val bytes = new Array[Byte](INT_LEN + 1)
-    if (StorageUtils.readBytes(inputStream, bytes, INT_LEN) != INT_LEN) {
-      throw new StorageWarnException(
-        FAILED_TO_READ_INTEGER.getErrorCode,
-        FAILED_TO_READ_INTEGER.getErrorDesc
-      )
-    }
-    getString(bytes, 0, INT_LEN).toInt
-  }
-
-  /**
-   * Print integers at a fixed length(将整数按固定长度打印)
-   * @param value
-   * @return
-   */
-  def getIntBytes(value: Int): Array[Byte] = {
-    val str = value.toString
-    val res = "0" * (INT_LEN - str.length) + str
-    Dolphin.getBytes(res)
-  }
-
-  def getType(inputStream: InputStream): String = {
-    val bytes = new Array[Byte](100)
-    val len = StorageUtils.readBytes(inputStream, bytes, Dolphin.MAGIC_LEN + INT_LEN)
-    if (len == -1) return null
-    getType(Dolphin.getString(bytes, 0, len))
-  }
-
-  def getType(content: String): String = {
-    if (content.length < MAGIC.length || content.substring(0, MAGIC.length) != MAGIC) {
-      throw new IOException(s"File header type must be dolphin,content:$content is not")
-    }
-    content.substring(MAGIC.length, MAGIC.length + INT_LEN).toInt.toString
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/domain/MethodEntity.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/domain/MethodEntity.scala
deleted file mode 100644
index fac0a2d01..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/domain/MethodEntity.scala
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.domain
-
-import java.lang.reflect.Type
-
-import com.google.gson.GsonBuilder
-
-/**
- * @param id
- *   Engine unique Id(engine唯一的Id)
- * @param fsType
- *   Fs type(fs类型)
- * @param creatorUser
- *   Create a user to start the corresponding jvm user(创建用户为对应启动的jvm用户)
- * @param proxyUser
- *   Proxy user(代理用户)
- * @param clientIp
- *   client Ip for whitelist control(ip用于白名单控制)
- * @param methodName
- *   Method name called(调用的方法名)
- * @param params
- *   Method parameter(方法参数)
- */
-case class MethodEntity(
-    id: Long,
-    fsType: String,
-    creatorUser: String,
-    proxyUser: String,
-    clientIp: String,
-    methodName: String,
-    params: Array[AnyRef]
-) {
-
-  override def toString: String = {
-    s"id:$id, methodName:$methodName, fsType:$fsType, " +
-      s"creatorUser:$creatorUser, proxyUser:$proxyUser, clientIp:$clientIp, "
-  }
-
-}
-
-object MethodEntitySerializer {
-
-  val gson = new GsonBuilder().setDateFormat("yyyy-MM-dd'T'HH:mm:ssZ").create
-
-  /**
-   * Serialized to code as a MethodEntity object 序列化为code为MethodEntity对象
-   * @param code
-   * @return
-   */
-  def deserializer(code: String): MethodEntity = gson.fromJson(code, classOf[MethodEntity])
-
-  /**
-   * Serialize MethodEntity to code 序列化MethodEntity为code
-   * @param methodEntity
-   * @return
-   */
-  def serializer(methodEntity: MethodEntity): String = gson.toJson(methodEntity)
-
-  /**
-   * Serialize a java object as a string 序列化java对象为字符串
-   * @param value
-   * @return
-   */
-  def serializerJavaObject(value: Any): String = gson.toJson(value)
-
-  /**
-   * Deserialize a string into a java object 将字符串解序列化为java对象
-   * @param json
-   * @param classType
-   * @tparam T
-   * @return
-   */
-  def deserializerToJavaObject[T](json: String, classType: Class[T]): T = {
-    gson.fromJson(json, classType)
-  }
-
-  def deserializerToJavaObject[T](json: String, oType: Type): T = {
-    gson.fromJson(json, oType)
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/ExcelFsWriter.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/ExcelFsWriter.scala
deleted file mode 100644
index 079920bef..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/ExcelFsWriter.scala
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.excel
-
-import org.apache.linkis.common.io.FsWriter
-
-import java.io.OutputStream
-
-abstract class ExcelFsWriter extends FsWriter {
-  val charset: String
-  val sheetName: String
-  val dateFormat: String
-  val autoFormat: Boolean
-}
-
-object ExcelFsWriter {
-
-  def getExcelFsWriter(
-      charset: String,
-      sheetName: String,
-      dateFormat: String,
-      outputStream: OutputStream,
-      autoFormat: Boolean
-  ): ExcelFsWriter = new StorageExcelWriter(
-    charset,
-    sheetName,
-    dateFormat,
-    outputStream: OutputStream,
-    autoFormat: Boolean
-  )
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/ExcelXlsReader.java b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/ExcelXlsReader.java
deleted file mode 100644
index aac0ff54a..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/ExcelXlsReader.java
+++ /dev/null
@@ -1,378 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
- 
-package org.apache.linkis.storage.excel;
-
-import org.apache.poi.hssf.eventusermodel.EventWorkbookBuilder.SheetRecordCollectingListener;
-import org.apache.poi.hssf.eventusermodel.*;
-import org.apache.poi.hssf.eventusermodel.dummyrecord.LastCellOfRowDummyRecord;
-import org.apache.poi.hssf.eventusermodel.dummyrecord.MissingCellDummyRecord;
-import org.apache.poi.hssf.model.HSSFFormulaParser;
-import org.apache.poi.hssf.record.*;
-import org.apache.poi.hssf.usermodel.HSSFWorkbook;
-import org.apache.poi.poifs.filesystem.POIFSFileSystem;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-
-
-public class ExcelXlsReader implements HSSFListener {
-    private int minColumns = -1;
-
-    private POIFSFileSystem fs;
-
-    private InputStream inputStream;
-
-    private int lastRowNumber;
-
-    private int lastColumnNumber;
-
-    /**
-     * Should we output the formula, or the value it has?
-     */
-    private boolean outputFormulaValues = true;
-
-    /**
-     * For parsing Formulas
-     */
-    private SheetRecordCollectingListener workbookBuildingListener;
-
-    // excel2003Workbook(工作薄)
-    private HSSFWorkbook stubWorkbook;
-
-    // Records we pick up as we process
-    private SSTRecord sstRecord;
-
-    private FormatTrackingHSSFListener formatListener;
-
-    // Table index(表索引)
-    private int sheetIndex = -1;
-
-    private BoundSheetRecord[] orderedBSRs;
-
-    @SuppressWarnings("unchecked")
-    private ArrayList boundSheetRecords = new ArrayList();
-
-    // For handling formulas with string results
-    private int nextRow;
-
-    private int nextColumn;
-
-    private boolean outputNextStringRecord;
-
-    // Current line(当前行)
-    private int curRow = 0;
-
-    // a container that stores row records(存储行记录的容器)
-    private List<String> rowlist = new ArrayList<String>();
-
-
-    @SuppressWarnings("unused")
-    private String sheetName;
-
-    private IExcelRowDeal excelRowDeal;
-
-
-    public void init(IExcelRowDeal excelRowDeal, InputStream inputStream) {
-        this.excelRowDeal = excelRowDeal;
-        this.inputStream = inputStream;
-    }
-
-
-    /**
-     * Traverse all the sheets under excel
-     * 遍历excel下所有的sheet
-     * @throws IOException
-     */
-    public void process() throws IOException {
-        this.fs = new POIFSFileSystem(this.inputStream);
-        MissingRecordAwareHSSFListener listener = new MissingRecordAwareHSSFListener(this);
-        formatListener = new FormatTrackingHSSFListener(listener);
-        HSSFEventFactory factory = new HSSFEventFactory();
-        HSSFRequest request = new HSSFRequest();
-        if (outputFormulaValues) {
-            request.addListenerForAllRecords(formatListener);
-        } else {
-            workbookBuildingListener = new SheetRecordCollectingListener(formatListener);
-            request.addListenerForAllRecords(workbookBuildingListener);
-        }
-        factory.processWorkbookEvents(request, fs);
-    }
-
-    /**
-     * HSSFListener listener method, processing Record
-     * HSSFListener 监听方法,处理 Record
-     */
-    @Override
-    @SuppressWarnings("unchecked")
-    public void processRecord(Record record) {
-        int thisRow = -1;
-        int thisColumn = -1;
-        String thisStr = null;
-        String value = null;
-        switch (record.getSid()) {
-            case BoundSheetRecord.sid:
-                boundSheetRecords.add(record);
-                break;
-            case BOFRecord.sid:
-                BOFRecord br = (BOFRecord) record;
-                if (br.getType() == BOFRecord.TYPE_WORKSHEET) {
-                    // Create a child workbook if needed(如果有需要,则建立子工作薄)
-                    if (workbookBuildingListener != null && stubWorkbook == null) {
-                        stubWorkbook = workbookBuildingListener.getStubHSSFWorkbook();
-                    }
-
-                    sheetIndex++;
-                    if (orderedBSRs == null) {
-                        orderedBSRs = BoundSheetRecord.orderByBofPosition(boundSheetRecords);
-                    }
-                    sheetName = orderedBSRs[sheetIndex].getSheetname();
-                }
-                break;
-
-            case SSTRecord.sid:
-                sstRecord = (SSTRecord) record;
-                break;
-
-            case BlankRecord.sid:
-                BlankRecord brec = (BlankRecord) record;
-                thisRow = brec.getRow();
-                thisColumn = brec.getColumn();
-                thisStr = "";
-                rowlist.add(thisColumn, thisStr);
-                break;
-            case BoolErrRecord.sid: // Cell is boolean(单元格为布尔类型)
-                BoolErrRecord berec = (BoolErrRecord) record;
-                thisRow = berec.getRow();
-                thisColumn = berec.getColumn();
-                thisStr = berec.getBooleanValue() + "";
-                rowlist.add(thisColumn, thisStr);
-                break;
-
-            case FormulaRecord.sid: // Cell is a formula type(单元格为公式类型)
-                FormulaRecord frec = (FormulaRecord) record;
-                thisRow = frec.getRow();
-                thisColumn = frec.getColumn();
-                if (outputFormulaValues) {
-                    if (Double.isNaN(frec.getValue())) {
-                        // Formula result is a string
-                        // This is stored in the next record
-                        outputNextStringRecord = true;
-                        nextRow = frec.getRow();
-                        nextColumn = frec.getColumn();
-                    } else {
-                        thisStr = formatListener.formatNumberDateCell(frec);
-                    }
-                } else {
-                    thisStr = '"' + HSSFFormulaParser.toFormulaString(stubWorkbook, frec.getParsedExpression()) + '"';
-                }
-                rowlist.add(thisColumn, thisStr);
-                break;
-            case StringRecord.sid:// a string of formulas in a cell(单元格中公式的字符串)
-                if (outputNextStringRecord) {
-                    // String for formula
-                    StringRecord srec = (StringRecord) record;
-                    thisStr = srec.getString();
-                    thisRow = nextRow;
-                    thisColumn = nextColumn;
-                    outputNextStringRecord = false;
-                }
-                break;
-            case LabelRecord.sid:
-                LabelRecord lrec = (LabelRecord) record;
-                curRow = thisRow = lrec.getRow();
-                thisColumn = lrec.getColumn();
-                value = lrec.getValue().trim();
-                value = value.equals("") ? " " : value;
-                this.rowlist.add(thisColumn, value);
-                break;
-            case LabelSSTRecord.sid: // Cell is a string type(单元格为字符串类型)
-                LabelSSTRecord lsrec = (LabelSSTRecord) record;
-                curRow = thisRow = lsrec.getRow();
-                thisColumn = lsrec.getColumn();
-                if (sstRecord == null) {
-                    rowlist.add(thisColumn, " ");
-                } else {
-                    value = sstRecord.getString(lsrec.getSSTIndex()).toString().trim();
-                    value = value.equals("") ? " " : value;
-                    rowlist.add(thisColumn, value);
-                }
-                break;
-            case NumberRecord.sid: // Cell is a numeric type(单元格为数字类型)
-                NumberRecord numrec = (NumberRecord) record;
-                curRow = thisRow = numrec.getRow();
-                thisColumn = numrec.getColumn();
-                value = formatListener.formatNumberDateCell(numrec).trim();
-                value = value.equals("") ? "0" : value;
-                // Add column values to the container(向容器加入列值)
-                rowlist.add(thisColumn, value);
-                break;
-            default:
-                break;
-        }
-
-        // Encountered a new line of operations(遇到新行的操作)(
-        if (thisRow != -1 && thisRow != lastRowNumber) {
-            lastColumnNumber = -1;
-        }
-
-        // Null operation(空值的操作)
-        if (record instanceof MissingCellDummyRecord) {
-            MissingCellDummyRecord mc = (MissingCellDummyRecord) record;
-            curRow = thisRow = mc.getRow();
-            thisColumn = mc.getColumn();
-            rowlist.add(thisColumn, " ");
-        }
-
-        // Update row and column values(更新行和列的值)
-        if (thisRow > -1)
-            lastRowNumber = thisRow;
-        if (thisColumn > -1)
-            lastColumnNumber = thisColumn;
-
-        // End of line operation(行结束时的操作)
-        if (record instanceof LastCellOfRowDummyRecord) {
-            if (minColumns > 0) {
-                // Column value is re-empted(列值重新置空)
-                if (lastColumnNumber == -1) {
-                    lastColumnNumber = 0;
-                }
-            }
-            lastColumnNumber = -1;
-
-            // At the end of each line, the dealRow() method(每行结束时, dealRow() 方法)
-            excelRowDeal.dealRow(orderedBSRs, sheetIndex, curRow, rowlist);
-            // Empty container(清空容器)
-            rowlist.clear();
-        }
-    }
-
-    public void close() {
-        try {
-            if (fs != null) {
-                fs.close();
-            }
-            if (inputStream != null) {
-                inputStream.close();
-            }
-        } catch (Exception e) {
-
-        }
-    }
-
-    public static void main(String[] args) {
-
-    }
-}
-
-class ExcelAnalysisException extends RuntimeException {
-    public ExcelAnalysisException() {
-    }
-
-    public ExcelAnalysisException(String message) {
-        super(message);
-    }
-
-    public ExcelAnalysisException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public ExcelAnalysisException(Throwable cause) {
-        super(cause);
-    }
-}
-
-interface IExcelRowDeal {
-    void dealRow(BoundSheetRecord[] orderedBSRs, int sheetIndex, int curRow, List<String> rowlist);
-}
-
-
-class FirstRowDeal implements IExcelRowDeal {
-
-    private List<String> sheetNames = new ArrayList<>();
-    private List<String> row;
-
-    public List<String> getSheetNames() {
-        return sheetNames;
-    }
-
-    public void setSheetNames(List<String> sheetNames) {
-        this.sheetNames = sheetNames;
-    }
-
-    public List<String> getRow() {
-        return row;
-    }
-
-    public void setRow(List<String> row) {
-        this.row = row;
-    }
-
-    @Override
-    public void dealRow(BoundSheetRecord[] orderedBSRs, int sheetIndex, int curRow, List<String> rowlist) {
-        for (BoundSheetRecord record : orderedBSRs) {
-            sheetNames.add(record.getSheetname());
-        }
-        row = rowlist;
-        throw new ExcelAnalysisException("Finished to deal first row");
-    }
-
-}
-
-class RowToCsvDeal implements IExcelRowDeal {
-
-    private Map<String, Object> params;
-    private List<String> sheetNames;
-    private OutputStream outputStream;
-    private Boolean hasHeader;
-    private Boolean fisrtRow = true;
-    public void init(Boolean hasHeader, List<String> sheetNames, OutputStream outputStream) {
-        this.hasHeader = hasHeader;
-        this.sheetNames = sheetNames;
-        this.outputStream = outputStream;
-    }
-
-    @Override
-    public void dealRow(BoundSheetRecord[] orderedBSRs, int sheetIndex, int curRow, List<String> rowlist) {
-        String sheetName = orderedBSRs[sheetIndex].getSheetname();
-        if (sheetNames == null || sheetNames.isEmpty() || sheetNames.contains(sheetName)) {
-            if (! (curRow == 0 && hasHeader)) {
-                try {
-                    if (fisrtRow) {
-                        fisrtRow = false;
-                    } else {
-                        outputStream.write("\n".getBytes());
-                    }
-                    int len = rowlist.size();
-                    for (int i =  0; i < len; i ++) {
-                        outputStream.write(rowlist.get(i).replaceAll("\n|\t", " ").getBytes("utf-8"));
-                        if (i < len -1) {
-                            outputStream.write("\t".getBytes());
-                        }
-                    }
-                } catch (IOException e) {
-                    throw new RuntimeException(e);
-                }
-            }
-        }
-    }
-}
\ No newline at end of file
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/StorageExcelReader.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/StorageExcelReader.scala
deleted file mode 100644
index 4a4f9bdf1..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/StorageExcelReader.scala
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.excel
-
-class StorageExcelReader {}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/StorageExcelWriter.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/StorageExcelWriter.scala
deleted file mode 100644
index 8d0f0bdde..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/excel/StorageExcelWriter.scala
+++ /dev/null
@@ -1,236 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.excel
-
-import org.apache.linkis.common.io.{MetaData, Record}
-import org.apache.linkis.common.utils.{Logging, Utils}
-import org.apache.linkis.storage.domain._
-import org.apache.linkis.storage.resultset.table.{TableMetaData, TableRecord}
-
-import org.apache.commons.io.IOUtils
-import org.apache.poi.ss.usermodel._
-import org.apache.poi.xssf.streaming.{SXSSFCell, SXSSFSheet, SXSSFWorkbook}
-
-import java.io._
-import java.math.BigDecimal
-import java.util
-import java.util.Date
-
-import scala.collection.mutable.ArrayBuffer
-
-class StorageExcelWriter(
-    val charset: String,
-    val sheetName: String,
-    val dateFormat: String,
-    val outputStream: OutputStream,
-    val autoFormat: Boolean
-) extends ExcelFsWriter
-    with Logging {
-
-  protected var workBook: SXSSFWorkbook = _
-  protected var sheet: SXSSFSheet = _
-  protected var format: DataFormat = _
-  protected var types: Array[DataType] = _
-  protected var rowPoint = 0
-  protected var columnCounter = 0
-  protected val styles = new util.HashMap[String, CellStyle]()
-  protected var isFlush = true
-  protected val os = new ByteArrayOutputStream()
-  protected var is: ByteArrayInputStream = _
-
-  def init: Unit = {
-    workBook = new SXSSFWorkbook()
-    sheet = workBook.createSheet(sheetName)
-  }
-
-  def getDefaultHeadStyle: CellStyle = {
-    val headerFont = workBook.createFont
-    headerFont.setBold(true)
-    headerFont.setFontHeightInPoints(14.toShort)
-    headerFont.setColor(IndexedColors.RED.getIndex)
-    val headerCellStyle = workBook.createCellStyle
-    headerCellStyle.setFont(headerFont)
-    headerCellStyle
-  }
-
-  def getWorkBook: Workbook = {
-    // 自适应列宽
-    sheet.trackAllColumnsForAutoSizing()
-    for (elem <- 0 to columnCounter) {
-      sheet.autoSizeColumn(elem)
-    }
-    workBook
-  }
-
-  def createCellStyle(dataType: DataType): CellStyle = {
-    val style = workBook.createCellStyle()
-    format = workBook.createDataFormat()
-    dataType.toString match {
-      case _ => style.setDataFormat(format.getFormat("@"))
-    }
-    if (autoFormat) {
-      dataType match {
-        case StringType => style.setDataFormat(format.getFormat("@"))
-        case TinyIntType => style.setDataFormat(format.getFormat("#"))
-        case ShortIntType => style.setDataFormat(format.getFormat("#"))
-        case IntType => style.setDataFormat(format.getFormat("#"))
-        case LongType => style.setDataFormat(format.getFormat("#.##E+00"))
-        case BigIntType => style.setDataFormat(format.getFormat("#.##E+00"))
-        case FloatType => style.setDataFormat(format.getFormat("#.0000000000"))
-        case DoubleType => style.setDataFormat(format.getFormat("#.0000000000"))
-        case CharType => style.setDataFormat(format.getFormat("@"))
-        case VarcharType => style.setDataFormat(format.getFormat("@"))
-        case DateType => style.setDataFormat(format.getFormat("m/d/yy h:mm"))
-        case TimestampType => style.setDataFormat(format.getFormat("m/d/yy h:mm"))
-        case DecimalType => style.setDataFormat(format.getFormat("#.000000000"))
-        case BigDecimalType => style.setDataFormat(format.getFormat("#.000000000"))
-        case _ => style.setDataFormat(format.getFormat("@"))
-      }
-    }
-    style
-  }
-
-  def getCellStyle(dataType: DataType): CellStyle = {
-    val style = styles.get(dataType.typeName)
-    if (style == null) {
-      val newStyle = createCellStyle(dataType)
-      styles.put(dataType.typeName, newStyle)
-      newStyle
-    } else {
-      style
-    }
-  }
-
-  @scala.throws[IOException]
-  override def addMetaData(metaData: MetaData): Unit = {
-    init
-    val tableHead = sheet.createRow(0)
-    val columns = metaData.asInstanceOf[TableMetaData].columns
-    val columnType = new ArrayBuffer[DataType]()
-    for (elem <- columns) {
-      val headCell = tableHead.createCell(columnCounter)
-      headCell.setCellValue(elem.columnName)
-      headCell.setCellStyle(getDefaultHeadStyle)
-      columnType += elem.dataType
-      columnCounter += 1
-    }
-    types = columnType.toArray
-    rowPoint += 1
-  }
-
-  @scala.throws[IOException]
-  override def addRecord(record: Record): Unit = {
-    // TODO: 是否需要替换null值
-    val tableBody = sheet.createRow(rowPoint)
-    var colunmPoint = 0
-    val excelRecord = record.asInstanceOf[TableRecord].row
-    for (elem <- excelRecord) {
-      val cell = tableBody.createCell(colunmPoint)
-      val dataType = types.apply(colunmPoint)
-      if (autoFormat) {
-        setCellTypeValue(dataType, elem, cell)
-      } else {
-        cell.setCellValue(DataType.valueToString(elem))
-      }
-      cell.setCellStyle(getCellStyle(dataType))
-      colunmPoint += 1
-    }
-    rowPoint += 1
-  }
-
-  private def setCellTypeValue(dataType: DataType, elem: Any, cell: SXSSFCell): Unit = {
-    if (null == elem) return
-    Utils.tryCatch {
-      dataType match {
-        case StringType => cell.setCellValue(DataType.valueToString(elem))
-        case TinyIntType => cell.setCellValue(elem.toString.toInt)
-        case ShortIntType => cell.setCellValue(elem.toString.toInt)
-        case IntType => cell.setCellValue(elem.toString.toInt)
-        case LongType => cell.setCellValue(elem.toString.toLong)
-        case BigIntType => cell.setCellValue(elem.toString.toLong)
-        case FloatType => cell.setCellValue(elem.toString.toFloat)
-        case DoubleType =>
-          doubleCheck(elem.toString)
-          cell.setCellValue(elem.toString.toDouble)
-        case CharType => cell.setCellValue(DataType.valueToString(elem))
-        case VarcharType => cell.setCellValue(DataType.valueToString(elem))
-        case DateType => cell.setCellValue(getDate(elem))
-        case TimestampType => cell.setCellValue(getDate(elem))
-        case DecimalType =>
-          doubleCheck(DataType.valueToString(elem))
-          cell.setCellValue(DataType.valueToString(elem).toDouble)
-        case BigDecimalType =>
-          doubleCheck(DataType.valueToString(elem))
-          cell.setCellValue(DataType.valueToString(elem).toDouble)
-        case _ =>
-          val value = DataType.valueToString(elem)
-          cell.setCellValue(value)
-      }
-    } { case e: Exception =>
-      cell.setCellValue(DataType.valueToString(elem))
-    }
-  }
-
-  private def getDate(value: Any): Date = {
-    if (value.isInstanceOf[Date]) {
-      value.asInstanceOf[Date]
-    } else {
-      throw new NumberFormatException(
-        s"Value ${value} with class : ${value.getClass.getName} is not a valid type of Date."
-      );
-    }
-  }
-
-  /**
-   * Check whether the double exceeds the number of digits, which will affect the data accuracy
-   * @param elemValue
-   */
-  private def doubleCheck(elemValue: String): Unit = {
-    val value = new BigDecimal(elemValue).stripTrailingZeros
-    if ((value.precision - value.scale) > 15) {
-      throw new NumberFormatException(
-        s"Value ${elemValue} error : This data exceeds 15 significant digits."
-      );
-    }
-  }
-
-  override def flush(): Unit = {
-    getWorkBook.write(os)
-    val content: Array[Byte] = os.toByteArray
-    is = new ByteArrayInputStream(content)
-    val buffer: Array[Byte] = new Array[Byte](1024)
-    var bytesRead: Int = 0
-    while (isFlush) {
-      bytesRead = is.read(buffer, 0, 1024)
-      if (bytesRead == -1) {
-        isFlush = false
-      } else {
-        outputStream.write(buffer, 0, bytesRead)
-      }
-    }
-  }
-
-  override def close(): Unit = {
-    if (isFlush) flush()
-    IOUtils.closeQuietly(outputStream)
-    IOUtils.closeQuietly(is)
-    IOUtils.closeQuietly(os)
-    IOUtils.closeQuietly(workBook)
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/io/IOClient.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/io/IOClient.scala
deleted file mode 100644
index b21bf7e49..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/io/IOClient.scala
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.io
-
-import org.apache.linkis.storage.domain.MethodEntity
-import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.MUST_REGISTER_TOC
-import org.apache.linkis.storage.exception.StorageErrorException
-
-import java.util.UUID
-
-import org.slf4j.{Logger, LoggerFactory}
-
-/**
- * IOClient is used to execute the proxy as the ujes code execution entry in io and get the return
- * result. IOClient用于在io进行代理作为ujes的代码执行入口,并获取返回结果
- */
-trait IOClient {
-
-  def execute(user: String, methodEntity: MethodEntity, params: java.util.Map[String, Any]): String
-
-  def executeWithEngine(
-      user: String,
-      methodEntity: MethodEntity,
-      params: java.util.Map[String, Any]
-  ): Array[String]
-
-}
-
-object IOClient {
-  val logger: Logger = LoggerFactory.getLogger(classOf[IOClient])
-  var ioClient: IOClient = null
-
-  val SUCCESS = "SUCCESS"
-  val FAILED = "FAILED"
-
-  def getIOClient(): IOClient = {
-    if (ioClient == null) {
-      throw new StorageErrorException(
-        MUST_REGISTER_TOC.getErrorCode,
-        MUST_REGISTER_TOC.getErrorDesc
-      )
-    }
-    ioClient
-  }
-
-  /**
-   * This method is called when ioClient is initialized. ioClient初始化时会调用该方法
-   * @param client
-   *   IOClient
-   */
-  def register(client: IOClient): Unit = {
-    this.ioClient = client
-    logger.debug(s"IOClient: ${ioClient.toString} registered")
-  }
-
-  def getFSId(): String = {
-    UUID.randomUUID().toString
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/DefaultResultSetFactory.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/DefaultResultSetFactory.scala
deleted file mode 100644
index d4836731d..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/DefaultResultSetFactory.scala
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset
-
-import org.apache.linkis.common.io.{Fs, FsPath, MetaData, Record}
-import org.apache.linkis.common.io.resultset.ResultSet
-import org.apache.linkis.common.utils.{Logging, Utils}
-import org.apache.linkis.storage.FSFactory
-import org.apache.linkis.storage.domain.Dolphin
-import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.{
-  THE_FILE_IS_EMPTY,
-  UNSUPPORTED_RESULT
-}
-import org.apache.linkis.storage.exception.{StorageErrorException, StorageWarnException}
-import org.apache.linkis.storage.utils.{StorageConfiguration, StorageUtils}
-
-import org.apache.commons.lang3.StringUtils
-
-import java.text.MessageFormat
-import java.util
-import java.util.Locale
-
-class DefaultResultSetFactory extends ResultSetFactory with Logging {
-
-  private val resultClasses: Map[String, Class[ResultSet[ResultMetaData, ResultRecord]]] =
-    StorageUtils.loadClasses(
-      StorageConfiguration.STORAGE_RESULT_SET_CLASSES.getValue,
-      StorageConfiguration.STORAGE_RESULT_SET_PACKAGE.getValue,
-      t => t.newInstance().resultSetType().toLowerCase(Locale.getDefault)
-    )
-
-  val resultTypes = ResultSetFactory.resultSetType.keys.toArray
-
-  override def getResultSetByType(resultSetType: String): ResultSet[_ <: MetaData, _ <: Record] = {
-    if (!resultClasses.contains(resultSetType)) {
-      throw new StorageErrorException(
-        UNSUPPORTED_RESULT.getErrorCode,
-        MessageFormat.format(UNSUPPORTED_RESULT.getErrorDesc, resultSetType)
-      )
-    }
-    resultClasses(resultSetType).newInstance()
-  }
-
-  override def getResultSetByPath(fsPath: FsPath): ResultSet[_ <: MetaData, _ <: Record] = {
-    getResultSetByPath(fsPath, StorageUtils.getJvmUser)
-  }
-
-  override def getResultSetByContent(content: String): ResultSet[_ <: MetaData, _ <: Record] = {
-    getResultSetByType(Dolphin.getType(content))
-  }
-
-  override def exists(resultSetType: String): Boolean = resultClasses.contains(resultSetType)
-
-  override def isResultSetPath(path: String): Boolean = {
-    path.endsWith(Dolphin.DOLPHIN_FILE_SUFFIX)
-  }
-
-  override def isResultSet(content: String): Boolean =
-    Utils.tryCatch(resultClasses.contains(Dolphin.getType(content))) { t =>
-      logger.info("Wrong result Set: " + t.getMessage)
-      false
-    }
-
-  override def getResultSet(output: String): ResultSet[_ <: MetaData, _ <: Record] =
-    getResultSet(output, StorageUtils.getJvmUser)
-
-  override def getResultSetType: Array[String] = resultTypes
-
-  override def getResultSetByPath(fsPath: FsPath, fs: Fs): ResultSet[_ <: MetaData, _ <: Record] = {
-    val inputStream = fs.read(fsPath)
-    val resultSetType = Dolphin.getType(inputStream)
-    if (StringUtils.isEmpty(resultSetType)) {
-      throw new StorageWarnException(
-        THE_FILE_IS_EMPTY.getErrorCode,
-        MessageFormat.format(THE_FILE_IS_EMPTY.getErrorDesc, fsPath.getPath)
-      )
-    }
-    Utils.tryQuietly(inputStream.close())
-    // Utils.tryQuietly(fs.close())
-    getResultSetByType(resultSetType)
-  }
-
-  override def getResultSetByPath(
-      fsPath: FsPath,
-      proxyUser: String
-  ): ResultSet[_ <: MetaData, _ <: Record] = {
-    if (fsPath == null) return null
-    logger.info("Get Result Set By Path:" + fsPath.getPath)
-    val fs = FSFactory.getFsByProxyUser(fsPath, proxyUser)
-    fs.init(new util.HashMap[String, String]())
-    val inputStream = fs.read(fsPath)
-    val resultSetType = Dolphin.getType(inputStream)
-    if (StringUtils.isEmpty(resultSetType)) {
-      throw new StorageWarnException(
-        THE_FILE_IS_EMPTY.getErrorCode,
-        MessageFormat.format(THE_FILE_IS_EMPTY.getErrorDesc, fsPath.getPath)
-      )
-    }
-    Utils.tryQuietly(inputStream.close())
-    Utils.tryQuietly(fs.close())
-    getResultSetByType(resultSetType)
-  }
-
-  override def getResultSet(
-      output: String,
-      proxyUser: String
-  ): ResultSet[_ <: MetaData, _ <: Record] = {
-    if (isResultSetPath(output)) {
-      getResultSetByPath(new FsPath(output), proxyUser)
-    } else if (isResultSet(output)) {
-      getResultSetByContent(output)
-    } else null
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultSetFactory.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultSetFactory.scala
deleted file mode 100644
index e6615873b..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultSetFactory.scala
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset
-
-import org.apache.linkis.common.io.{Fs, FsPath, MetaData, Record}
-import org.apache.linkis.common.io.resultset.ResultSet
-
-import scala.collection.mutable
-
-trait ResultSetFactory extends scala.AnyRef {
-
-  def getResultSetByType(resultSetType: scala.Predef.String): ResultSet[_ <: MetaData, _ <: Record]
-
-  def getResultSetByPath(fsPath: FsPath): ResultSet[_ <: MetaData, _ <: Record]
-  def getResultSetByPath(fsPath: FsPath, fs: Fs): ResultSet[_ <: MetaData, _ <: Record]
-  def getResultSetByContent(content: scala.Predef.String): ResultSet[_ <: MetaData, _ <: Record]
-  def exists(resultSetType: scala.Predef.String): scala.Boolean
-  def isResultSetPath(path: scala.Predef.String): scala.Boolean
-  def isResultSet(content: scala.Predef.String): scala.Boolean
-  def getResultSet(output: String): ResultSet[_ <: MetaData, _ <: Record]
-
-  def getResultSetByPath(fsPath: FsPath, proxyUser: String): ResultSet[_ <: MetaData, _ <: Record]
-
-  def getResultSet(output: String, proxyUser: String): ResultSet[_ <: MetaData, _ <: Record]
-
-  /**
-   * The first must-time text(第一个必须时text)
-   * @return
-   */
-  def getResultSetType: Array[String]
-}
-
-object ResultSetFactory {
-
-  val TEXT_TYPE = "1"
-  val TABLE_TYPE = "2"
-  val IO_TYPE = "3"
-  val PICTURE_TYPE = "4"
-  val HTML_TYPE = "5"
-
-  /**
-   * TODO 修改为注册形式,并修改ResultSet的getResultType逻辑 Result set corresponding type record(结果集对应类型记录)
-   */
-  val resultSetType = mutable.LinkedHashMap[String, String](
-    TEXT_TYPE -> "TEXT",
-    TABLE_TYPE -> "TABLE",
-    IO_TYPE -> "IO",
-    PICTURE_TYPE -> "PICTURE",
-    HTML_TYPE -> "HTML"
-  )
-
-  val factory = new DefaultResultSetFactory
-  def getInstance: ResultSetFactory = factory
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultSetReader.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultSetReader.scala
deleted file mode 100644
index e61cf36b3..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultSetReader.scala
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset
-
-import org.apache.linkis.common.io.{FsPath, MetaData, Record}
-import org.apache.linkis.common.io.resultset.{ResultSet, ResultSetReader}
-import org.apache.linkis.common.utils.Logging
-import org.apache.linkis.storage.FSFactory
-import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.TABLE_ARE_NOT_SUPPORTED
-import org.apache.linkis.storage.exception.StorageErrorException
-import org.apache.linkis.storage.resultset.table.{TableMetaData, TableRecord, TableResultSet}
-
-import java.io.InputStream
-
-object ResultSetReader extends Logging {
-
-  def getResultSetReader[K <: MetaData, V <: Record](
-      resultSet: ResultSet[K, V],
-      inputStream: InputStream
-  ): ResultSetReader[K, V] = {
-    new StorageResultSetReader[K, V](resultSet, inputStream)
-  }
-
-  def getResultSetReader[K <: MetaData, V <: Record](
-      resultSet: ResultSet[K, V],
-      value: String
-  ): ResultSetReader[K, V] = {
-    new StorageResultSetReader[K, V](resultSet, value)
-  }
-
-  def getResultSetReader(res: String): ResultSetReader[_ <: MetaData, _ <: Record] = {
-    val rsFactory = ResultSetFactory.getInstance
-    if (rsFactory.isResultSet(res)) {
-      val resultSet = rsFactory.getResultSet(res)
-      ResultSetReader.getResultSetReader(resultSet, res)
-    } else {
-      val resPath = new FsPath(res)
-      val resultSet = rsFactory.getResultSetByPath(resPath)
-      val fs = FSFactory.getFs(resPath)
-      fs.init(null)
-      val reader = ResultSetReader.getResultSetReader(resultSet, fs.read(resPath))
-      reader match {
-        case storageResultSetReader: StorageResultSetReader[_, _] =>
-          storageResultSetReader.setFs(fs)
-        case _ =>
-      }
-      reader
-    }
-  }
-
-  def getTableResultReader(res: String): ResultSetReader[TableMetaData, TableRecord] = {
-    val rsFactory = ResultSetFactory.getInstance
-    if (rsFactory.isResultSet(res)) {
-      val resultSet = rsFactory.getResultSet(res)
-      if (ResultSetFactory.TABLE_TYPE != resultSet.resultSetType()) {
-        throw new StorageErrorException(
-          TABLE_ARE_NOT_SUPPORTED.getErrorCode,
-          TABLE_ARE_NOT_SUPPORTED.getErrorDesc
-        )
-      }
-      ResultSetReader.getResultSetReader(resultSet.asInstanceOf[TableResultSet], res)
-    } else {
-      val resPath = new FsPath(res)
-      val resultSet = rsFactory.getResultSetByPath(resPath)
-      if (ResultSetFactory.TABLE_TYPE != resultSet.resultSetType()) {
-        throw new StorageErrorException(
-          TABLE_ARE_NOT_SUPPORTED.getErrorCode,
-          TABLE_ARE_NOT_SUPPORTED.getErrorDesc
-        )
-      }
-      val fs = FSFactory.getFs(resPath)
-      logger.info("Try to init Fs with path:" + resPath.getPath)
-      fs.init(null)
-      ResultSetReader.getResultSetReader(resultSet.asInstanceOf[TableResultSet], fs.read(resPath))
-    }
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultSetWriter.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultSetWriter.scala
deleted file mode 100644
index cea00bbd6..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/ResultSetWriter.scala
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset
-
-import org.apache.linkis.common.io.{FsPath, MetaData, Record}
-import org.apache.linkis.common.io.resultset.{ResultSet, ResultSetWriter}
-
-import scala.collection.mutable.ArrayBuffer
-
-object ResultSetWriter {
-
-  def getResultSetWriter[K <: MetaData, V <: Record](
-      resultSet: ResultSet[K, V],
-      maxCacheSize: Long,
-      storePath: FsPath
-  ): ResultSetWriter[K, V] =
-    new StorageResultSetWriter[K, V](resultSet, maxCacheSize, storePath)
-
-  def getResultSetWriter[K <: MetaData, V <: Record](
-      resultSet: ResultSet[K, V],
-      maxCacheSize: Long,
-      storePath: FsPath,
-      proxyUser: String
-  ): ResultSetWriter[K, V] = {
-    val writer = new StorageResultSetWriter[K, V](resultSet, maxCacheSize, storePath)
-    writer.setProxyUser(proxyUser)
-    writer
-  }
-
-  def getRecordByWriter(
-      writer: ResultSetWriter[_ <: MetaData, _ <: Record],
-      limit: Long
-  ): Array[Record] = {
-    val res = writer.toString
-    getRecordByRes(res, limit)
-  }
-
-  def getRecordByRes(res: String, limit: Long): Array[Record] = {
-    val reader = ResultSetReader.getResultSetReader(res)
-    var count = 0
-    val records = new ArrayBuffer[Record]()
-    reader.getMetaData
-    while (reader.hasNext && count < limit) {
-      records += reader.getRecord
-      count = count + 1
-    }
-    records.toArray
-  }
-
-  def getLastRecordByRes(res: String): Record = {
-    val reader = ResultSetReader.getResultSetReader(res)
-    reader.getMetaData
-    while (reader.hasNext) {
-      reader.getRecord
-    }
-    reader.getRecord
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/StorageResultSet.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/StorageResultSet.scala
deleted file mode 100644
index aa23f2ac0..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/StorageResultSet.scala
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset
-
-import org.apache.linkis.common.io.{FsPath, MetaData, Record}
-import org.apache.linkis.common.io.resultset.ResultSet
-import org.apache.linkis.common.utils.{Logging, Utils}
-import org.apache.linkis.storage.domain.Dolphin
-import org.apache.linkis.storage.utils.StorageConfiguration
-
-abstract class StorageResultSet[K <: MetaData, V <: Record] extends ResultSet[K, V] with Logging {
-
-  val resultHeaderBytes = Dolphin.MAGIC_BYTES ++ Dolphin.getIntBytes(resultSetType().toInt)
-  override val charset: String = StorageConfiguration.STORAGE_RS_FILE_TYPE.getValue
-
-  override def getResultSetPath(parentDir: FsPath, fileName: String): FsPath = {
-    val path = if (parentDir.getPath.endsWith("/")) {
-      parentDir.getUriString + fileName + Dolphin.DOLPHIN_FILE_SUFFIX
-    } else {
-      parentDir.getUriString + "/" + fileName + Dolphin.DOLPHIN_FILE_SUFFIX
-    }
-    logger.info(s"Get result set path:${path}")
-    new FsPath(path)
-  }
-
-  override def getResultSetHeader: Array[Byte] = resultHeaderBytes
-
-  override def belongToPath(path: String): Boolean = path.endsWith(Dolphin.DOLPHIN_FILE_SUFFIX)
-
-  override def belongToResultSet(content: String): Boolean =
-    Utils.tryCatch(Dolphin.getType(content) == resultSetType()) { t =>
-      logger.info("Wrong result Set: ", t)
-      false
-    }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/StorageResultSetReader.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/StorageResultSetReader.scala
deleted file mode 100644
index 3f49faf3e..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/StorageResultSetReader.scala
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset
-
-import org.apache.linkis.common.io.{Fs, MetaData, Record}
-import org.apache.linkis.common.io.resultset.{ResultSet, ResultSetReader}
-import org.apache.linkis.common.utils.{Logging, Utils}
-import org.apache.linkis.storage.domain.Dolphin
-import org.apache.linkis.storage.exception.StorageWarnException
-import org.apache.linkis.storage.utils.StorageUtils
-
-import java.io.{ByteArrayInputStream, InputStream, IOException}
-
-import scala.collection.mutable.ArrayBuffer
-
-class StorageResultSetReader[K <: MetaData, V <: Record](
-    resultSet: ResultSet[K, V],
-    inputStream: InputStream
-) extends ResultSetReader[K, V](resultSet, inputStream)
-    with Logging {
-
-  private val deserializer = resultSet.createResultSetDeserializer
-  private var metaData: K = _
-  private var row: Record = _
-  private var colCount = 0
-  private var rowCount = 0
-
-  private var fs: Fs = _
-
-  private val READ_CACHE = 1024
-  private val bytes = new Array[Byte](READ_CACHE)
-
-  def this(resultSet: ResultSet[K, V], value: String) = {
-    this(resultSet, new ByteArrayInputStream(value.getBytes(Dolphin.CHAR_SET)))
-  }
-
-  def init(): Unit = {
-    val resType = Dolphin.getType(inputStream)
-    if (resultSet.resultSetType != resType) {
-      throw new IOException(
-        "File type does not match(文件类型不匹配): " + ResultSetFactory.resultSetType
-          .getOrElse(resType, "TABLE")
-      )
-    }
-  }
-
-  /**
-   * Read a row of data Read the line length first Get the entire row of data by the length of the
-   * line, first obtain the column length in the entire row of data, and then divide into column
-   * length to split the data 读取一行数据 先读取行长度 通过行长度获取整行数据,在整行数据中先获取列长度,进而分割成列长度从而分割数据
-   * @return
-   */
-  def readLine(): Array[Byte] = {
-
-    var rowLen = 0
-    try rowLen = Dolphin.readInt(inputStream)
-    catch {
-      case _: StorageWarnException => logger.info(s"Read finished(读取完毕)"); return null
-      case t: Throwable => throw t
-    }
-
-    val rowBuffer = ArrayBuffer[Byte]()
-    var len = 0
-
-    // Read the entire line, except for the data of the line length(读取整行,除了行长的数据)
-    while (rowLen > 0 && len >= 0) {
-      if (rowLen > READ_CACHE) {
-        len = StorageUtils.readBytes(inputStream, bytes, READ_CACHE)
-      } else {
-        len = StorageUtils.readBytes(inputStream, bytes, rowLen)
-      }
-
-      if (len > 0) {
-        rowLen -= len
-        rowBuffer ++= bytes.slice(0, len)
-      }
-    }
-    rowCount = rowCount + 1
-    rowBuffer.toArray
-  }
-
-  @scala.throws[IOException]
-  override def getRecord: Record = {
-    if (metaData == null) throw new IOException("Must read metadata first(必须先读取metadata)")
-    if (row == null) {
-      throw new IOException(
-        "Can't get the value of the field, maybe the IO stream has been read or has been closed!(拿不到字段的值,也许IO流已读取完毕或已被关闭!)"
-      )
-    }
-    row
-  }
-
-  def setFs(fs: Fs): Unit = this.fs = fs
-  def getFs: Fs = this.fs
-
-  @scala.throws[IOException]
-  override def getMetaData: MetaData = {
-    if (metaData == null) init()
-    metaData = deserializer.createMetaData(readLine())
-    metaData
-  }
-
-  @scala.throws[IOException]
-  override def skip(recordNum: Int): Int = {
-    if (recordNum < 0) return -1
-
-    if (metaData == null) getMetaData
-    for (i <- recordNum until (0, -1)) {
-      try inputStream.skip(Dolphin.readInt(inputStream))
-      catch {
-        case t: Throwable =>
-          return recordNum - i
-      }
-    }
-    recordNum
-  }
-
-  @scala.throws[IOException]
-  override def getPosition: Long = rowCount
-
-  @scala.throws[IOException]
-  override def hasNext: Boolean = {
-    if (metaData == null) getMetaData
-    val line = readLine()
-    if (line == null) return false
-    row = deserializer.createRecord(line)
-    if (row == null) return false
-    true
-  }
-
-  @scala.throws[IOException]
-  override def available: Long = inputStream.available()
-
-  override def close(): Unit = {
-    inputStream.close()
-    if (this.fs != null) Utils.tryQuietly(this.fs.close())
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/StorageResultSetWriter.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/StorageResultSetWriter.scala
deleted file mode 100644
index de849078b..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/StorageResultSetWriter.scala
+++ /dev/null
@@ -1,222 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset
-
-import org.apache.linkis.common.io.{Fs, FsPath, MetaData, Record}
-import org.apache.linkis.common.io.resultset.{ResultSerializer, ResultSet, ResultSetWriter}
-import org.apache.linkis.common.utils.{Logging, Utils}
-import org.apache.linkis.storage.FSFactory
-import org.apache.linkis.storage.conf.LinkisStorageConf
-import org.apache.linkis.storage.domain.Dolphin
-import org.apache.linkis.storage.utils.{FileSystemUtils, StorageUtils}
-
-import org.apache.commons.io.IOUtils
-import org.apache.hadoop.hdfs.client.HdfsDataOutputStream
-
-import java.io.{IOException, OutputStream}
-
-import scala.collection.mutable.ArrayBuffer
-
-class StorageResultSetWriter[K <: MetaData, V <: Record](
-    resultSet: ResultSet[K, V],
-    maxCacheSize: Long,
-    storePath: FsPath
-) extends ResultSetWriter[K, V](
-      resultSet = resultSet,
-      maxCacheSize = maxCacheSize,
-      storePath = storePath
-    )
-    with Logging {
-
-  private val serializer: ResultSerializer = resultSet.createResultSetSerializer
-
-  private var moveToWriteRow = false
-
-  private var outputStream: OutputStream = _
-
-  private var rowCount = 0
-
-  private val buffer = new ArrayBuffer[Byte]()
-
-  private var fs: Fs = _
-
-  private var rMetaData: MetaData = _
-
-  private var proxyUser: String = StorageUtils.getJvmUser
-
-  private var fileCreated = false
-
-  private var closed = false
-
-  private val WRITER_LOCK_CREATE = new Object()
-
-  private val WRITER_LOCK_CLOSE = new Object()
-
-  def getMetaData: MetaData = rMetaData
-
-  def setProxyUser(proxyUser: String): Unit = {
-    this.proxyUser = proxyUser
-  }
-
-  def isEmpty: Boolean = {
-    rMetaData == null && buffer.length <= Dolphin.FILE_EMPTY
-  }
-
-  def init(): Unit = {
-    writeLine(resultSet.getResultSetHeader, true)
-  }
-
-  def createNewFile: Unit = {
-    if (!fileCreated) {
-      WRITER_LOCK_CREATE.synchronized {
-        if (!fileCreated) {
-          if (storePath != null && outputStream == null) {
-            logger.info(s"Try to create a new file:${storePath}, with proxy user:${proxyUser}")
-            fs = FSFactory.getFsByProxyUser(storePath, proxyUser)
-            fs.init(null)
-            FileSystemUtils.createNewFile(storePath, proxyUser, true)
-            outputStream = fs.write(storePath, true)
-            logger.info(s"Succeed to create a new file:$storePath")
-            fileCreated = true
-          }
-        }
-      }
-    } else if (null != storePath && null == outputStream) {
-      logger.warn("outputStream had been set null, but createNewFile() was called again.")
-    }
-  }
-
-  def writeLine(bytes: Array[Byte], cache: Boolean = false): Unit = {
-    if (closed) {
-      logger.warn("the writer had been closed, but writeLine() was still called.")
-      return
-    }
-    if (bytes.length > LinkisStorageConf.ROW_BYTE_MAX_LEN) {
-      throw new IOException(
-        s"A single row of data cannot exceed ${LinkisStorageConf.ROW_BYTE_MAX_LEN_STR}"
-      )
-    }
-    if (buffer.length > maxCacheSize && !cache) {
-      if (outputStream == null) {
-        createNewFile
-      }
-      flush()
-      outputStream.write(bytes)
-    } else {
-      buffer.appendAll(bytes)
-    }
-  }
-
-  override def toString: String = {
-    if (outputStream == null) {
-      if (isEmpty) return ""
-      new String(buffer.toArray, Dolphin.CHAR_SET)
-    } else {
-      storePath.getSchemaPath
-    }
-  }
-
-  override def toFSPath: FsPath = storePath
-
-  override def addMetaDataAndRecordString(content: String): Unit = {
-    if (!moveToWriteRow) {
-      val bytes = content.getBytes(Dolphin.CHAR_SET)
-      writeLine(bytes)
-    }
-    moveToWriteRow = true
-  }
-
-  override def addRecordString(content: String): Unit = {}
-
-  @scala.throws[IOException]
-  override def addMetaData(metaData: MetaData): Unit = {
-    if (!moveToWriteRow) {
-      {
-        rMetaData = metaData
-        init()
-        if (null == metaData) {
-          writeLine(serializer.metaDataToBytes(metaData), true)
-        } else {
-          writeLine(serializer.metaDataToBytes(metaData))
-        }
-      }
-      moveToWriteRow = true
-    }
-  }
-
-  @scala.throws[IOException]
-  override def addRecord(record: Record): Unit = {
-    if (moveToWriteRow) {
-      rowCount = rowCount + 1
-      writeLine(serializer.recordToBytes(record))
-    }
-  }
-
-  def closeFs: Unit = {
-    if (fs != null) {
-      IOUtils.closeQuietly(fs)
-      fs = null
-    }
-  }
-
-  override def close(): Unit = {
-    if (closed) {
-      logger.warn("the writer had been closed, but close() was still called.")
-      return
-    } else {
-      WRITER_LOCK_CLOSE.synchronized {
-        if (!closed) {
-          closed = true
-        } else {
-          return
-        }
-      }
-    }
-    Utils.tryFinally(if (outputStream != null) flush()) {
-      if (outputStream != null) {
-        IOUtils.closeQuietly(outputStream)
-        outputStream = null
-      }
-      closeFs
-    }
-  }
-
-  override def flush(): Unit = {
-    createNewFile
-    if (outputStream != null) {
-      if (buffer.nonEmpty) {
-        outputStream.write(buffer.toArray)
-        buffer.clear()
-      }
-      Utils.tryAndWarnMsg[Unit] {
-        outputStream match {
-          case hdfs: HdfsDataOutputStream =>
-            hdfs.hflush()
-          case _ =>
-            outputStream.flush()
-        }
-      }(s"Error encounters when flush result set ")
-    }
-    if (closed) {
-      if (logger.isDebugEnabled()) {
-        logger.debug("the writer had been closed, but flush() was still called.")
-      }
-    }
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/html/HtmlResultSet.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/html/HtmlResultSet.scala
deleted file mode 100644
index 7a65a9b9d..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/html/HtmlResultSet.scala
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset.html
-
-import org.apache.linkis.common.io.resultset.{ResultDeserializer, ResultSerializer}
-import org.apache.linkis.storage.{LineMetaData, LineRecord}
-import org.apache.linkis.storage.resultset.{ResultSetFactory, StorageResultSet}
-import org.apache.linkis.storage.resultset.txt.{TextResultDeserializer, TextResultSerializer}
-
-class HtmlResultSet extends StorageResultSet[LineMetaData, LineRecord] with Serializable {
-
-  override def resultSetType(): String = ResultSetFactory.HTML_TYPE
-
-  override def createResultSetSerializer(): ResultSerializer = new TextResultSerializer
-
-  override def createResultSetDeserializer(): ResultDeserializer[LineMetaData, LineRecord] =
-    new TextResultDeserializer
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultSerializer.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultSerializer.scala
deleted file mode 100644
index be527e396..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultSerializer.scala
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset.io
-
-import org.apache.linkis.common.io.{MetaData, Record}
-import org.apache.linkis.common.io.resultset.ResultSerializer
-import org.apache.linkis.storage.domain.Dolphin
-
-import org.apache.commons.codec.binary.Base64
-
-class IOResultSerializer extends ResultSerializer {
-
-  override def metaDataToBytes(metaData: MetaData): Array[Byte] = {
-    val ioMetaData = metaData.asInstanceOf[IOMetaData]
-    lineToBytes(s"${ioMetaData.off}${Dolphin.COL_SPLIT}${ioMetaData.len}")
-  }
-
-  override def recordToBytes(record: Record): Array[Byte] = {
-    val ioRecord = record.asInstanceOf[IORecord]
-    lineToBytes(Base64.encodeBase64String(ioRecord.value))
-  }
-
-  def lineToBytes(value: String): Array[Byte] = {
-    val bytes = if (value == null) Dolphin.NULL_BYTES else Dolphin.getBytes(value)
-    Dolphin.getIntBytes(bytes.length) ++ bytes
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultSet.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultSet.scala
deleted file mode 100644
index 5fd952dbf..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/io/IOResultSet.scala
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset.io
-
-import org.apache.linkis.common.io.resultset.{ResultDeserializer, ResultSerializer}
-import org.apache.linkis.storage.resultset.{ResultSetFactory, StorageResultSet}
-
-class IOResultSet extends StorageResultSet[IOMetaData, IORecord] with Serializable {
-
-  override def resultSetType(): String = ResultSetFactory.IO_TYPE
-
-  override def createResultSetSerializer(): ResultSerializer = new IOResultSerializer
-
-  override def createResultSetDeserializer(): ResultDeserializer[IOMetaData, IORecord] =
-    new IOResultDeserializer
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/picture/PictureResultSet.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/picture/PictureResultSet.scala
deleted file mode 100644
index 95c1d3ad1..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/picture/PictureResultSet.scala
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset.picture
-
-import org.apache.linkis.common.io.resultset.{ResultDeserializer, ResultSerializer}
-import org.apache.linkis.storage.{LineMetaData, LineRecord}
-import org.apache.linkis.storage.resultset.{ResultSetFactory, StorageResultSet}
-import org.apache.linkis.storage.resultset.txt.{TextResultDeserializer, TextResultSerializer}
-
-class PictureResultSet extends StorageResultSet[LineMetaData, LineRecord] with Serializable {
-
-  override def resultSetType(): String = ResultSetFactory.PICTURE_TYPE
-
-  override def createResultSetSerializer(): ResultSerializer = new TextResultSerializer
-
-  override def createResultSetDeserializer(): ResultDeserializer[LineMetaData, LineRecord] =
-    new TextResultDeserializer
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableResultDeserializer.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableResultDeserializer.scala
deleted file mode 100644
index 40c4e031f..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableResultDeserializer.scala
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset.table
-
-import org.apache.linkis.common.io.resultset.ResultDeserializer
-import org.apache.linkis.storage.domain.{Column, DataType, Dolphin}
-import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary.PARSING_METADATA_FAILED
-import org.apache.linkis.storage.exception.StorageErrorException
-
-import scala.collection.mutable.ArrayBuffer
-
-class TableResultDeserializer extends ResultDeserializer[TableMetaData, TableRecord] {
-
-  var metaData: TableMetaData = _
-
-  import DataType._
-
-  override def createMetaData(bytes: Array[Byte]): TableMetaData = {
-    val colByteLen = Dolphin.getString(bytes, 0, Dolphin.INT_LEN).toInt
-    val colString = Dolphin.getString(bytes, Dolphin.INT_LEN, colByteLen)
-    val colArray =
-      if (colString.endsWith(Dolphin.COL_SPLIT)) {
-        colString.substring(0, colString.length - 1).split(Dolphin.COL_SPLIT)
-      } else colString.split(Dolphin.COL_SPLIT)
-    var index = Dolphin.INT_LEN + colByteLen
-    if (colArray.length % 3 != 0) {
-      throw new StorageErrorException(
-        PARSING_METADATA_FAILED.getErrorCode,
-        PARSING_METADATA_FAILED.getErrorDesc
-      )
-    }
-    val columns = new ArrayBuffer[Column]()
-    for (i <- 0 until (colArray.length, 3)) {
-      var len = colArray(i).toInt
-      val colName = Dolphin.getString(bytes, index, len)
-      index += len
-      len = colArray(i + 1).toInt
-      val colType = Dolphin.getString(bytes, index, len)
-      index += len
-      len = colArray(i + 2).toInt
-      val colComment = Dolphin.getString(bytes, index, len)
-      index += len
-      columns += Column(colName, colType, colComment)
-    }
-    metaData = new TableMetaData(columns.toArray)
-    metaData
-  }
-
-  /**
-   * colByteLen:All column fields are long(所有列字段长 记录的长度) colString:Obtain column
-   * length(获得列长):10,20,21 colArray:Column length array(列长数组) Get data by column length(通过列长获得数据)
-   * @param bytes
-   * @return
-   */
-  override def createRecord(bytes: Array[Byte]): TableRecord = {
-    val colByteLen = Dolphin.getString(bytes, 0, Dolphin.INT_LEN).toInt
-    val colString = Dolphin.getString(bytes, Dolphin.INT_LEN, colByteLen)
-    val colArray =
-      if (colString.endsWith(Dolphin.COL_SPLIT)) {
-        colString.substring(0, colString.length - 1).split(Dolphin.COL_SPLIT)
-      } else colString.split(Dolphin.COL_SPLIT)
-    var index = Dolphin.INT_LEN + colByteLen
-    val data = colArray.indices.map { i =>
-      val len = colArray(i).toInt
-      val res = Dolphin.getString(bytes, index, len)
-      index += len
-      if (i >= metaData.columns.length) res
-      else {
-        toValue(metaData.columns(i).dataType, res)
-      }
-    }.toArray
-    new TableRecord(data)
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableResultSerializer.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableResultSerializer.scala
deleted file mode 100644
index 94b6cb4c0..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableResultSerializer.scala
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset.table
-
-import org.apache.linkis.common.io.{MetaData, Record}
-import org.apache.linkis.common.io.resultset.ResultSerializer
-import org.apache.linkis.storage.domain.Dolphin
-
-import scala.collection.mutable.ArrayBuffer
-
-class TableResultSerializer extends ResultSerializer {
-
-  override def metaDataToBytes(metaData: MetaData): Array[Byte] = {
-    val tableMetaData = metaData.asInstanceOf[TableMetaData]
-    lineToBytes(tableMetaData.columns.map(_.toArray).reduce((a1, a2) => a1 ++ a2))
-  }
-
-  override def recordToBytes(record: Record): Array[Byte] = {
-    val tableRecord = record.asInstanceOf[TableRecord]
-    lineToBytes(tableRecord.row)
-  }
-
-  /**
-   * Convert a row of data to an array of Bytes Convert the data to byte and get the corresponding
-   * total byte length to write to the file Data write format: line length (fixed length) column
-   * length (fixed length) field index comma segmentation real data For example:
-   * 000000004900000000116,10,3,4,5,peace1johnnwang1101true11.51 The length of the line does not
-   * include its own length 将一行数据转换为Bytes的数组 对数据转换为byte,并获取相应的总byte长度写入文件 数据写入格式:行长(固定长度) 列长(固定长度)
-   * 字段索引逗号分割 真实数据 如:000000004900000000116,10,3,4,5,peace1johnnwang1101true11.51 其中行长不包括自身长度
-   * @param line
-   */
-  def lineToBytes(line: Array[Any]): Array[Byte] = {
-    // Data cache(数据缓存)
-    val dataBytes = ArrayBuffer[Array[Byte]]()
-    // Column cache(列缓存)
-    val colIndex = ArrayBuffer[Array[Byte]]()
-    var colByteLen = 0
-    var length = 0
-    line.foreach { data =>
-      val bytes = if (data == null) Dolphin.NULL_BYTES else Dolphin.getBytes(data)
-      dataBytes += bytes
-      val colBytes = Dolphin.getBytes(bytes.length)
-      colIndex += colBytes += Dolphin.COL_SPLIT_BYTES
-      colByteLen += colBytes.length + Dolphin.COL_SPLIT_LEN
-      length += bytes.length
-    }
-    length += colByteLen + Dolphin.INT_LEN
-    toByteArray(length, colByteLen, colIndex, dataBytes)
-  }
-
-  /**
-   * Splice a row of data into a byte array(将一行的数据拼接成byte数组)
-   * @param length
-   *   The total length of the line data byte, excluding its own length(行数据byte总长度,不包括自身的长度)
-   * @param colByteLen
-   *   Record field index byte column length(记录字段索引byte的列长)
-   * @param colIndex
-   *   Field index, including separator comma(字段索引,包括分割符逗号)
-   * @param dataBytes
-   *   Byte of real data(真实数据的byte)
-   * @return
-   */
-  def toByteArray(
-      length: Int,
-      colByteLen: Int,
-      colIndex: ArrayBuffer[Array[Byte]],
-      dataBytes: ArrayBuffer[Array[Byte]]
-  ): Array[Byte] = {
-    val row = ArrayBuffer[Byte]()
-    colIndex ++= dataBytes
-    row.appendAll(Dolphin.getIntBytes(length))
-    row.appendAll(Dolphin.getIntBytes(colByteLen))
-    colIndex.foreach(row.appendAll(_))
-    row.toArray
-  }
-
-}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableResultSet.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableResultSet.scala
deleted file mode 100644
index fe8c4e9cd..000000000
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/resultset/table/TableResultSet.scala
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.linkis.storage.resultset.table
-
... 3845 lines suppressed ...


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