You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ha...@apache.org on 2021/09/27 11:31:55 UTC

[iotdb] branch rel/0.12 updated: [To rel/0.12][IOTDB-842] Better Export/Import-CSV Tool (#4045)

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

haonan pushed a commit to branch rel/0.12
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/rel/0.12 by this push:
     new f9a08ac  [To rel/0.12][IOTDB-842] Better Export/Import-CSV Tool (#4045)
f9a08ac is described below

commit f9a08acbece5fc83e323b0b9e4a0ae5967b20b01
Author: Xuan Ronaldo <xu...@qq.com>
AuthorDate: Mon Sep 27 19:31:25 2021 +0800

    [To rel/0.12][IOTDB-842] Better Export/Import-CSV Tool (#4045)
---
 checkstyle.xml                                     |   2 +-
 cli/pom.xml                                        |   5 +
 .../java/org/apache/iotdb/cli/AbstractCli.java     |  14 +-
 .../org/apache/iotdb/tool/AbstractCsvTool.java     |  96 ++-
 .../main/java/org/apache/iotdb/tool/ExportCsv.java | 240 +++----
 .../main/java/org/apache/iotdb/tool/ImportCsv.java | 789 +++++++++++++--------
 .../java/org/apache/iotdb/cli/AbstractScript.java  |  10 +-
 .../org/apache/iotdb/tool/CsvLineSplitTest.java    |  33 -
 .../tool/{ => integration}/ExportCsvTestIT.java    |   2 +-
 .../tool/{ => integration}/ImportCsvTestIT.java    |   2 +-
 .../apache/iotdb/tool/unit/WriteCsvFileTestUT.java |  46 ++
 cross-tests/pom.xml                                |   9 +
 .../tests/tools/importCsv/AbstractScript.java      |  54 +-
 .../tests/tools/importCsv/ExportCsvTestIT.java     | 209 ++----
 .../tests/tools/importCsv/ImportCsvTestIT.java     | 383 ++++++----
 docs/UserGuide/System-Tools/CSV-Tool.md            | 204 ++++--
 docs/zh/UserGuide/System-Tools/CSV-Tool.md         | 199 ++++--
 pom.xml                                            |   2 +-
 .../iotdb/db/query/control/QueryTimeManager.java   |  25 +-
 19 files changed, 1382 insertions(+), 942 deletions(-)

diff --git a/checkstyle.xml b/checkstyle.xml
index 0a9d199..76fe6e1 100644
--- a/checkstyle.xml
+++ b/checkstyle.xml
@@ -227,4 +227,4 @@
         </module>
         <module name="CommentsIndentation"/>
     </module>
-</module>
+</module>
\ No newline at end of file
diff --git a/cli/pom.xml b/cli/pom.xml
index 2e1fd4e..37c3453 100644
--- a/cli/pom.xml
+++ b/cli/pom.xml
@@ -75,6 +75,11 @@
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-csv</artifactId>
+            <version>1.9.0</version>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/cli/src/main/java/org/apache/iotdb/cli/AbstractCli.java b/cli/src/main/java/org/apache/iotdb/cli/AbstractCli.java
index 467c234..4c286a5 100644
--- a/cli/src/main/java/org/apache/iotdb/cli/AbstractCli.java
+++ b/cli/src/main/java/org/apache/iotdb/cli/AbstractCli.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.cli;
 import org.apache.iotdb.exception.ArgsErrorException;
 import org.apache.iotdb.jdbc.IoTDBConnection;
 import org.apache.iotdb.jdbc.IoTDBJDBCResultSet;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.service.rpc.thrift.ServerProperties;
 import org.apache.iotdb.tool.ImportCsv;
@@ -478,8 +479,17 @@ public abstract class AbstractCli {
       return;
     }
     println(cmd.split(" ")[1]);
-    ImportCsv.importCsvFromFile(
-        host, port, username, password, cmd.split(" ")[1], connection.getTimeZone());
+    try {
+      ImportCsv.importFromTargetPath(
+          host,
+          Integer.valueOf(port),
+          username,
+          password,
+          cmd.split(" ")[1],
+          connection.getTimeZone());
+    } catch (IoTDBConnectionException e) {
+      e.printStackTrace();
+    }
   }
 
   private static void executeQuery(IoTDBConnection connection, String cmd) {
diff --git a/cli/src/main/java/org/apache/iotdb/tool/AbstractCsvTool.java b/cli/src/main/java/org/apache/iotdb/tool/AbstractCsvTool.java
index 90764b5..f03275d 100644
--- a/cli/src/main/java/org/apache/iotdb/tool/AbstractCsvTool.java
+++ b/cli/src/main/java/org/apache/iotdb/tool/AbstractCsvTool.java
@@ -23,13 +23,18 @@ import org.apache.iotdb.rpc.IoTDBConnectionException;
 import org.apache.iotdb.rpc.StatementExecutionException;
 import org.apache.iotdb.session.Session;
 
-import jline.console.ConsoleReader;
+import jline.internal.Nullable;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVPrinter;
+import org.apache.commons.csv.QuoteMode;
 
 import java.io.IOException;
+import java.io.PrintWriter;
 import java.time.ZoneId;
+import java.util.List;
 
 public abstract class AbstractCsvTool {
 
@@ -55,33 +60,44 @@ public abstract class AbstractCsvTool {
   protected static final int MAX_HELP_CONSOLE_WIDTH = 92;
   protected static final String[] TIME_FORMAT =
       new String[] {"default", "long", "number", "timestamp"};
-  protected static final String[] STRING_TIME_FORMAT =
+  public static final String[] STRING_TIME_FORMAT =
       new String[] {
-        "yyyy-MM-dd'T'HH:mm:ss.SSSZ",
-        "yyyy/MM/dd HH:mm:ss.SSS",
+        "yyyy-MM-dd HH:mm:ss.SSSX",
+        "yyyy/MM/dd HH:mm:ss.SSSX",
+        "yyyy.MM.dd HH:mm:ss.SSSX",
+        "yyyy-MM-dd HH:mm:ssX",
+        "yyyy/MM/dd HH:mm:ssX",
+        "yyyy.MM.dd HH:mm:ssX",
+        "yyyy-MM-dd HH:mm:ss.SSSz",
+        "yyyy/MM/dd HH:mm:ss.SSSz",
+        "yyyy.MM.dd HH:mm:ss.SSSz",
+        "yyyy-MM-dd HH:mm:ssz",
+        "yyyy/MM/dd HH:mm:ssz",
+        "yyyy.MM.dd HH:mm:ssz",
         "yyyy-MM-dd HH:mm:ss.SSS",
+        "yyyy/MM/dd HH:mm:ss.SSS",
         "yyyy.MM.dd HH:mm:ss.SSS",
-        "yyyy/MM/dd'T'HH:mm:ss.SSS",
-        "yyyy-MM-dd'T'HH:mm:ss.SSS",
-        "yyyy-MM-dd'T'HH:mm:ss.SSS",
-        "yyyy.MM.dd'T'HH:mm:ss.SSS",
-        "yyyy-MM-dd HH:mm:ss.SSSZZ",
-        "yyyy/MM/dd HH:mm:ss.SSSZZ",
-        "yyyy.MM.dd HH:mm:ss.SSSZZ",
-        "yyyy-MM-dd'T'HH:mm:ss.SSSZZ",
-        "yyyy/MM/dd'T'HH:mm:ss.SSSZZ",
         "yyyy-MM-dd HH:mm:ss",
         "yyyy/MM/dd HH:mm:ss",
         "yyyy.MM.dd HH:mm:ss",
+        "yyyy-MM-dd'T'HH:mm:ss.SSSX",
+        "yyyy/MM/dd'T'HH:mm:ss.SSSX",
+        "yyyy.MM.dd'T'HH:mm:ss.SSSX",
+        "yyyy-MM-dd'T'HH:mm:ssX",
+        "yyyy/MM/dd'T'HH:mm:ssX",
+        "yyyy.MM.dd'T'HH:mm:ssX",
+        "yyyy-MM-dd'T'HH:mm:ss.SSSz",
+        "yyyy/MM/dd'T'HH:mm:ss.SSSz",
+        "yyyy.MM.dd'T'HH:mm:ss.SSSz",
+        "yyyy-MM-dd'T'HH:mm:ssz",
+        "yyyy/MM/dd'T'HH:mm:ssz",
+        "yyyy.MM.dd'T'HH:mm:ssz",
+        "yyyy-MM-dd'T'HH:mm:ss.SSS",
+        "yyyy/MM/dd'T'HH:mm:ss.SSS",
+        "yyyy.MM.dd'T'HH:mm:ss.SSS",
         "yyyy-MM-dd'T'HH:mm:ss",
         "yyyy/MM/dd'T'HH:mm:ss",
-        "yyyy.MM.dd'T'HH:mm:ss",
-        "yyyy-MM-dd HH:mm:ssZZ",
-        "yyyy/MM/dd HH:mm:ssZZ",
-        "yyyy.MM.dd HH:mm:ssZZ",
-        "yyyy-MM-dd'T'HH:mm:ssZZ",
-        "yyyy/MM/dd'T'HH:mm:ssZZ",
-        "yyyy.MM.dd'T'HH:mm:ssZZ",
+        "yyyy.MM.dd'T'HH:mm:ss"
       };
   protected static String host;
   protected static String port;
@@ -93,7 +109,7 @@ public abstract class AbstractCsvTool {
   protected static String timeFormat;
   protected static Session session;
 
-  AbstractCsvTool() {}
+  public AbstractCsvTool() {}
 
   protected static String checkRequiredArg(String arg, String name, CommandLine commandLine)
       throws ArgsErrorException {
@@ -114,16 +130,13 @@ public abstract class AbstractCsvTool {
     zoneId = ZoneId.of(session.getTimeZone());
   }
 
-  protected static void parseBasicParams(CommandLine commandLine, ConsoleReader reader)
+  protected static void parseBasicParams(CommandLine commandLine)
       throws ArgsErrorException, IOException {
     host = checkRequiredArg(HOST_ARGS, HOST_NAME, commandLine);
     port = checkRequiredArg(PORT_ARGS, PORT_NAME, commandLine);
     username = checkRequiredArg(USERNAME_ARGS, USERNAME_NAME, commandLine);
 
     password = commandLine.getOptionValue(PASSWORD_ARGS);
-    if (password == null) {
-      password = reader.readLine("please input your password:", '\0');
-    }
   }
 
   protected static boolean checkTimeFormat() {
@@ -183,9 +196,40 @@ public abstract class AbstractCsvTool {
             .optionalArg(true)
             .argName(PASSWORD_NAME)
             .hasArg()
-            .desc("Password (optional)")
+            .desc("Password (required)")
             .build();
     options.addOption(opPassword);
     return options;
   }
+
+  /**
+   * write data to CSV file.
+   *
+   * @param headerNames the header names of CSV file
+   * @param records the records of CSV file
+   * @param filePath the directory to save the file
+   */
+  public static Boolean writeCsvFile(
+      @Nullable List<String> headerNames, List<List<Object>> records, String filePath) {
+    try {
+      CSVPrinter printer =
+          CSVFormat.DEFAULT
+              .withFirstRecordAsHeader()
+              .withEscape('\\')
+              .withQuoteMode(QuoteMode.NONE)
+              .print(new PrintWriter(filePath));
+      if (headerNames != null) {
+        printer.printRecord(headerNames);
+      }
+      for (List record : records) {
+        printer.printRecord(record);
+      }
+      printer.flush();
+      printer.close();
+      return true;
+    } catch (IOException e) {
+      e.printStackTrace();
+      return false;
+    }
+  }
 }
diff --git a/cli/src/main/java/org/apache/iotdb/tool/ExportCsv.java b/cli/src/main/java/org/apache/iotdb/tool/ExportCsv.java
index 5b1d9ca..a6e5c86 100644
--- a/cli/src/main/java/org/apache/iotdb/tool/ExportCsv.java
+++ b/cli/src/main/java/org/apache/iotdb/tool/ExportCsv.java
@@ -26,7 +26,6 @@ import org.apache.iotdb.rpc.StatementExecutionException;
 import org.apache.iotdb.session.Session;
 import org.apache.iotdb.session.SessionDataSet;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.common.Field;
 import org.apache.iotdb.tsfile.read.common.RowRecord;
 
 import jline.console.ConsoleReader;
@@ -39,14 +38,13 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 
 import java.io.BufferedReader;
-import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileReader;
-import java.io.FileWriter;
 import java.io.IOException;
 import java.time.Instant;
 import java.time.ZonedDateTime;
 import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -65,6 +63,12 @@ public class ExportCsv extends AbstractCsvTool {
   private static final String SQL_FILE_ARGS = "s";
   private static final String SQL_FILE_NAME = "sqlfile";
 
+  private static final String DATA_TYPE_ARGS = "datatype";
+  private static final String DATA_TYPE_NAME = "datatype";
+
+  private static final String QUERY_COMMAND_ARGS = "q";
+  private static final String QUERY_COMMAND_NAME = "queryCommand";
+
   private static final String TSFILEDB_CLI_PREFIX = "ExportCsv";
 
   private static final String DUMP_FILE_NAME_DEFAULT = "dump";
@@ -72,6 +76,10 @@ public class ExportCsv extends AbstractCsvTool {
 
   private static String targetDirectory;
 
+  private static Boolean needDataTypePrinted;
+
+  private static String queryCommand;
+
   private static final int EXPORT_PER_LINE_COUNT = 10000;
 
   /** main function of export csv tool. */
@@ -100,31 +108,38 @@ public class ExportCsv extends AbstractCsvTool {
       return;
     }
 
-    ConsoleReader reader = new ConsoleReader();
-    reader.setExpandEvents(false);
-
     try {
-      parseBasicParams(commandLine, reader);
+      parseBasicParams(commandLine);
       parseSpecialParams(commandLine);
       if (!checkTimeFormat()) {
         return;
       }
 
-      String sqlFile = commandLine.getOptionValue(SQL_FILE_ARGS);
-      String sql;
       session = new Session(host, Integer.parseInt(port), username, password);
       session.open(false);
       setTimeZone();
 
-      if (sqlFile == null) {
-        sql = reader.readLine(TSFILEDB_CLI_PREFIX + "> please input query: ");
-        String[] values = sql.trim().split(";");
-        for (int i = 0; i < values.length; i++) {
-          dumpResult(values[i], i);
+      if (queryCommand == null) {
+        String sqlFile = commandLine.getOptionValue(SQL_FILE_ARGS);
+        String sql;
+
+        if (sqlFile == null) {
+          ConsoleReader reader = new ConsoleReader();
+          reader.setExpandEvents(false);
+          sql = reader.readLine(TSFILEDB_CLI_PREFIX + "> please input query: ");
+          System.out.println(sql);
+          String[] values = sql.trim().split(";");
+          for (int i = 0; i < values.length; i++) {
+            dumpResult(values[i], i);
+          }
+          reader.close();
+        } else {
+          dumpFromSqlFile(sqlFile);
         }
       } else {
-        dumpFromSqlFile(sqlFile);
+        dumpResult(queryCommand, 0);
       }
+
     } catch (IOException e) {
       System.out.println("Failed to operate on file, because " + e.getMessage());
     } catch (ArgsErrorException e) {
@@ -132,7 +147,6 @@ public class ExportCsv extends AbstractCsvTool {
     } catch (IoTDBConnectionException | StatementExecutionException e) {
       System.out.println("Connect failed because " + e.getMessage());
     } finally {
-      reader.close();
       if (session != null) {
         try {
           session.close();
@@ -147,6 +161,12 @@ public class ExportCsv extends AbstractCsvTool {
   private static void parseSpecialParams(CommandLine commandLine) throws ArgsErrorException {
     targetDirectory = checkRequiredArg(TARGET_DIR_ARGS, TARGET_DIR_NAME, commandLine);
     targetFile = commandLine.getOptionValue(TARGET_FILE_ARGS);
+    needDataTypePrinted = Boolean.valueOf(commandLine.getOptionValue(DATA_TYPE_ARGS));
+    queryCommand = commandLine.getOptionValue(QUERY_COMMAND_ARGS);
+
+    if (needDataTypePrinted == null) {
+      needDataTypePrinted = true;
+    }
     if (targetFile == null) {
       targetFile = DUMP_FILE_NAME_DEFAULT;
     }
@@ -155,7 +175,7 @@ public class ExportCsv extends AbstractCsvTool {
       timeFormat = "default";
     }
     timeZoneID = commandLine.getOptionValue(TIME_ZONE_ARGS);
-    if (!targetDirectory.endsWith(File.separator)) {
+    if (!targetDirectory.endsWith("/") && !targetDirectory.endsWith("\\")) {
       targetDirectory += File.separator;
     }
   }
@@ -212,6 +232,25 @@ public class ExportCsv extends AbstractCsvTool {
             .build();
     options.addOption(opTimeZone);
 
+    Option opDataType =
+        Option.builder(DATA_TYPE_ARGS)
+            .argName(DATA_TYPE_NAME)
+            .hasArg()
+            .desc(
+                "Will the data type of timeseries be printed in the head line of the CSV file?"
+                    + '\n'
+                    + "You can choose true) or false) . (optional)")
+            .build();
+    options.addOption(opDataType);
+
+    Option opQuery =
+        Option.builder(QUERY_COMMAND_ARGS)
+            .argName(QUERY_COMMAND_NAME)
+            .hasArg()
+            .desc("The query command that you want to execute. (optional)")
+            .build();
+    options.addOption(opQuery);
+
     Option opHelp =
         Option.builder(HELP_ARGS)
             .longOpt(HELP_ARGS)
@@ -223,6 +262,12 @@ public class ExportCsv extends AbstractCsvTool {
     return options;
   }
 
+  /**
+   * This method will be called, if the query commands are written in a sql file.
+   *
+   * @param filePath
+   * @throws IOException
+   */
   private static void dumpFromSqlFile(String filePath) throws IOException {
     try (BufferedReader reader = new BufferedReader(new FileReader(filePath))) {
       String sql;
@@ -243,130 +288,81 @@ public class ExportCsv extends AbstractCsvTool {
   private static void dumpResult(String sql, int index) {
 
     final String path = targetDirectory + targetFile + index + ".csv";
-    File tf = new File(path);
     try {
-      if (!tf.exists() && !tf.createNewFile()) {
-        System.out.println("Could not create target file for sql statement: " + sql);
-        return;
-      }
-    } catch (IOException e) {
-      System.out.println("Cannot create dump file " + path + " " + "because: " + e.getMessage());
-      return;
-    }
-    System.out.println("Start to export data from sql statement: " + sql);
-    try (BufferedWriter bw = new BufferedWriter(new FileWriter(tf))) {
       SessionDataSet sessionDataSet = session.executeQueryStatement(sql);
-      long startTime = System.currentTimeMillis();
-      // write data in csv file
-      writeMetadata(bw, sessionDataSet.getColumnNames());
-
-      int line = writeResultSet(sessionDataSet, bw);
-      System.out.printf(
-          "Statement [%s] has dumped to file %s successfully! It costs "
-              + "%dms to export %d lines.%n",
-          sql, path, System.currentTimeMillis() - startTime, line);
-    } catch (IOException | StatementExecutionException | IoTDBConnectionException e) {
+      List<List<Object>> records = loadDataFromDataSet(sessionDataSet);
+      writeCsvFile(null, records, path);
+      System.out.println("Export completely!");
+    } catch (StatementExecutionException | IoTDBConnectionException e) {
       System.out.println("Cannot dump result because: " + e.getMessage());
     }
   }
 
-  private static void writeMetadata(BufferedWriter bw, List<String> columnNames)
-      throws IOException {
-    if (!columnNames.get(0).equals("Time")) {
-      bw.write("Time" + ",");
-    }
-    for (int i = 0; i < columnNames.size() - 1; i++) {
-      bw.write(columnNames.get(i) + ",");
+  /**
+   * Load data from the result of query command.
+   *
+   * @param sessionDataSet
+   * @return
+   * @throws IoTDBConnectionException
+   * @throws StatementExecutionException
+   */
+  public static List<List<Object>> loadDataFromDataSet(SessionDataSet sessionDataSet)
+      throws IoTDBConnectionException, StatementExecutionException {
+    List<List<Object>> records = new ArrayList<>();
+    List<Object> headers = new ArrayList<>();
+    List<String> names = sessionDataSet.getColumnNames();
+    List<String> types = sessionDataSet.getColumnTypes();
+
+    if (needDataTypePrinted == true) {
+      for (int i = 0; i < names.size(); i++) {
+        if (!names.get(i).equals("Time") && !names.get(i).equals("Device"))
+          headers.add(String.format("%s(%s)", names.get(i), types.get(i)));
+        else headers.add(names.get(i));
+      }
+    } else {
+      names.forEach(name -> headers.add(name));
     }
-    bw.write(columnNames.get(columnNames.size() - 1) + "\n");
-  }
+    records.add(headers);
 
-  private static int writeResultSet(SessionDataSet rs, BufferedWriter bw)
-      throws IOException, StatementExecutionException, IoTDBConnectionException {
-    int line = 0;
-    long timestamp = System.currentTimeMillis();
-    while (rs.hasNext()) {
-      RowRecord rowRecord = rs.next();
-      List<Field> fields = rowRecord.getFields();
-      writeTime(rowRecord.getTimestamp(), bw);
-      writeValue(fields, bw);
-      line++;
-      if (line % EXPORT_PER_LINE_COUNT == 0) {
-        long tmp = System.currentTimeMillis();
-        System.out.printf("%d lines have been exported, it takes %dms%n", line, (tmp - timestamp));
-        timestamp = tmp;
+    while (sessionDataSet.hasNext()) {
+      RowRecord rowRecord = sessionDataSet.next();
+      ArrayList<Object> record = new ArrayList<>();
+      if (rowRecord.getTimestamp() != 0) {
+        record.add(timeTrans(rowRecord.getTimestamp()));
       }
+      rowRecord
+          .getFields()
+          .forEach(
+              field -> {
+                String fieldStringValue = field.getStringValue();
+                if (!field.getStringValue().equals("null")) {
+                  if (field.getDataType() == TSDataType.TEXT
+                      && !fieldStringValue.startsWith("root.")) {
+                    fieldStringValue = "\"" + fieldStringValue + "\"";
+                  }
+                  record.add(fieldStringValue);
+                } else {
+                  record.add("");
+                }
+              });
+      records.add(record);
     }
-    return line;
+    return records;
   }
 
-  private static void writeTime(Long time, BufferedWriter bw) throws IOException {
-    ZonedDateTime dateTime;
+  public static String timeTrans(Long time) {
     String timestampPrecision = "ms";
     switch (timeFormat) {
       case "default":
-        String str =
-            RpcUtils.parseLongToDateWithPrecision(
-                DateTimeFormatter.ISO_OFFSET_DATE_TIME, time, zoneId, timestampPrecision);
-        bw.write(str + ",");
-        break;
+        return RpcUtils.parseLongToDateWithPrecision(
+            DateTimeFormatter.ISO_OFFSET_DATE_TIME, time, zoneId, timestampPrecision);
       case "timestamp":
       case "long":
       case "number":
-        bw.write(time + ",");
-        break;
+        return String.valueOf(time);
       default:
-        dateTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(time), zoneId);
-        bw.write(dateTime.format(DateTimeFormatter.ofPattern(timeFormat)) + ",");
-        break;
-    }
-  }
-
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private static void writeValue(List<Field> fields, BufferedWriter bw) throws IOException {
-    for (int j = 0; j < fields.size() - 1; j++) {
-      String value = fields.get(j).getStringValue();
-      if ("null".equalsIgnoreCase(value)) {
-        bw.write(",");
-      } else {
-        if (fields.get(j).getDataType() == TSDataType.TEXT) {
-          int location = value.indexOf("\"");
-          if (location > -1) {
-            if (location == 0 || value.charAt(location - 1) != '\\') {
-              bw.write("\"" + value.replace("\"", "\\\"") + "\",");
-            } else {
-              bw.write("\"" + value + "\",");
-            }
-          } else if (value.contains(",")) {
-            bw.write("\"" + value + "\",");
-          } else {
-            bw.write(value + ",");
-          }
-        } else {
-          bw.write(value + ",");
-        }
-      }
-    }
-    String lastValue = fields.get(fields.size() - 1).getStringValue();
-    if ("null".equalsIgnoreCase(lastValue)) {
-      bw.write("\n");
-    } else {
-      if (fields.get(fields.size() - 1).getDataType() == TSDataType.TEXT) {
-        int location = lastValue.indexOf("\"");
-        if (location > -1) {
-          if (location == 0 || lastValue.charAt(location - 1) != '\\') {
-            bw.write("\"" + lastValue.replace("\"", "\\\"") + "\"\n");
-          } else {
-            bw.write("\"" + lastValue + "\"\n");
-          }
-        } else if (lastValue.contains(",")) {
-          bw.write("\"" + lastValue + "\"\n");
-        } else {
-          bw.write(lastValue + "\n");
-        }
-      } else {
-        bw.write(lastValue + "\n");
-      }
+        return ZonedDateTime.ofInstant(Instant.ofEpochMilli(time), zoneId)
+            .format(DateTimeFormatter.ofPattern(timeFormat));
     }
   }
 }
diff --git a/cli/src/main/java/org/apache/iotdb/tool/ImportCsv.java b/cli/src/main/java/org/apache/iotdb/tool/ImportCsv.java
index fb7a878..6dd76d9 100644
--- a/cli/src/main/java/org/apache/iotdb/tool/ImportCsv.java
+++ b/cli/src/main/java/org/apache/iotdb/tool/ImportCsv.java
@@ -16,53 +16,67 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.tool;
 
 import org.apache.iotdb.exception.ArgsErrorException;
 import org.apache.iotdb.rpc.IoTDBConnectionException;
 import org.apache.iotdb.rpc.StatementExecutionException;
 import org.apache.iotdb.session.Session;
-import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.session.SessionDataSet;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-import jline.console.ConsoleReader;
-import me.tongfei.progressbar.ProgressBar;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.DefaultParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.csv.CSVRecord;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.thrift.annotation.Nullable;
 
-import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
-import java.io.LineNumberReader;
-import java.nio.charset.StandardCharsets;
+import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.iotdb.tsfile.file.metadata.enums.TSDataType.BOOLEAN;
+import static org.apache.iotdb.tsfile.file.metadata.enums.TSDataType.DOUBLE;
+import static org.apache.iotdb.tsfile.file.metadata.enums.TSDataType.FLOAT;
+import static org.apache.iotdb.tsfile.file.metadata.enums.TSDataType.INT32;
+import static org.apache.iotdb.tsfile.file.metadata.enums.TSDataType.INT64;
+import static org.apache.iotdb.tsfile.file.metadata.enums.TSDataType.TEXT;
 
-/** read a CSV formatted data File and insert all the data into IoTDB. */
 public class ImportCsv extends AbstractCsvTool {
 
   private static final String FILE_ARGS = "f";
   private static final String FILE_NAME = "file or folder";
-  private static final String FILE_SUFFIX = "csv";
+
+  private static final String FAILED_FILE_ARGS = "fd";
+  private static final String FAILED_FILE_NAME = "failed file directory";
+
+  private static final String CSV_SUFFIXS = "csv";
+  private static final String TXT_SUFFIXS = "txt";
 
   private static final String TSFILEDB_CLI_PREFIX = "ImportCsv";
   private static final String ILLEGAL_PATH_ARGUMENT = "Path parameter is null";
 
-  // put these variable in here, because sonar fails.  have to extract some code into a function.
-  // nextNode method.
-  private static int i;
-  private static int startIndex;
+  private static String targetPath;
+  private static String failedFileDirectory = null;
 
   /**
    * create the commandline options.
@@ -83,6 +97,15 @@ public class ImportCsv extends AbstractCsvTool {
             .build();
     options.addOption(opFile);
 
+    Option opFailedFile =
+        Option.builder(FAILED_FILE_ARGS)
+            .argName(FAILED_FILE_NAME)
+            .hasArg()
+            .desc(
+                "Specifying a directory to save failed file, default YOUR_CSV_FILE_PATH (optional)")
+            .build();
+    options.addOption(opFailedFile);
+
     Option opHelp =
         Option.builder(HELP_ARGS)
             .longOpt(HELP_ARGS)
@@ -102,107 +125,25 @@ public class ImportCsv extends AbstractCsvTool {
     return options;
   }
 
-  /** Data from csv To tsfile. */
-  @SuppressWarnings("squid:S1135")
-  private static void loadDataFromCSV(File file) {
-    int fileLine;
-    try {
-      fileLine = getFileLineCount(file);
-    } catch (IOException e) {
-      System.out.println("Failed to import file: " + file.getName());
-      return;
-    }
-    System.out.println("Start to import data from: " + file.getName());
-    try (BufferedReader br =
-            new BufferedReader(
-                new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8));
-        ProgressBar pb = new ProgressBar("Import from: " + file.getName(), fileLine)) {
-      pb.setExtraMessage("Importing...");
-      String header = br.readLine();
-      String[] cols = splitCsvLine(header);
-      if (cols.length <= 1) {
-        System.out.println("The CSV file " + file.getName() + " illegal, please check first line");
-        return;
-      }
-
-      List<String> devices = new ArrayList<>();
-      List<Long> times = new ArrayList<>();
-      List<List<String>> measurementsList = new ArrayList<>();
-      List<List<String>> valuesList = new ArrayList<>();
-      Map<String, List<Integer>> devicesToPositions = new HashMap<>();
-      Map<String, List<String>> devicesToMeasurements = new HashMap<>();
-
-      for (int i = 1; i < cols.length; i++) {
-        splitColToDeviceAndMeasurement(cols[i], devicesToPositions, devicesToMeasurements, i);
-      }
-
-      SimpleDateFormat timeFormatter = null;
-      boolean useFormatter = false;
-
-      int lineNumber = 0;
-      String line;
-      while ((line = br.readLine()) != null) {
-        cols = splitCsvLine(line);
-        lineNumber++;
-        if (lineNumber == 1) {
-          timeFormatter = formatterInit(cols[0]);
-          useFormatter = (timeFormatter != null);
-        }
-        for (Entry<String, List<Integer>> deviceToPositions : devicesToPositions.entrySet()) {
-          String device = deviceToPositions.getKey();
-          devices.add(device);
-
-          times.add(parseTime(cols[0], useFormatter, timeFormatter));
-
-          List<String> values = new ArrayList<>();
-          for (int position : deviceToPositions.getValue()) {
-            values.add(cols[position]);
-          }
-          valuesList.add(values);
-
-          measurementsList.add(devicesToMeasurements.get(device));
-        }
-        if (lineNumber % 10000 == 0) {
-          try {
-            session.insertRecords(devices, times, measurementsList, valuesList);
-          } catch (StatementExecutionException e) {
-            if (e.getMessage().contains("failed to insert measurements")) {
-              System.out.println("Meet error when insert csv because " + e.getMessage());
-              System.out.println("Continue inserting... ");
-            } else {
-              throw e;
-            }
-          }
-          pb.stepTo(lineNumber + 1L);
-          devices = new ArrayList<>();
-          times = new ArrayList<>();
-          measurementsList = new ArrayList<>();
-          valuesList = new ArrayList<>();
-        }
-      }
-      // TODO change it to insertTablet, now is slow
-      try {
-        session.insertRecords(devices, times, measurementsList, valuesList);
-      } catch (StatementExecutionException e) {
-        if (e.getMessage().contains("failed to insert measurements")) {
-          System.out.println("Meet error when insert csv because " + e.getMessage());
-          System.out.println("Continue inserting... ");
-        } else {
-          throw e;
-        }
+  /**
+   * parse optional params
+   *
+   * @param commandLine
+   */
+  private static void parseSpecialParams(CommandLine commandLine) {
+    timeZoneID = commandLine.getOptionValue(TIME_ZONE_ARGS);
+    targetPath = commandLine.getOptionValue(FILE_ARGS);
+    if (commandLine.getOptionValue(FAILED_FILE_ARGS) != null) {
+      failedFileDirectory = commandLine.getOptionValue(FAILED_FILE_ARGS);
+      File file = new File(failedFileDirectory);
+      if (!file.isDirectory()) {
+        file.mkdir();
+        failedFileDirectory = file.getAbsolutePath() + File.separator;
       }
-      System.out.println("Insert csv successfully!");
-      pb.stepTo(fileLine);
-    } catch (FileNotFoundException e) {
-      System.out.println("Cannot find " + file.getName() + " because: " + e.getMessage());
-    } catch (IOException e) {
-      System.out.println("CSV file read exception because: " + e.getMessage());
-    } catch (IoTDBConnectionException | StatementExecutionException e) {
-      System.out.println("Meet error when insert csv because " + e.getMessage());
     }
   }
 
-  public static void main(String[] args) throws IOException {
+  public static void main(String[] args) throws IOException, IoTDBConnectionException {
     Options options = createOptions();
     HelpFormatter hf = new HelpFormatter();
     hf.setOptionComparator(null);
@@ -217,7 +158,7 @@ public class ImportCsv extends AbstractCsvTool {
     }
     try {
       commandLine = parser.parse(options, args);
-    } catch (ParseException e) {
+    } catch (org.apache.commons.cli.ParseException e) {
       System.out.println("Parse error: " + e.getMessage());
       hf.printHelp(TSFILEDB_CLI_PREFIX, options, true);
       return;
@@ -227,235 +168,517 @@ public class ImportCsv extends AbstractCsvTool {
       return;
     }
 
-    ConsoleReader reader = new ConsoleReader();
-    reader.setExpandEvents(false);
     try {
-      parseBasicParams(commandLine, reader);
+      parseBasicParams(commandLine);
       String filename = commandLine.getOptionValue(FILE_ARGS);
       if (filename == null) {
         hf.printHelp(TSFILEDB_CLI_PREFIX, options, true);
         return;
       }
       parseSpecialParams(commandLine);
-      importCsvFromFile(host, port, username, password, filename, timeZoneID);
     } catch (ArgsErrorException e) {
       System.out.println("Args error: " + e.getMessage());
     } catch (Exception e) {
       System.out.println("Encounter an error, because: " + e.getMessage());
-    } finally {
-      reader.close();
-    }
-  }
-
-  private static long parseTime(String str, boolean useFormatter, SimpleDateFormat timeFormatter) {
-    try {
-      if (useFormatter) {
-        return timeFormatter.parse(str).getTime();
-      } else {
-        return Long.parseLong(str);
-      }
-    } catch (Exception e) {
-      throw new IllegalArgumentException(
-          "Input time format "
-              + str
-              + "error. Input like yyyy-MM-dd HH:mm:ss, yyyy-MM-ddTHH:mm:ss or yyyy-MM-ddTHH:mm:ss.SSSZ");
-    }
-  }
-
-  private static SimpleDateFormat formatterInit(String time) {
-
-    try {
-      Long.parseLong(time);
-      return null;
-    } catch (Exception ignored) {
-      // do nothing
     }
 
-    for (String timeFormat : STRING_TIME_FORMAT) {
-      SimpleDateFormat format = new SimpleDateFormat(timeFormat);
-      try {
-        format.parse(time).getTime();
-        return format;
-      } catch (java.text.ParseException ignored) {
-        // do nothing
-      }
-    }
-    return null;
+    importFromTargetPath(host, Integer.valueOf(port), username, password, targetPath, timeZoneID);
   }
 
-  private static void parseSpecialParams(CommandLine commandLine) {
-    timeZoneID = commandLine.getOptionValue(TIME_ZONE_ARGS);
-  }
-
-  public static void importCsvFromFile(
-      String ip, String port, String username, String password, String filename, String timeZone) {
+  /**
+   * Specifying a CSV file or a directory including CSV files that you want to import. This method
+   * can be offered to console cli to implement importing CSV file by command.
+   *
+   * @param host
+   * @param port
+   * @param username
+   * @param password
+   * @param targetPath a CSV file or a directory including CSV files
+   * @param timeZone
+   * @throws IoTDBConnectionException
+   */
+  public static void importFromTargetPath(
+      String host, int port, String username, String password, String targetPath, String timeZone)
+      throws IoTDBConnectionException {
     try {
-      session = new Session(ip, Integer.parseInt(port), username, password, false);
+      session = new Session(host, Integer.valueOf(port), username, password, false);
       session.open(false);
       timeZoneID = timeZone;
       setTimeZone();
 
-      File file = new File(filename);
+      File file = new File(targetPath);
       if (file.isFile()) {
         importFromSingleFile(file);
       } else if (file.isDirectory()) {
-        importFromDirectory(file);
+        File[] files = file.listFiles();
+        if (files == null) {
+          return;
+        }
+
+        for (File subFile : files) {
+          if (subFile.isFile()) {
+            importFromSingleFile(subFile);
+          }
+        }
+      } else {
+        System.out.println("File not found!");
       }
-    } catch (IoTDBConnectionException e) {
+    } catch (IoTDBConnectionException | StatementExecutionException e) {
       System.out.println("Encounter an error when connecting to server, because " + e.getMessage());
-    } catch (StatementExecutionException e) {
-      System.out.println(
-          "Encounter an error when executing the statement, because " + e.getMessage());
     } finally {
       if (session != null) {
-        try {
-          session.close();
-        } catch (IoTDBConnectionException e) {
-          System.out.println(
-              "Encounter an error when closing the connection, because " + e.getMessage());
-        }
+        session.close();
       }
     }
   }
 
+  /**
+   * import the CSV file and load headers and records.
+   *
+   * @param file the File object of the CSV file that you want to import.
+   */
   private static void importFromSingleFile(File file) {
-    if (file.getName().endsWith(FILE_SUFFIX)) {
-      loadDataFromCSV(file);
+    if (file.getName().endsWith(CSV_SUFFIXS) || file.getName().endsWith(TXT_SUFFIXS)) {
+      try {
+        CSVParser csvRecords = readCsvFile(file.getAbsolutePath());
+        List<String> headerNames = csvRecords.getHeaderNames();
+        List<CSVRecord> records = csvRecords.getRecords();
+        if (headerNames.isEmpty()) {
+          System.out.println("Empty file!");
+          return;
+        }
+        if (!headerNames.contains("Time")) {
+          System.out.println("No headers!");
+          return;
+        }
+        if (records.isEmpty()) {
+          System.out.println("No records!");
+          return;
+        }
+        String failedFilePath = null;
+        if (failedFileDirectory == null) {
+          failedFilePath = file.getAbsolutePath() + ".failed";
+        } else {
+          failedFilePath = failedFileDirectory + file.getName() + ".failed";
+        }
+        if (!headerNames.contains("Device")) {
+          writeDataAlignedByTime(headerNames, records, failedFilePath);
+        } else {
+          writeDataAlignedByDevice(headerNames, records, failedFilePath);
+        }
+      } catch (IOException e) {
+        System.out.println("CSV file read exception because: " + e.getMessage());
+      }
     } else {
-      System.out.println(
-          "File " + file.getName() + "  should ends with '.csv' if you want to import");
+      System.out.println("The file name must end with \"csv\" or \"txt\"!");
     }
   }
 
-  private static void importFromDirectory(File file) {
-    File[] files = file.listFiles();
-    if (files == null) {
-      return;
+  /**
+   * if the data is aligned by time, the data will be written by this method.
+   *
+   * @param headerNames the header names of CSV file
+   * @param records the records of CSV file
+   * @param failedFilePath the directory to save the failed files
+   */
+  private static void writeDataAlignedByTime(
+      List<String> headerNames, List<CSVRecord> records, String failedFilePath) {
+    HashMap<String, List<String>> deviceAndMeasurementNames = new HashMap<>();
+    HashMap<String, TSDataType> headerTypeMap = new HashMap<>();
+    HashMap<String, String> headerNameMap = new HashMap<>();
+    parseHeaders(headerNames, deviceAndMeasurementNames, headerTypeMap, headerNameMap);
+
+    Set<String> devices = deviceAndMeasurementNames.keySet();
+    String devicesStr = StringUtils.join(devices, ",");
+    try {
+      queryType(devicesStr, headerTypeMap, "Time");
+    } catch (StatementExecutionException | IoTDBConnectionException e) {
+      e.printStackTrace();
     }
 
-    for (File subFile : files) {
-      if (subFile.isFile()) {
-        if (subFile.getName().endsWith(FILE_SUFFIX)) {
-          loadDataFromCSV(subFile);
-        } else {
-          System.out.println(
-              "File " + file.getName() + " should ends with '.csv' if you want to import");
-        }
+    SimpleDateFormat timeFormatter = formatterInit(records.get(0).get("Time"));
+
+    ArrayList<List<Object>> failedRecords = new ArrayList<>();
+
+    for (Map.Entry<String, List<String>> entry : deviceAndMeasurementNames.entrySet()) {
+      String deviceId = entry.getKey();
+      List<Long> times = new ArrayList<>();
+      List<String> measurementNames = entry.getValue();
+      List<List<TSDataType>> typesList = new ArrayList<>();
+      List<List<Object>> valuesList = new ArrayList<>();
+      List<List<String>> measurementsList = new ArrayList<>();
+      records.stream()
+          .forEach(
+              record -> {
+                ArrayList<TSDataType> types = new ArrayList<>();
+                ArrayList<Object> values = new ArrayList<>();
+                ArrayList<String> measurements = new ArrayList<>();
+                AtomicReference<Boolean> isFail = new AtomicReference<>(false);
+                measurementNames.stream()
+                    .forEach(
+                        measurementName -> {
+                          String header = deviceId + "." + measurementName;
+                          String value = record.get(header);
+                          if (!value.equals("")) {
+                            TSDataType type;
+                            if (!headerTypeMap.containsKey(headerNameMap.get(header))) {
+                              type = typeInfer(value);
+                              headerTypeMap.put(header, type);
+                            }
+                            type = headerTypeMap.get(headerNameMap.get(header));
+                            Object valueTransed = typeTrans(value, type);
+                            if (valueTransed == null) {
+                              isFail.set(true);
+                              System.out.println(
+                                  "Line "
+                                      + (records.indexOf(record) + 1)
+                                      + ": "
+                                      + value
+                                      + " can't convert to "
+                                      + type);
+                            } else {
+                              measurements.add(
+                                  headerNameMap.get(header).replace(deviceId + '.', ""));
+                              types.add(type);
+                              values.add(valueTransed);
+                            }
+                          }
+                        });
+                if (isFail.get()) {
+                  failedRecords.add(record.stream().collect(Collectors.toList()));
+                }
+                if (!measurements.isEmpty()) {
+                  try {
+                    if (timeFormatter == null) {
+                      try {
+                        times.add(Long.valueOf(record.get("Time")));
+                      } catch (Exception e) {
+                        System.out.println(
+                            "Meet error when insert csv because the format of time is not supported");
+                        System.exit(0);
+                      }
+                    } else {
+                      times.add(timeFormatter.parse(record.get("Time")).getTime());
+                    }
+                  } catch (ParseException e) {
+                    e.printStackTrace();
+                  }
+                  typesList.add(types);
+                  valuesList.add(values);
+                  measurementsList.add(measurements);
+                }
+              });
+      try {
+        session.insertRecordsOfOneDevice(deviceId, times, measurementsList, typesList, valuesList);
+      } catch (StatementExecutionException | IoTDBConnectionException e) {
+        System.out.println("Meet error when insert csv because " + e.getMessage());
+        System.exit(0);
       }
     }
+    if (!failedRecords.isEmpty()) {
+      writeCsvFile(headerNames, failedRecords, failedFilePath);
+    }
+    System.out.println("Import completely!");
   }
 
-  private static int getFileLineCount(File file) throws IOException {
-    int line;
-    try (LineNumberReader count =
-        new LineNumberReader(
-            new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8))) {
-      while (count.skip(Long.MAX_VALUE) > 0) {
-        // Loop just in case the file is > Long.MAX_VALUE or skip() decides to not read the entire
-        // file
-      }
-      // +1 because line index starts at 0
-      line = count.getLineNumber() + 1;
+  /**
+   * if the data is aligned by device, the data will be written by this method.
+   *
+   * @param headerNames the header names of CSV file
+   * @param records the records of CSV file
+   * @param failedFilePath the directory to save the failed files
+   */
+  private static void writeDataAlignedByDevice(
+      List<String> headerNames, List<CSVRecord> records, String failedFilePath) {
+    HashMap<String, TSDataType> headerTypeMap = new HashMap<>();
+    HashMap<String, String> headerNameMap = new HashMap<>();
+    parseHeaders(headerNames, null, headerTypeMap, headerNameMap);
+    Set<String> devices =
+        records.stream().map(record -> record.get("Device")).collect(Collectors.toSet());
+    String devicesStr = StringUtils.join(devices, ",");
+    try {
+      queryType(devicesStr, headerTypeMap, "Device");
+    } catch (StatementExecutionException | IoTDBConnectionException e) {
+      e.printStackTrace();
     }
-    return line;
+
+    SimpleDateFormat timeFormatter = formatterInit(records.get(0).get("Time"));
+    Set<String> measurementNames = headerNameMap.keySet();
+    ArrayList<List<Object>> failedRecords = new ArrayList<>();
+
+    devices.stream()
+        .forEach(
+            device -> {
+              List<Long> times = new ArrayList<>();
+
+              List<List<TSDataType>> typesList = new ArrayList<>();
+              List<List<Object>> valuesList = new ArrayList<>();
+              List<List<String>> measurementsList = new ArrayList<>();
+
+              records.stream()
+                  .filter(record -> record.get("Device").equals(device))
+                  .forEach(
+                      record -> {
+                        ArrayList<TSDataType> types = new ArrayList<>();
+                        ArrayList<Object> values = new ArrayList<>();
+                        ArrayList<String> measurements = new ArrayList<>();
+
+                        AtomicReference<Boolean> isFail = new AtomicReference<>(false);
+
+                        measurementNames.stream()
+                            .forEach(
+                                measurement -> {
+                                  String value = record.get(measurement);
+                                  if (!value.equals("")) {
+                                    TSDataType type;
+                                    if (!headerTypeMap.containsKey(
+                                        headerNameMap.get(measurement))) {
+                                      type = typeInfer(value);
+                                      headerTypeMap.put(measurement, type);
+                                    }
+                                    type = headerTypeMap.get(headerNameMap.get(measurement));
+                                    Object valueTransed = typeTrans(value, type);
+                                    if (valueTransed == null) {
+                                      isFail.set(true);
+                                      System.out.println(
+                                          "Line "
+                                              + (records.indexOf(record) + 1)
+                                              + ": "
+                                              + value
+                                              + " can't convert to "
+                                              + type);
+                                    } else {
+                                      values.add(valueTransed);
+                                      measurements.add(headerNameMap.get(measurement));
+                                      types.add(type);
+                                    }
+                                  }
+                                });
+                        if (isFail.get()) {
+                          failedRecords.add(record.stream().collect(Collectors.toList()));
+                        }
+                        if (!measurements.isEmpty()) {
+                          try {
+                            if (timeFormatter == null) {
+                              try {
+                                times.add(Long.valueOf(record.get("Time")));
+                              } catch (Exception e) {
+                                System.out.println(
+                                    "Meet error when insert csv because the format of time is not supported");
+                                System.exit(0);
+                              }
+                            } else {
+                              times.add(timeFormatter.parse(record.get("Time")).getTime());
+                            }
+                          } catch (ParseException e) {
+                            e.printStackTrace();
+                          }
+                          typesList.add(types);
+                          valuesList.add(values);
+                          measurementsList.add(measurements);
+                        }
+                      });
+              try {
+                session.insertRecordsOfOneDevice(
+                    device, times, measurementsList, typesList, valuesList);
+              } catch (StatementExecutionException | IoTDBConnectionException e) {
+                System.out.println("Meet error when insert csv because " + e.getMessage());
+                System.exit(0);
+              }
+            });
+    if (!failedRecords.isEmpty()) {
+      writeCsvFile(headerNames, failedRecords, failedFilePath);
+    }
+    System.out.println("Import completely!");
   }
 
-  private static void splitColToDeviceAndMeasurement(
-      String col,
-      Map<String, List<Integer>> devicesToPositions,
-      Map<String, List<String>> devicesToMeasurements,
-      int position) {
-    if (col.length() > 0) {
-      if (col.charAt(col.length() - 1) == TsFileConstant.DOUBLE_QUOTE) {
-        int endIndex = col.lastIndexOf('"', col.length() - 2);
-        // if a double quotes with escape character
-        while (endIndex != -1 && col.charAt(endIndex - 1) == '\\') {
-          endIndex = col.lastIndexOf('"', endIndex - 2);
-        }
-        if (endIndex != -1 && (endIndex == 0 || col.charAt(endIndex - 1) == '.')) {
-          putDeviceAndMeasurement(
-              col.substring(0, endIndex - 1),
-              col.substring(endIndex),
-              devicesToPositions,
-              devicesToMeasurements,
-              position);
-        } else {
-          throw new IllegalArgumentException(ILLEGAL_PATH_ARGUMENT);
-        }
-      } else if (col.charAt(col.length() - 1) != TsFileConstant.DOUBLE_QUOTE
-          && col.charAt(col.length() - 1) != TsFileConstant.PATH_SEPARATOR_CHAR) {
-        int endIndex = col.lastIndexOf(TsFileConstant.PATH_SEPARATOR_CHAR);
-        if (endIndex < 0) {
-          putDeviceAndMeasurement("", col, devicesToPositions, devicesToMeasurements, position);
-        } else {
-          putDeviceAndMeasurement(
-              col.substring(0, endIndex),
-              col.substring(endIndex + 1),
-              devicesToPositions,
-              devicesToMeasurements,
-              position);
-        }
+  /**
+   * read data from the CSV file
+   *
+   * @param path
+   * @return
+   * @throws IOException
+   */
+  private static CSVParser readCsvFile(String path) throws IOException {
+    return CSVFormat.EXCEL
+        .withFirstRecordAsHeader()
+        .withQuote('\'')
+        .withEscape('\\')
+        .withIgnoreEmptyLines()
+        .parse(new InputStreamReader(new FileInputStream(path)));
+  }
+
+  /**
+   * parse deviceNames, measurementNames(aligned by time), headerType from headers
+   *
+   * @param headerNames
+   * @param deviceAndMeasurementNames
+   * @param headerTypeMap
+   * @param headerNameMap
+   */
+  private static void parseHeaders(
+      List<String> headerNames,
+      @Nullable HashMap<String, List<String>> deviceAndMeasurementNames,
+      HashMap<String, TSDataType> headerTypeMap,
+      HashMap<String, String> headerNameMap) {
+    String regex = "(?<=\\()\\S+(?=\\))";
+    Pattern pattern = Pattern.compile(regex);
+    for (String headerName : headerNames) {
+      if (headerName.equals("Time") || headerName.equals("Device")) continue;
+      Matcher matcher = pattern.matcher(headerName);
+      String type;
+      if (matcher.find()) {
+        type = matcher.group();
+        String headerNameWithoutType =
+            headerName.replace("(" + type + ")", "").replaceAll("\\s+", "");
+        headerNameMap.put(headerName, headerNameWithoutType);
+        headerTypeMap.put(headerNameWithoutType, getType(type));
       } else {
-        throw new IllegalArgumentException(ILLEGAL_PATH_ARGUMENT);
+        headerNameMap.put(headerName, headerName);
+      }
+      String[] split = headerName.split("\\.");
+      String measurementName = split[split.length - 1];
+      String deviceName = headerName.replace("." + measurementName, "");
+      if (deviceAndMeasurementNames != null) {
+        if (!deviceAndMeasurementNames.containsKey(deviceName)) {
+          deviceAndMeasurementNames.put(deviceName, new ArrayList<>());
+        }
+        deviceAndMeasurementNames.get(deviceName).add(measurementName);
       }
-    } else {
-      putDeviceAndMeasurement("", col, devicesToPositions, devicesToMeasurements, position);
     }
   }
 
-  private static void putDeviceAndMeasurement(
-      String device,
-      String measurement,
-      Map<String, List<Integer>> devicesToPositions,
-      Map<String, List<String>> devicesToMeasurements,
-      int position) {
-    if (devicesToMeasurements.get(device) == null && devicesToPositions.get(device) == null) {
-      List<String> measurements = new ArrayList<>();
-      measurements.add(measurement);
-      devicesToMeasurements.put(device, measurements);
-      List<Integer> positions = new ArrayList<>();
-      positions.add(position);
-      devicesToPositions.put(device, positions);
-    } else {
-      devicesToMeasurements.get(device).add(measurement);
-      devicesToPositions.get(device).add(position);
+  /**
+   * query data type of timeseries from IoTDB
+   *
+   * @param deviceNames
+   * @param headerTypeMap
+   * @param alignedType
+   * @throws IoTDBConnectionException
+   * @throws StatementExecutionException
+   */
+  private static void queryType(
+      String deviceNames, HashMap<String, TSDataType> headerTypeMap, String alignedType)
+      throws IoTDBConnectionException, StatementExecutionException {
+    String sql = "select * from " + deviceNames + " limit 1";
+    SessionDataSet sessionDataSet = session.executeQueryStatement(sql);
+    List<String> columnNames = sessionDataSet.getColumnNames();
+    List<String> columnTypes = sessionDataSet.getColumnTypes();
+    for (int i = 1; i < columnNames.size(); i++) {
+      if (alignedType == "Time") {
+        headerTypeMap.put(columnNames.get(i), getType(columnTypes.get(i)));
+      } else if (alignedType == "Device") {
+        String[] split = columnNames.get(i).split("\\.");
+        String measurement = split[split.length - 1];
+        headerTypeMap.put(measurement, getType(columnTypes.get(i)));
+      }
     }
   }
 
-  public static String[] splitCsvLine(String path) {
-    List<String> nodes = new ArrayList<>();
-    startIndex = 0;
-    for (i = 0; i < path.length(); i++) {
-      if (path.charAt(i) == ',') {
-        nodes.add(path.substring(startIndex, i));
-        startIndex = i + 1;
-      } else if (path.charAt(i) == '"') {
-        nextNode(path, nodes, '"');
-      } else if (path.charAt(i) == '\'') {
-        nextNode(path, nodes, '\'');
-      }
-    }
-    if (path.charAt(path.length() - 1) == ',') {
-      nodes.add("");
+  /**
+   * return a suit time formatter
+   *
+   * @param time
+   * @return
+   */
+  private static SimpleDateFormat formatterInit(String time) {
+    try {
+      Long.parseLong(time);
+      return null;
+    } catch (Exception ignored) {
+      // do nothing
     }
-    if (startIndex <= path.length() - 1) {
-      nodes.add(path.substring(startIndex));
+
+    for (String timeFormat : STRING_TIME_FORMAT) {
+      SimpleDateFormat format = new SimpleDateFormat(timeFormat);
+      try {
+        format.parse(time).getTime();
+        System.out.println(timeFormat);
+        return format;
+      } catch (java.text.ParseException ignored) {
+        // do nothing
+      }
     }
-    return nodes.toArray(new String[0]);
+    return null;
   }
 
-  public static void nextNode(String path, List<String> nodes, char enclose) {
-    int endIndex = path.indexOf(enclose, i + 1);
-    // if a double quotes with escape character
-    while (endIndex != -1 && path.charAt(endIndex - 1) == '\\') {
-      endIndex = path.indexOf(enclose, endIndex + 1);
+  /**
+   * return the TSDataType
+   *
+   * @param typeStr
+   * @return
+   */
+  private static TSDataType getType(String typeStr) {
+    switch (typeStr) {
+      case "TEXT":
+        return TEXT;
+      case "BOOLEAN":
+        return BOOLEAN;
+      case "INT32":
+        return INT32;
+      case "INT64":
+        return INT64;
+      case "FLOAT":
+        return FLOAT;
+      case "DOUBLE":
+        return DOUBLE;
+      default:
+        return null;
     }
-    if (endIndex != -1 && (endIndex == path.length() - 1 || path.charAt(endIndex + 1) == ',')) {
-      nodes.add(path.substring(startIndex + 1, endIndex));
-      i = endIndex + 1;
-      startIndex = endIndex + 2;
+  }
+
+  /**
+   * if data type of timeseries is not defined in headers of schema, this method will be called to
+   * do type inference
+   *
+   * @param value
+   * @return
+   */
+  private static TSDataType typeInfer(String value) {
+    if (value.contains("\"")) return TEXT;
+    else if (value.equals("true") || value.equals("false")) return BOOLEAN;
+    else if (!value.contains(".")) {
+      try {
+        Integer.valueOf(value);
+        return INT32;
+      } catch (Exception e) {
+        return INT64;
+      }
     } else {
-      throw new IllegalArgumentException("Illegal csv line" + path);
+      if (Float.valueOf(value).toString().length() == Double.valueOf(value).toString().length())
+        return FLOAT;
+      else return DOUBLE;
+    }
+  }
+
+  /**
+   * @param value
+   * @param type
+   * @return
+   */
+  private static Object typeTrans(String value, TSDataType type) {
+    try {
+      switch (type) {
+        case TEXT:
+          return value.substring(1, value.length() - 1);
+        case BOOLEAN:
+          if (!value.equals("true") && !value.equals("false")) {
+            return null;
+          }
+          return Boolean.valueOf(value);
+        case INT32:
+          return Integer.valueOf(value);
+        case INT64:
+          return Long.valueOf(value);
+        case FLOAT:
+          return Float.valueOf(value);
+        case DOUBLE:
+          return Double.valueOf(value);
+        default:
+          return null;
+      }
+    } catch (NumberFormatException e) {
+      return null;
     }
   }
 }
diff --git a/cli/src/test/java/org/apache/iotdb/cli/AbstractScript.java b/cli/src/test/java/org/apache/iotdb/cli/AbstractScript.java
index 0be9b55..12eaae8 100644
--- a/cli/src/test/java/org/apache/iotdb/cli/AbstractScript.java
+++ b/cli/src/test/java/org/apache/iotdb/cli/AbstractScript.java
@@ -18,6 +18,8 @@
  */
 package org.apache.iotdb.cli;
 
+import org.apache.thrift.annotation.Nullable;
+
 import java.io.*;
 import java.util.ArrayList;
 import java.util.List;
@@ -27,7 +29,7 @@ import static org.junit.Assert.assertEquals;
 
 public abstract class AbstractScript {
 
-  protected void testOutput(ProcessBuilder builder, String[] output) throws IOException {
+  protected void testOutput(ProcessBuilder builder, @Nullable String[] output) throws IOException {
     builder.redirectErrorStream(true);
     Process p = builder.start();
     BufferedReader r = new BufferedReader(new InputStreamReader(p.getInputStream()));
@@ -54,8 +56,10 @@ public abstract class AbstractScript {
       System.out.println(s);
     }
 
-    for (int i = 0; i < output.length; i++) {
-      assertEquals(output[output.length - 1 - i], outputList.get(outputList.size() - 1 - i));
+    if (output != null) {
+      for (int i = 0; i < output.length; i++) {
+        assertEquals(output[output.length - 1 - i], outputList.get(outputList.size() - 1 - i));
+      }
     }
   }
 
diff --git a/cli/src/test/java/org/apache/iotdb/tool/CsvLineSplitTest.java b/cli/src/test/java/org/apache/iotdb/tool/CsvLineSplitTest.java
deleted file mode 100644
index fd1c9ba..0000000
--- a/cli/src/test/java/org/apache/iotdb/tool/CsvLineSplitTest.java
+++ /dev/null
@@ -1,33 +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.iotdb.tool;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class CsvLineSplitTest {
-
-  @Test
-  public void testSplit() {
-    Assert.assertArrayEquals(
-        new String[] {"", "a", "b", "c", "\\\""}, ImportCsv.splitCsvLine(",a,b,c,\"\\\"\""));
-    Assert.assertArrayEquals(
-        new String[] {"", "a", "b", "\\'"}, ImportCsv.splitCsvLine(",a,b,\"\\'\""));
-  }
-}
diff --git a/cli/src/test/java/org/apache/iotdb/tool/ExportCsvTestIT.java b/cli/src/test/java/org/apache/iotdb/tool/integration/ExportCsvTestIT.java
similarity index 98%
rename from cli/src/test/java/org/apache/iotdb/tool/ExportCsvTestIT.java
rename to cli/src/test/java/org/apache/iotdb/tool/integration/ExportCsvTestIT.java
index 23bc812..b745e86 100644
--- a/cli/src/test/java/org/apache/iotdb/tool/ExportCsvTestIT.java
+++ b/cli/src/test/java/org/apache/iotdb/tool/integration/ExportCsvTestIT.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.tool;
+package org.apache.iotdb.tool.integration;
 
 import org.apache.iotdb.cli.AbstractScript;
 
diff --git a/cli/src/test/java/org/apache/iotdb/tool/ImportCsvTestIT.java b/cli/src/test/java/org/apache/iotdb/tool/integration/ImportCsvTestIT.java
similarity index 98%
rename from cli/src/test/java/org/apache/iotdb/tool/ImportCsvTestIT.java
rename to cli/src/test/java/org/apache/iotdb/tool/integration/ImportCsvTestIT.java
index 6516b0e..f5f8f86 100644
--- a/cli/src/test/java/org/apache/iotdb/tool/ImportCsvTestIT.java
+++ b/cli/src/test/java/org/apache/iotdb/tool/integration/ImportCsvTestIT.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.tool;
+package org.apache.iotdb.tool.integration;
 
 import org.apache.iotdb.cli.AbstractScript;
 
diff --git a/cli/src/test/java/org/apache/iotdb/tool/unit/WriteCsvFileTestUT.java b/cli/src/test/java/org/apache/iotdb/tool/unit/WriteCsvFileTestUT.java
new file mode 100644
index 0000000..7c7bd5e
--- /dev/null
+++ b/cli/src/test/java/org/apache/iotdb/tool/unit/WriteCsvFileTestUT.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.iotdb.tool.unit;
+
+import org.apache.iotdb.tool.AbstractCsvTool;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+
+public class WriteCsvFileTestUT {
+  @Test
+  public void writeCsvFileTest() {
+    List<String> headerNames =
+        new ArrayList<>(Arrays.asList("Time", "column1", "column2", "column3"));
+
+    List<Object> row1 = new ArrayList<>(Arrays.asList(1, null, "hello,world", true));
+    List<Object> row2 = new ArrayList<>(Arrays.asList(2, "", "hello,world", false));
+    List<Object> row3 = new ArrayList<>(Arrays.asList(3, "100", "hello world!!!", false));
+    ArrayList<List<Object>> records = new ArrayList<>(Arrays.asList(row1, row2, row3));
+
+    assertTrue(AbstractCsvTool.writeCsvFile(headerNames, records, "./test0.csv"));
+    assertTrue(AbstractCsvTool.writeCsvFile(null, records, "./test1.csv"));
+  }
+}
diff --git a/cross-tests/pom.xml b/cross-tests/pom.xml
index 369339e..b7cefed 100644
--- a/cross-tests/pom.xml
+++ b/cross-tests/pom.xml
@@ -52,6 +52,15 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-csv</artifactId>
+            <version>1.9.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/AbstractScript.java b/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/AbstractScript.java
index b5f9456..b7e0a80 100644
--- a/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/AbstractScript.java
+++ b/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/AbstractScript.java
@@ -18,11 +18,12 @@
  */
 package org.apache.iotdb.cross.tests.tools.importCsv;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.thrift.annotation.Nullable;
+
+import java.io.*;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
@@ -30,8 +31,10 @@ import java.util.Properties;
 import static org.junit.Assert.assertTrue;
 
 public abstract class AbstractScript {
+  protected String[] command;
+  protected final String CSV_FILE = "target" + File.separator + "test.csv";
 
-  protected void testOutput(ProcessBuilder builder, String[] output) throws IOException {
+  protected void testOutput(ProcessBuilder builder, @Nullable String[] output) throws IOException {
     builder.redirectErrorStream(true);
     Process p = builder.start();
     BufferedReader r = new BufferedReader(new InputStreamReader(p.getInputStream()));
@@ -48,17 +51,18 @@ public abstract class AbstractScript {
     r.close();
     p.destroy();
 
-    System.out.println("should contains:");
-    for (String s : output) {
-      System.out.println(s);
-    }
+    if (output != null) {
+      System.out.println("should contains:");
+      for (String s : output) {
+        System.out.println(s);
+      }
 
-    System.out.println("actualOutput:");
-    for (String out : actualOutput) {
-      System.out.println(out);
+      System.out.println("actualOutput:");
+      for (String out : actualOutput) {
+        System.out.println(out);
+      }
+      assertTrue(actualOutput.get(actualOutput.size() - 1).contains(output[output.length - 1]));
     }
-
-    assertTrue(actualOutput.get(actualOutput.size() - 1).contains(output[output.length - 1]));
   }
 
   protected String getCliPath() {
@@ -93,7 +97,23 @@ public abstract class AbstractScript {
         .getAbsolutePath();
   }
 
-  protected abstract void testOnWindows(String[] output) throws IOException;
+  protected void testMethod(@Nullable String[] params, @Nullable String[] output)
+      throws IOException {
+    String[] basicParams =
+        new String[] {"-h", "127.0.0.1", "-p", "6667", "-u", "root", "-pw", "root"};
+    command = ArrayUtils.addAll(command, basicParams);
+    command = ArrayUtils.addAll(command, params);
+    if (params != null) {
+      command = ArrayUtils.addAll(command, basicParams);
+    }
+    ProcessBuilder processBuilder = new ProcessBuilder(command);
+    testOutput(processBuilder, output);
+  }
 
-  protected abstract void testOnUnix(String[] output) throws IOException;
+  protected static CSVParser readCsvFile(String path) throws IOException {
+    return CSVFormat.EXCEL
+        .withQuote('\'')
+        .withEscape('\\')
+        .parse(new InputStreamReader(new FileInputStream(path)));
+  }
 }
diff --git a/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ExportCsvTestIT.java b/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ExportCsvTestIT.java
index 3ffb0f9..4c7c57f 100644
--- a/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ExportCsvTestIT.java
+++ b/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ExportCsvTestIT.java
@@ -24,40 +24,40 @@ import org.apache.iotdb.rpc.IoTDBConnectionException;
 import org.apache.iotdb.rpc.StatementExecutionException;
 import org.apache.iotdb.session.Session;
 
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.csv.CSVRecord;
+import org.apache.commons.lang3.StringUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
 import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 public class ExportCsvTestIT extends AbstractScript {
 
-  private final String SQL_FILE = "target" + File.separator + "sql.txt";
-
-  private final String EXPORT_FILE = "target" + File.separator + "dump0.csv";
-
-  private final String[] output = {
-    "------------------------------------------",
-    "Starting IoTDB Client Export Script",
-    "------------------------------------------",
-    "Start to export data from sql statement",
-    "successfully",
-  };
-
   @Before
   public void setUp() {
     EnvironmentUtils.closeStatMonitor();
     EnvironmentUtils.envSetUp();
+    String os = System.getProperty("os.name").toLowerCase();
+    if (os.startsWith("windows")) {
+      command =
+          new String[] {
+            "cmd.exe",
+            "/c",
+            getCliPath() + File.separator + "tools" + File.separator + "export-csv.bat"
+          };
+    } else {
+      command =
+          new String[] {
+            "sh", getCliPath() + File.separator + "tools" + File.separator + "export-csv.sh"
+          };
+    }
   }
 
   @After
@@ -65,140 +65,61 @@ public class ExportCsvTestIT extends AbstractScript {
     EnvironmentUtils.cleanEnv();
   }
 
-  @Override
-  protected void testOnWindows(String[] output) throws IOException {
-
-    String dir = getCliPath();
-    ProcessBuilder builder =
-        new ProcessBuilder(
-            "cmd.exe",
-            "/c",
-            dir + File.separator + "tools" + File.separator + "export-csv.bat",
-            "-h",
-            "127.0.0.1",
-            "-p",
-            "6667",
-            "-u",
-            "root",
-            "-pw",
-            "root",
-            "-td",
-            "./target",
-            "-s",
-            SQL_FILE);
-    testOutput(builder, output);
-  }
-
-  @Override
-  protected void testOnUnix(String[] output) throws IOException {
-
-    String dir = getCliPath();
-    ProcessBuilder builder =
-        new ProcessBuilder(
-            "sh",
-            dir + File.separator + "tools" + File.separator + "export-csv.sh",
-            "-h",
-            "127.0.0.1",
-            "-p",
-            "6667",
-            "-u",
-            "root",
-            "-pw",
-            "root",
-            "-td",
-            "./target",
-            "-s",
-            SQL_FILE);
-    testOutput(builder, output);
-  }
-
-  private boolean generateSQLFile(String[] sql) {
-    BufferedWriter writer;
-    try {
-      writer = new BufferedWriter(new FileWriter(SQL_FILE));
-      writer.write("");
-      for (String s : sql) {
-        writer.write(s);
-        writer.newLine();
-      }
-      writer.flush();
-      writer.close();
-      return true;
-    } catch (IOException e) {
-      System.out.println("failed to create test csv");
+  @Test
+  public void testExport()
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+    String[] params = {"-td", "target/", "-q", "select c1,c2,c3 from root.test.t1"};
+    prepareData();
+    testMethod(params, null);
+    CSVParser parser = readCsvFile("target/dump0.csv");
+    String[] realRecords = {
+      "root.test.t1.c1,root.test.t1.c2,root.test.t1.c3", "1.0,\"\"abc\",aa\",\"abbe's\""
+    };
+    List<CSVRecord> records = parser.getRecords();
+    for (int i = 0; i < records.size(); i++) {
+      String record = StringUtils.join(records.get(i).toList(), ',');
+      record = record.substring(record.indexOf(',') + 1);
+      assertEquals(realRecords[i], record);
     }
-    return false;
   }
 
   @Test
-  public void testRawDataQuery()
-      throws IOException, StatementExecutionException, IoTDBConnectionException {
-    final String[] expectCsv =
-        new String[] {
-          "Time,root.sg1.d1.s3,root.sg1.d1.s1,root.sg1.d1.s2", "abbe's,1.0,\"\\\"abc\\\",aa\""
-        };
+  public void testWithDataType()
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+    String[] params = {
+      "-td", "target/", "-datatype", "true", "-q", "select c1,c2,c3 from root.test.t1"
+    };
     prepareData();
-    String os = System.getProperty("os.name").toLowerCase();
-    String[] sql = {"select * from root"};
-    assertTrue(generateSQLFile(sql));
-    if (os.startsWith("windows")) {
-      testOnWindows(output);
-    } else {
-      testOnUnix(output);
-    }
-    FileReader fileReader = new FileReader(EXPORT_FILE);
-    BufferedReader br = new BufferedReader(fileReader);
-    String line = br.readLine();
-    int i = 0;
-    while (line != null) {
-      if (i == 0) {
-        assertEquals(expectCsv[i], line);
-      } else {
-        String lineWithoutTime = line.substring(line.indexOf(',') + 1);
-        assertEquals(expectCsv[i], lineWithoutTime);
-      }
-      i++;
-      line = br.readLine();
-    }
-    File file = new File(EXPORT_FILE);
-    if (file.exists()) {
-      file.delete();
+    testMethod(params, null);
+    CSVParser parser = readCsvFile("target/dump0.csv");
+    String[] realRecords = {
+      "root.test.t1.c1(FLOAT),root.test.t1.c2(TEXT),root.test.t1.c3(TEXT)",
+      "1.0,\"\"abc\",aa\",\"abbe's\""
+    };
+    List<CSVRecord> records = parser.getRecords();
+    for (int i = 0; i < records.size(); i++) {
+      String record = StringUtils.join(records.get(i).toList(), ',');
+      record = record.substring(record.indexOf(',') + 1);
+      assertEquals(realRecords[i], record);
     }
   }
 
   @Test
   public void testAggregationQuery()
-      throws StatementExecutionException, IoTDBConnectionException, IOException {
-    final String[] expectCsv =
-        new String[] {
-          "Time,count(root.sg1.d1.s3),count(root.sg1.d1.s1),count(root.sg1.d1.s2)", "1,1,1"
-        };
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+    String[] params = {
+      "-td", "target/", "-q", "select count(c1),count(c2),count(c3) from root.test.t1"
+    };
     prepareData();
-    String os = System.getProperty("os.name").toLowerCase();
-    String[] sql = {"select count(*) from root"};
-    generateSQLFile(sql);
-    if (os.startsWith("windows")) {
-      testOnWindows(output);
-    } else {
-      testOnUnix(output);
-    }
-    FileReader fileReader = new FileReader(EXPORT_FILE);
-    BufferedReader br = new BufferedReader(fileReader);
-    String line = br.readLine();
-    int i = 0;
-    while (line != null) {
-      if (i == 0) {
-        assertEquals(expectCsv[i], line);
-      } else {
-        String lineWithoutTime = line.substring(line.indexOf(',') + 1);
-        assertEquals(expectCsv[i], lineWithoutTime);
-      }
-      i++;
-      line = br.readLine();
-    }
-    File file = new File(EXPORT_FILE);
-    if (file.exists()) {
-      file.delete();
+    testMethod(params, null);
+    CSVParser parser = readCsvFile("target/dump0.csv");
+    String[] realRecords = {
+      "count(root.test.t1.c1),count(root.test.t1.c2),count(root.test.t1.c3)", "1,1,1"
+    };
+    List<CSVRecord> records = parser.getRecords();
+    for (int i = 0; i < records.size(); i++) {
+      String record = StringUtils.join(records.get(i).toList(), ',');
+      assertEquals(realRecords[i], record);
     }
   }
 
@@ -206,11 +127,11 @@ public class ExportCsvTestIT extends AbstractScript {
     Session session = new Session("127.0.0.1", 6667, "root", "root");
     session.open();
 
-    String deviceId = "root.sg1.d1";
+    String deviceId = "root.test.t1";
     List<String> measurements = new ArrayList<>();
-    measurements.add("s1");
-    measurements.add("s2");
-    measurements.add("s3");
+    measurements.add("c1");
+    measurements.add("c2");
+    measurements.add("c3");
 
     List<String> values = new ArrayList<>();
     values.add("1.0");
diff --git a/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ImportCsvTestIT.java b/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ImportCsvTestIT.java
index e2dfb4b..29facf7 100644
--- a/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ImportCsvTestIT.java
+++ b/cross-tests/src/test/java/org/apache/iotdb/cross/tests/tools/importCsv/ImportCsvTestIT.java
@@ -22,10 +22,13 @@ package org.apache.iotdb.cross.tests.tools.importCsv;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 
+import org.apache.commons.csv.CSVRecord;
+import org.apache.commons.lang3.StringUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.awt.*;
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
@@ -36,7 +39,6 @@ import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
@@ -44,8 +46,6 @@ import static org.junit.Assert.assertTrue;
 
 public class ImportCsvTestIT extends AbstractScript {
 
-  private final String CSV_FILE = "target" + File.separator + "test.csv";
-
   private static final String[] sqls =
       new String[] {
         "SET STORAGE GROUP TO root.fit.d1",
@@ -58,54 +58,53 @@ public class ImportCsvTestIT extends AbstractScript {
         "CREATE TIMESERIES root.fit.p.s1 WITH DATATYPE=INT32,ENCODING=RLE",
       };
 
-  private final String[] noDataOutputForWindows = {
+  private final String[] noDataOutput = {
     "````````````````````````````````````````````````",
     "Starting IoTDB Client Import Script",
     "````````````````````````````````````````````````",
     "Start to import data from: test.csv",
-    "",
-    "Import from: test.csv   0% │         │ 0/2 (0:00:00 / ?) Importing...",
-    "Meet error when insert csv because 411: sub plan are empty."
-  };
-
-  private final String[] noDataOutputForUnix = {
-    "------------------------------------------",
-    "Starting IoTDB Client Import Script",
-    "------------------------------------------",
-    "Start to import data from: test.csv",
-    "",
-    "Import from: test.csv   0% │         │ 0/2 (0:00:00 / ?) Importing...",
-    "Meet error when insert csv because 411: sub plan are empty."
+    "No records!"
   };
 
-  private final String[] hasDataOutputForWindows = {
+  private final String[] noHeaderOutput = {
     "````````````````````````````````````````````````",
     "Starting IoTDB Client Import Script",
     "````````````````````````````````````````````````",
-    "Start to import data from: test.csv",
-    "",
-    "Import from: test.csv",
-    "Import from: test.csv 100%"
+    "No headers!"
   };
 
-  private final String[] hasDataOutputForUnix = {
-    "------------------------------------------",
+  private final String[] emptyOutput = {
+    "````````````````````````````````````````````````",
     "Starting IoTDB Client Import Script",
-    "------------------------------------------",
-    "Start to import data from: test.csv",
-    "",
-    "Import from: test.csv",
-    "Import from: test.csv 100%"
+    "````````````````````````````````````````````````",
+    "Empty file!"
   };
 
   @Before
   public void setUp() {
+    // start an IotDB server environment
     EnvironmentUtils.closeStatMonitor();
     EnvironmentUtils.envSetUp();
+    // choose an execute command by system.
+    String os = System.getProperty("os.name").toLowerCase();
+    if (os.startsWith("windows")) {
+      command =
+          new String[] {
+            "cmd.exe",
+            "/c",
+            getCliPath() + File.separator + "tools" + File.separator + "import-csv.bat"
+          };
+    } else {
+      command =
+          new String[] {
+            "sh", getCliPath() + File.separator + "tools" + File.separator + "import-csv.sh"
+          };
+    }
   }
 
   @After
   public void tearDown() throws Exception {
+    // shutdown IotDB server environment
     EnvironmentUtils.cleanEnv();
   }
 
@@ -124,31 +123,86 @@ public class ImportCsvTestIT extends AbstractScript {
     }
   }
 
+  /**
+   * test the situation that the schema has not been created and CSV file has no records
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testImportNoRecordsCSV() throws IOException {
+    assertTrue(generateTestCSV(false, false, true, false, false));
+    String[] params = {"-f", CSV_FILE};
+    testMethod(params, noDataOutput);
+    File file = new File(CSV_FILE);
+    if (file.exists()) {
+      file.delete();
+    }
+  }
+
+  /**
+   * test the situation that the CSV file has no headers
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testNoHeader() throws IOException {
+    assertTrue(generateTestCSV(false, true, false, false, false));
+    String[] params = {"-f", CSV_FILE};
+    testMethod(params, noHeaderOutput);
+  }
+
+  /**
+   * test the situation that the CSV file is an empty file
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testEmptyCSV() throws IOException {
+    assertTrue(generateTestCSV(true, false, false, false, false));
+    String[] params = {"-f", CSV_FILE};
+    testMethod(params, emptyOutput);
+  }
+
+  /**
+   * test the situation that the schema has been created and CSV file has no problem
+   *
+   * @throws IOException
+   */
   @Test
   public void test() throws IOException, ClassNotFoundException {
     createSchema();
-    String os = System.getProperty("os.name").toLowerCase();
-    assertTrue(generateTestCSV());
-    if (os.startsWith("windows")) {
-      testOnWindows(hasDataOutputForWindows);
-    } else {
-      testOnUnix(hasDataOutputForUnix);
-    }
+    assertTrue(generateTestCSV(false, false, false, false, false));
+    String[] params = {"-f", CSV_FILE};
+    testMethod(params, null);
     File file = new File(CSV_FILE);
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection =
+            DriverManager.getConnection(
+                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      if (statement.execute("select * from root")) {
+        ResultSet resultSet = statement.getResultSet();
+        testResult(resultSet, 6, 3);
+        resultSet.close();
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
     if (file.exists()) {
       file.delete();
     }
   }
 
+  /**
+   * test the situation that the schema has not been created and CSV file has no problem
+   *
+   * @throws IOException
+   */
   @Test
   public void testWithoutCreateSchema() throws IOException, ClassNotFoundException {
-    String os = System.getProperty("os.name").toLowerCase();
-    assertTrue(generateTestCSV());
-    if (os.startsWith("windows")) {
-      testOnWindows(hasDataOutputForWindows);
-    } else {
-      testOnUnix(hasDataOutputForUnix);
-    }
+    assertTrue(generateTestCSV(false, false, false, false, false));
+    String[] params = {"-f", CSV_FILE};
+    testMethod(params, null);
     File file = new File(CSV_FILE);
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
@@ -158,6 +212,7 @@ public class ImportCsvTestIT extends AbstractScript {
       if (statement.execute("select * from root")) {
         ResultSet resultSet = statement.getResultSet();
         testResult(resultSet, 6, 3);
+        resultSet.close();
       }
     } catch (Exception e) {
       e.printStackTrace();
@@ -167,24 +222,57 @@ public class ImportCsvTestIT extends AbstractScript {
     }
   }
 
+  /**
+   * test the situation that the schema has not been created and CSV file has no data type
+   *
+   * @throws IOException
+   * @throws ClassNotFoundException
+   */
   @Test
-  public void testBigCsvFile() throws IOException, ClassNotFoundException {
-    String os = System.getProperty("os.name").toLowerCase();
-    assertTrue(generateBigCsvFile());
-    if (os.startsWith("windows")) {
-      testOnWindows(hasDataOutputForWindows);
-    } else {
-      testOnUnix(hasDataOutputForUnix);
+  public void testWithDataType() throws IOException, ClassNotFoundException {
+    assertTrue(generateTestCSV(false, false, false, false, true));
+    String[] params = {"-f", CSV_FILE};
+    testMethod(params, null);
+    File file = new File(CSV_FILE);
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection =
+            DriverManager.getConnection(
+                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      if (statement.execute("select * from root")) {
+        ResultSet resultSet = statement.getResultSet();
+        testResult(resultSet, 6, 3);
+        resultSet.close();
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
     }
+    if (file.exists()) {
+      file.delete();
+    }
+  }
+
+  /**
+   * test the situation that the schema has not been created and CSV file has no data type
+   *
+   * @throws IOException
+   * @throws ClassNotFoundException
+   */
+  @Test
+  public void testWithException() throws IOException, ClassNotFoundException {
+    assertTrue(generateTestCSV(false, false, false, true, true));
+    String[] params = {"-f", CSV_FILE};
+    testMethod(params, null);
     File file = new File(CSV_FILE);
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection =
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      if (statement.execute("select s1 from root.fit.d1")) {
+      if (statement.execute("select * from root")) {
         ResultSet resultSet = statement.getResultSet();
-        testResult(resultSet, 2, 25000);
+        testResult(resultSet, 6, 3);
+        resultSet.close();
       }
     } catch (Exception e) {
       e.printStackTrace();
@@ -192,8 +280,23 @@ public class ImportCsvTestIT extends AbstractScript {
     if (file.exists()) {
       file.delete();
     }
+    // check the failed file
+    List<CSVRecord> records = readCsvFile(CSV_FILE + ".failed").getRecords();
+    String[] realRecords = {
+      "Time,root.fit.d1.s1(INT32),root.fit.d1.s2(TEXT),root.fit.d2.s1(INT32),root.fit.d2.s3(INT32),root.fit.p.s1(INT32)",
+      "1,100,\"hello\",200,\"300\",400"
+    };
+    for (int i = 0; i < records.size(); i++) {
+      String record = StringUtils.join(records.get(i).toList(), ',');
+      assertEquals(realRecords[i], record);
+    }
   }
 
+  /**
+   * test whether the shape of data is correct
+   *
+   * @throws IOException
+   */
   private static void testResult(
       ResultSet resultSet, int expectedColumnNumber, int expectedRowNumber) throws SQLException {
     if (resultSet != null) {
@@ -208,72 +311,80 @@ public class ImportCsvTestIT extends AbstractScript {
     }
   }
 
-  @Test
-  public void testImportHeaderCSV() throws IOException {
-    String os = System.getProperty("os.name").toLowerCase();
-    assertTrue(generateHeaderTestCSV());
-    if (os.startsWith("windows")) {
-      testOnWindows(noDataOutputForWindows);
+  /**
+   * generate the test CSV file by setting parameters
+   *
+   * @param isEmpty
+   * @param isHeaderEmpty
+   * @param isRecordsEmpty
+   * @param isException
+   * @param dataType
+   * @return
+   */
+  private boolean generateTestCSV(
+      Boolean isEmpty,
+      Boolean isHeaderEmpty,
+      Boolean isRecordsEmpty,
+      Boolean isException,
+      Boolean dataType) {
+    String[] csvText;
+    if (isEmpty) {
+      csvText = new String[] {};
     } else {
-      testOnUnix(noDataOutputForUnix);
-    }
-    File file = new File(CSV_FILE);
-    if (file.exists()) {
-      file.delete();
-    }
-  }
-
-  private boolean generateTestCSV() {
-    String[] csvText = {
-      "Time,root.fit.d1.s1,root.fit.d1.s2,root.fit.d2.s1,root.fit.d2.s3,root.fit.p.s1",
-      "1,100,'hello',200,300,400",
-      "2,500,'',600,700,800",
-      "3,900,'Io\"TDB',1000,1100,1200"
-    };
-    BufferedWriter writer;
-    try {
-      writer = new BufferedWriter(new FileWriter(CSV_FILE));
-      writer.write("");
-      for (String s : csvText) {
-        writer.write(s);
-        writer.newLine();
-      }
-      writer.flush();
-      writer.close();
-      return true;
-    } catch (IOException e) {
-      System.out.println("failed to create test csv");
-    }
-    return false;
-  }
-
-  private boolean generateBigCsvFile() {
-    List<String> csvText = new ArrayList<>();
-    csvText.add("Time,root.fit.d1.s1,root.fit.d1.s2,root.fit.d2.s1");
-    for (int i = 0; i < 25000; i++) {
-      csvText.add(i + "," + i + "," + i + "," + i);
-    }
-    BufferedWriter writer;
-    try {
-      writer = new BufferedWriter(new FileWriter(CSV_FILE));
-      writer.write("");
-      for (String s : csvText) {
-        writer.write(s);
-        writer.newLine();
+      if (isHeaderEmpty) {
+        csvText =
+            new String[] {
+              "1,100,\"hello\",200,300,400",
+              "2,500,\"\",600,700,800",
+              "3,900,\"Io\"TDB\",1000,1100,1200"
+            };
+      } else {
+        if (isRecordsEmpty) {
+          csvText =
+              new String[] {
+                "Time,root.fit.d1.s1(INT32),root.fit.d1.s2(TEXT),root.fit.d2.s1(INT32),root.fit.d2.s3(INT32),root.fit.p.s1(INT32)"
+              };
+        } else {
+          if (dataType) {
+            if (isException) {
+              csvText =
+                  new String[] {
+                    "Time,root.fit.d1.s1(INT32),root.fit.d1.s2(TEXT),root.fit.d2.s1(INT32),root.fit.d2.s3(INT32),root.fit.p.s1(INT32)",
+                    "1,100,\"hello\",200,\"300\",400",
+                    "2,500,\"\",600,700,800",
+                    "3,900,\"Io\"TDB\",1000,1100,1200"
+                  };
+            } else {
+              csvText =
+                  new String[] {
+                    "Time,root.fit.d1.s1(INT32),root.fit.d1.s2(TEXT),root.fit.d2.s1(INT32),root.fit.d2.s3(INT32),root.fit.p.s1(INT32)",
+                    "1,100,\"hello\",200,300,400",
+                    "2,500,\"\",600,700,800",
+                    "3,900,\"Io\"TDB\",1000,1100,1200"
+                  };
+            }
+          } else {
+            if (isException) {
+              csvText =
+                  new String[] {
+                    "Time,root.fit.d1.s1,root.fit.d1.s2,root.fit.d2.s1,root.fit.d2.s3,root.fit.p.s1",
+                    "1,100,\"hello\",200,\"300\",400",
+                    "2,500,\"\",600,700,800",
+                    "3,900,\"Io\"TDB\",1000,1100,1200"
+                  };
+            } else {
+              csvText =
+                  new String[] {
+                    "Time,root.fit.d1.s1,root.fit.d1.s2,root.fit.d2.s1,root.fit.d2.s3,root.fit.p.s1",
+                    "1,100,\"hello\",200,300,400",
+                    "2,500,\"\",600,700,800",
+                    "3,900,\"Io\"TDB\",1000,1100,1200"
+                  };
+            }
+          }
+        }
       }
-      writer.flush();
-      writer.close();
-      return true;
-    } catch (IOException e) {
-      System.out.println("failed to create test csv");
     }
-    return false;
-  }
-
-  private boolean generateHeaderTestCSV() {
-    String[] csvText = {
-      "Time,root.fit.d1.\"s1\",root.fit.d1.s2,root.fit.d2.s1,root.fit.d2.s3,root.fit.p.s1"
-    };
     BufferedWriter writer;
     try {
       writer = new BufferedWriter(new FileWriter(CSV_FILE));
@@ -290,46 +401,4 @@ public class ImportCsvTestIT extends AbstractScript {
     }
     return false;
   }
-
-  @Override
-  protected void testOnWindows(String[] output) throws IOException {
-    String dir = getCliPath();
-    ProcessBuilder builder =
-        new ProcessBuilder(
-            "cmd.exe",
-            "/c",
-            dir + File.separator + "tools" + File.separator + "import-csv.bat",
-            "-h",
-            "127.0.0.1",
-            "-p",
-            "6667",
-            "-u",
-            "root",
-            "-pw",
-            "root",
-            "-f",
-            CSV_FILE);
-    testOutput(builder, output);
-  }
-
-  @Override
-  protected void testOnUnix(String[] output) throws IOException {
-
-    String dir = getCliPath();
-    ProcessBuilder builder =
-        new ProcessBuilder(
-            "sh",
-            dir + File.separator + "tools" + File.separator + "import-csv.sh",
-            "-h",
-            "127.0.0.1",
-            "-p",
-            "6667",
-            "-u",
-            "root",
-            "-pw",
-            "root",
-            "-f",
-            CSV_FILE);
-    testOutput(builder, output);
-  }
 }
diff --git a/docs/UserGuide/System-Tools/CSV-Tool.md b/docs/UserGuide/System-Tools/CSV-Tool.md
index 56d47e4..eefc05c 100644
--- a/docs/UserGuide/System-Tools/CSV-Tool.md
+++ b/docs/UserGuide/System-Tools/CSV-Tool.md
@@ -19,126 +19,192 @@
 
 -->
 
-## CSV Tool
+# CSV Tool
 
 The CSV tool can help you import data in CSV format to IoTDB or export data from IoTDB to a CSV file.
 
+## Usage of export-csv.sh
 
-### Usage of import-csv.sh
+### Syntax
 
-#### Create metadata (optional)
+```shell
+# Unix/OS X
+> tools/export-csv.sh  -h <ip> -p <port> -u <username> -pw <password> -td <directory> [-tf <time-format> -datatype <true/false> -q <query command> -s <sql file>]
 
-```sql
-SET STORAGE GROUP TO root.fit.d1;
-SET STORAGE GROUP TO root.fit.d2;
-SET STORAGE GROUP TO root.fit.p;
-CREATE TIMESERIES root.fit.d1.s1 WITH DATATYPE=INT32,ENCODING=RLE;
-CREATE TIMESERIES root.fit.d1.s2 WITH DATATYPE=TEXT,ENCODING=PLAIN;
-CREATE TIMESERIES root.fit.d2.s1 WITH DATATYPE=INT32,ENCODING=RLE;
-CREATE TIMESERIES root.fit.d2.s3 WITH DATATYPE=INT32,ENCODING=RLE;
-CREATE TIMESERIES root.fit.p.s1 WITH DATATYPE=INT32,ENCODING=RLE;
+# Windows
+> tools\export-csv.bat -h <ip> -p <port> -u <username> -pw <password> -td <directory> [-tf <time-format> -datatype <true/false> -q <query command> -s <sql file>]
 ```
 
-IoTDB has the ability of type inference, so it is not necessary to create metadata before data import. However, we still recommend creating metadata before importing data using the CSV import tool, as this can avoid unnecessary type conversion errors.
-
-
+Description:
 
-#### Sample CSV file to be imported
-
-```sql
-Time,root.fit.d1.s1,root.fit.d1.s2,root.fit.d2.s1,root.fit.d2.s3,root.fit.p.s1
-1,100,hello,200,300,400
-2,500,world,600,700,800
-3,900,"hello, \"world\"",1000,1100,1200
-```
+* `-datatype`:
+  - true (by default): print the data type of timesries in the head line of CSV file. i.e., `Time, root.sg1.d1.s1(INT32), root.sg1.d1.s2(INT64)`.
+  - false: only print the timeseries name in the head line of the CSV file. i.e., `Time, root.sg1.d1.s1 , root.sg1.d1.s2`
+* `-q <query command>`:
+  - specifying a query command that you want to execute
+  - example: `select * from root limit 100`, or `select * from root limit 100 align by device`
+* `-s <sql file>`:
+  - specifying a SQL file which can consist of more than one sql. If there are multiple SQLs in one SQL file, the SQLs should be separated by line breaks. And, for each SQL, a output CSV file will be generated.
+* `-td <directory>`:
+  - specifying  the directory that the data will be exported
+* `-tf <time-format>`:
+  - specifying a time format that you want. The time format have to obey [ISO 8601](https://calendars.wikia.org/wiki/ISO_8601) standard. If you want to save the time as the timestamp, then setting `-tf timestamp`
+  - example: `-tf yyyy-MM-dd\ HH:mm:ss` or `-tf timestamp`
 
+More, if you don't use one of `-s` and `-q`, you need to enter some queries after running the export script. The results of the different query will be saved to different CSV files.
 
-#### Syntax
+### example
 
 ```shell
 # Unix/OS X
-> tools/import-csv.sh -h <ip> -p <port> -u <username> -pw <password> -f <xxx.csv>
+> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./
+# Or
+> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss
+# or
+> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -q "select * from root"
+# Or
+> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -s sql.txt
+# Or
+> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss -s sql.txt
 
 # Windows
-> tools\import-csv.bat -h <ip> -p <port> -u <username> -pw <password> -f <xxx.csv>
+> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./
+# Or
+> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss
+# or
+> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -q "select * from root"
+# Or
+> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -s sql.txt
+# Or
+> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss -s sql.txt
 ```
 
+### Sample SQL file
+
+```sql
+select * from root;
+select * from root align by device;
+```
 
+The result of `select * from root`
 
-#### Example
+```sql
+Time,root.ln.wf04.wt04.status(BOOLEAN),root.ln.wf03.wt03.hardware(TEXT),root.ln.wf02.wt02.status(BOOLEAN),root.ln.wf02.wt02.hardware(TEXT),root.ln.wf01.wt01.hardware(TEXT),root.ln.wf01.wt01.status(BOOLEAN)
+1970-01-01T08:00:00.001+08:00,true,"v1",true,"v1",v1,true
+1970-01-01T08:00:00.002+08:00,true,"v1",,,,true
+```
 
-```shell
-# Unix/OS X
-> tools/import-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv
+The result of `select * from root align by device`
 
-# Windows
-> tools\import-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv
+```sql
+Time,Device,hardware(TEXT),status(BOOLEAN)
+1970-01-01T08:00:00.001+08:00,root.ln.wf01.wt01,"v1",true
+1970-01-01T08:00:00.002+08:00,root.ln.wf01.wt01,,true
+1970-01-01T08:00:00.001+08:00,root.ln.wf02.wt02,"v1",true
+1970-01-01T08:00:00.001+08:00,root.ln.wf03.wt03,"v1",
+1970-01-01T08:00:00.002+08:00,root.ln.wf03.wt03,"v1",
+1970-01-01T08:00:00.001+08:00,root.ln.wf04.wt04,,true
+1970-01-01T08:00:00.002+08:00,root.ln.wf04.wt04,,true
 ```
 
+The data of boolean type signed by `true` and `false` without double quotes. And the text data will be enclosed in double quotes.
 
+### Note
 
-#### Note
+Note that if fields exported by the export tool have the following special characters:
 
-Note that the following special characters in fields need to be checked before importing:
+1. `,`: the field will be escaped by `\`.
 
-1. `,` : fields containing `,` should be quoted by a pair of `"` or a pair of `'`.
-2. `"` : `"` in fields should be replaced by `\"` or fields should be enclosed by `'`.
-3. `'` : `'` in fields should be replaced by `\'` or fields should be enclosed by `"`.
-4. you can input time format like `yyyy-MM-dd'T'HH:mm:ss`, `yyy-MM-dd HH:mm:ss`, or `yyyy-MM-dd'T'HH:mm:ss.SSSZ`.
+## Usage of import-csv.sh
 
+### Create metadata (optional)
 
+```sql
+SET STORAGE GROUP TO root.fit.d1;
+SET STORAGE GROUP TO root.fit.d2;
+SET STORAGE GROUP TO root.fit.p;
+CREATE TIMESERIES root.fit.d1.s1 WITH DATATYPE=INT32,ENCODING=RLE;
+CREATE TIMESERIES root.fit.d1.s2 WITH DATATYPE=TEXT,ENCODING=PLAIN;
+CREATE TIMESERIES root.fit.d2.s1 WITH DATATYPE=INT32,ENCODING=RLE;
+CREATE TIMESERIES root.fit.d2.s3 WITH DATATYPE=INT32,ENCODING=RLE;
+CREATE TIMESERIES root.fit.p.s1 WITH DATATYPE=INT32,ENCODING=RLE;
+```
 
-### Usage of export-csv.sh
+IoTDB has the ability of type inference, so it is not necessary to create metadata before data import. However, we still recommend creating metadata before importing data using the CSV import tool, as this can avoid unnecessary type conversion errors.
 
-#### Syntax
-```shell
-# Unix/OS X
-> tools/export-csv.sh -h <ip> -p <port> -u <username> -pw <password> -td <directory> [-tf <time-format> -s <sqlfile>]
+### Sample CSV file to be imported
 
-# Windows
-> tools\export-csv.bat -h <ip> -p <port> -u <username> -pw <password> -td <directory> [-tf <time-format> -s <sqlfile>]
-```
+The data aligned by time, and headers without data type.
 
-After running the export script, you need to enter some queries or specify some SQL files. If there are multiple SQLs in one SQL file, the SQLs should be separated by line breaks.
+```sql
+Time,root.test.t1.str,root.test.t2.str,root.test.t2.int
+1970-01-01T08:00:00.001+08:00,"123hello world","123\,abc",100
+1970-01-01T08:00:00.002+08:00,"123",,
+```
 
+The data aligned by time, and headers with data type.
 
+```sql
+Time,root.test.t1.str(TEXT),root.test.t2.str(TEXT),root.test.t2.int(INT32)
+1970-01-01T08:00:00.001+08:00,"123hello world","123\,abc",100
+1970-01-01T08:00:00.002+08:00,"123",,
+```
 
-#### Sample SQL file
+The data aligned by device, and headers without data type.
 
 ```sql
-select * from root.fit.d1
-select * from root.sg1.d1
+Time,Device,str,int
+1970-01-01T08:00:00.001+08:00,root.test.t1,"123hello world",
+1970-01-01T08:00:00.002+08:00,root.test.t1,"123",
+1970-01-01T08:00:00.001+08:00,root.test.t2,"123\,abc",100
 ```
 
+The data aligned by device,  and headers with data type.
 
-#### Example
+```sql
+Time,Device,str(TEXT),int(INT32)
+1970-01-01T08:00:00.001+08:00,root.test.t1,"123hello world",
+1970-01-01T08:00:00.002+08:00,root.test.t1,"123",
+1970-01-01T08:00:00.001+08:00,root.test.t2,"123\,abc",100
+```
+
+### Syntax
 
 ```shell
 # Unix/OS X
-> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./
-# Or
-> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss
-# Or
-> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -s sql.txt
-# Or
-> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss -s sql.txt
+> tools/import-csv.sh -h <ip> -p <port> -u <username> -pw <password> -f <xxx.csv> [-fd <./failedDirectory>]
 
 # Windows
-> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./
-# Or
-> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss
-# Or
-> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -s sql.txt
-# Or
-> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss -s sql.txt
+> tools\import-csv.bat -h <ip> -p <port> -u <username> -pw <password> -f <xxx.csv> [-fd <./failedDirectory>]
 ```
 
+Description:
 
+* `-f`:
+  - the CSV file that you want to import
+  - example: `-f filename.csv`
 
-#### Note
+* `-fd`:
+  - specifying a directory to save files which save failed lines. If you don't use this parameter, the failed file will be saved at original directory, and the filename will be the source filename with suffix `.failed`.
+  - example: `-fd ./failed/`
 
-Note that if fields exported by the export tool have the following special characters:
+### Example
 
-1. `,`: the field will be enclosed by `"`.
-2. `"`: the field will be enclosed by `"` and the original characters `"` in the field will be replaced by `\"`.
+```sh
+# Unix/OS X
+> tools/import-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv -fd ./failed
+# or
+> tools/import-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv -fd ./failed
+
+# Windows
+> tools\import-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv
+# or
+> tools/import-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv -fd .\failed
+```
+
+### Note
+
+Note that the following special characters in fields need to be checked before importing:
 
+1. `,` : fields containing `,` should be escaped by `\`.
+2. you can input time format like `yyyy-MM-dd'T'HH:mm:ss`, `yyy-MM-dd HH:mm:ss`, or `yyyy-MM-dd'T'HH:mm:ss.SSSZ`.
\ No newline at end of file
diff --git a/docs/zh/UserGuide/System-Tools/CSV-Tool.md b/docs/zh/UserGuide/System-Tools/CSV-Tool.md
index 97052e8..1afec9b 100644
--- a/docs/zh/UserGuide/System-Tools/CSV-Tool.md
+++ b/docs/zh/UserGuide/System-Tools/CSV-Tool.md
@@ -19,126 +19,185 @@
 
 -->
 
-## 导入导出 CSV
+# CSV Tool
 
-CSV工具可帮您将CSV格式的数据导入到IoTDB或者将数据从IoTDB导出到CSV文件。
+CSV 工具可帮您将 CSV 格式的数据导入到 IoTDB 或者将数据从 IoTDB 导出到 CSV 文件。
 
+### 使用 export-csv.sh
 
-### 使用 import-csv.sh
+#### 运行方法
 
-#### 创建元数据(可选)
+```shell
+# Unix/OS X
+> tools/export-csv.sh  -h <ip> -p <port> -u <username> -pw <password> -td <directory> [-tf <time-format> -datatype <true/false> -q <query command> -s <sql file>]
 
-```sql
-SET STORAGE GROUP TO root.fit.d1;
-SET STORAGE GROUP TO root.fit.d2;
-SET STORAGE GROUP TO root.fit.p;
-CREATE TIMESERIES root.fit.d1.s1 WITH DATATYPE=INT32,ENCODING=RLE;
-CREATE TIMESERIES root.fit.d1.s2 WITH DATATYPE=TEXT,ENCODING=PLAIN;
-CREATE TIMESERIES root.fit.d2.s1 WITH DATATYPE=INT32,ENCODING=RLE;
-CREATE TIMESERIES root.fit.d2.s3 WITH DATATYPE=INT32,ENCODING=RLE;
-CREATE TIMESERIES root.fit.p.s1 WITH DATATYPE=INT32,ENCODING=RLE;
+# Windows
+> tools\export-csv.bat -h <ip> -p <port> -u <username> -pw <password> -td <directory> [-tf <time-format> -datatype <true/false> -q <query command> -s <sql file>]
 ```
-IoTDB具有类型推断的能力,因此在数据导入前创建元数据不是必须的。但我们仍然推荐在使用CSV导入工具导入数据前创建元数据,因为这可以避免不必要的类型转换错误。
 
+参数:
 
-#### 待导入 CSV 文件示例
+* `-datatype`:
+  - true (默认): 在CSV文件的header中时间序列的后面打印出对应的数据类型。例如:`Time, root.sg1.d1.s1(INT32), root.sg1.d1.s2(INT64)`.
+  - false: 只在CSV的header中打印出时间序列的名字, `Time, root.sg1.d1.s1 , root.sg1.d1.s2`
+* `-q <query command>`:
+  - 在命令中直接指定想要执行的查询语句。
+  - 例如: `select * from root limit 100`, or `select * from root limit 100 align by device`
+* `-s <sql file>`:
+  - 指定一个SQL文件,里面包含一条或多条SQL语句。如果一个SQL文件中包含多条SQL语句,SQL语句之间应该用换行符进行分割。每一条SQL语句对应一个输出的CSV文件。
+* `-td <directory>`:
+  - 为导出的CSV文件指定输出路径。
+* `-tf <time-format>`:
+  - 指定一个你想要得到的时间格式。时间格式必须遵守[ISO 8601](https://calendars.wikia.org/wiki/ISO_8601)标准。如果说你想要以时间戳来保存时间,那就设置为`-tf timestamp`。
+  - 例如: `-tf yyyy-MM-dd\ HH:mm:ss` or `-tf timestamp`
 
-```sql
-Time,root.fit.d1.s1,root.fit.d1.s2,root.fit.d2.s1,root.fit.d2.s3,root.fit.p.s1
-1,100,hello,200,300,400
-2,500,world,600,700,800
-3,900,"hello, \"world\"",1000,1100,1200
-```
+除此之外,如果你没有使用`-s`和`-q`参数,在导出脚本被启动之后你需要按照程序提示输入查询语句,不同的查询结果会被保存到不同的CSV文件中。
 
-
-#### 运行方法
+### 运行示例
 
 ```shell
 # Unix/OS X
-> tools/import-csv.sh -h <ip> -p <port> -u <username> -pw <password> -f <xxx.csv>
+> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./
+# Or
+> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss
+# or
+> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -q "select * from root"
+# Or
+> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -s sql.txt
+# Or
+> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss -s sql.txt
 
 # Windows
-> tools\import-csv.bat -h <ip> -p <port> -u <username> -pw <password> -f <xxx.csv>
+> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./
+# Or
+> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss
+# or
+> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -q "select * from root"
+# Or
+> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -s sql.txt
+# Or
+> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss -s sql.txt
 ```
 
+#### SQL 文件示例
 
-#### 运行示例
+```sql
+select * from root;
+select * from root align by device;
+```
 
-```shell
-# Unix/OS X
-> tools/import-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv
+`select * from root`的执行结果:
 
-# Windows
-> tools\import-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv
+```sql
+Time,root.ln.wf04.wt04.status(BOOLEAN),root.ln.wf03.wt03.hardware(TEXT),root.ln.wf02.wt02.status(BOOLEAN),root.ln.wf02.wt02.hardware(TEXT),root.ln.wf01.wt01.hardware(TEXT),root.ln.wf01.wt01.status(BOOLEAN)
+1970-01-01T08:00:00.001+08:00,true,"v1",true,"v1",v1,true
+1970-01-01T08:00:00.002+08:00,true,"v1",,,,true
 ```
 
+`select * from root align by device`的执行结果:
 
+```sql
+Time,Device,hardware(TEXT),status(BOOLEAN)
+1970-01-01T08:00:00.001+08:00,root.ln.wf01.wt01,"v1",true
+1970-01-01T08:00:00.002+08:00,root.ln.wf01.wt01,,true
+1970-01-01T08:00:00.001+08:00,root.ln.wf02.wt02,"v1",true
+1970-01-01T08:00:00.001+08:00,root.ln.wf03.wt03,"v1",
+1970-01-01T08:00:00.002+08:00,root.ln.wf03.wt03,"v1",
+1970-01-01T08:00:00.001+08:00,root.ln.wf04.wt04,,true
+1970-01-01T08:00:00.002+08:00,root.ln.wf04.wt04,,true
+```
 
-#### 注意事项
+布尔类型的数据用`true`或者`false`来表示,此处没有用双引号括起来。文本数据需要使用双引号括起来。
 
-注意,在导入数据前,需要特殊处理下列的字符:
+### 注意
 
-1. `,` : 包含`,`的字段需要使用单引号或者双引号括起来
-2. `"` : "字段中的`"`需要被替换成转义字符`\"`或者用`\'`将字段括起来。
-3. `'` : "字段中的`'`需要被替换成转义字符`\'`或者用`\"`将字段括起来。
-4. 你可以输入时间格式像`yyyy-MM-dd'T'HH:mm:ss`, `yyy-MM-dd HH:mm:ss`, 或者`yyyy-MM-dd'T'HH:mm:ss.SSSZ`.
+注意,如果导出字段存在如下特殊字符:
 
+1. `,`: 需要在字符前加`\`来进行转义。
 
+## 使用import-csv.sh
 
-### 使用 export-csv.sh
+### 创建元数据 (可选)
 
-#### 运行方法
+```sql
+SET STORAGE GROUP TO root.fit.d1;
+SET STORAGE GROUP TO root.fit.d2;
+SET STORAGE GROUP TO root.fit.p;
+CREATE TIMESERIES root.fit.d1.s1 WITH DATATYPE=INT32,ENCODING=RLE;
+CREATE TIMESERIES root.fit.d1.s2 WITH DATATYPE=TEXT,ENCODING=PLAIN;
+CREATE TIMESERIES root.fit.d2.s1 WITH DATATYPE=INT32,ENCODING=RLE;
+CREATE TIMESERIES root.fit.d2.s3 WITH DATATYPE=INT32,ENCODING=RLE;
+CREATE TIMESERIES root.fit.p.s1 WITH DATATYPE=INT32,ENCODING=RLE;
+```
 
-```shell
-# Unix/OS X
-> tools/export-csv.sh -h <ip> -p <port> -u <username> -pw <password> -td <directory> [-tf <time-format> -s <sqlfile>]
+IoTDB 具有类型推断的能力,因此在数据导入前创建元数据不是必须的。但我们仍然推荐在使用 CSV 导入工具导入数据前创建元数据,因为这可以避免不必要的类型转换错误。
 
-# Windows
-> tools\export-csv.bat -h <ip> -p <port> -u <username> -pw <password> -td <directory> [-tf <time-format> -s <sqlfile>]
-```
+#### 待导入 CSV 文件示例
 
-在运行导出脚本之后,您需要输入一些查询或指定一些SQL文件。如果在一个SQL文件中有多个SQL,SQL应该被换行符分割。
+通过时间对齐,并且header中不包含数据类型的数据。
 
+```sql
+Time,root.test.t1.str,root.test.t2.str,root.test.t2.int
+1970-01-01T08:00:00.001+08:00,"123hello world","123\,abc",100
+1970-01-01T08:00:00.002+08:00,"123",,
+```
+
+通过时间对齐,并且header中包含数据类型的数据。
 
+```sql
+Time,root.test.t1.str(TEXT),root.test.t2.str(TEXT),root.test.t2.int(INT32)
+1970-01-01T08:00:00.001+08:00,"123hello world","123\,abc",100
+1970-01-01T08:00:00.002+08:00,"123",,
+```
 
-#### SQL文件示例
+通过设备对齐,并且header中不包含数据类型的数据。
 
 ```sql
-select * from root.fit.d1
-select * from root.sg1.d1
+Time,Device,str,int1970-01-01T08:00:00.001+08:00,root.test.t1,"123hello world",1970-01-01T08:00:00.002+08:00,root.test.t1,"123",1970-01-01T08:00:00.001+08:00,root.test.t2,"123\,abc",100
 ```
 
+通过设备对齐,并且header中包含数据类型的数据。
 
-#### 运行示例
+```sql
+Time,Device,str(TEXT),int(INT32)1970-01-01T08:00:00.001+08:00,root.test.t1,"123hello world",1970-01-01T08:00:00.002+08:00,root.test.t1,"123",1970-01-01T08:00:00.001+08:00,root.test.t2,"123\,abc",100
+```
+
+### 运行方法
 
 ```shell
-# Unix/OS X
-> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./
-# Or
-> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss
-# Or
-> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -s sql.txt
-# Or
-> tools/export-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss -s sql.txt
+# Unix/OS X>
+tools/import-csv.sh -h <ip> -p <port> -u <username> -pw <password> -f <xxx.csv> [-fd <./failedDirectory>]
 
-# Windows
-> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./
-# Or
-> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss
-# Or
-> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -s sql.txt
-# Or
-> tools/export-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -td ./ -tf yyyy-MM-dd\ HH:mm:ss -s sql.txt
+# Windows>
+tools\import-csv.bat -h <ip> -p <port> -u <username> -pw <password> -f <xxx.csv> [-fd <./failedDirectory>]
 ```
 
+参数:
 
+* `-f`:
+  - 指定你想要导入的数据。
+  - 例如: `-f filename.csv`
 
-#### 注意事项
+* `-fd`:
+  - 指定一个目录来存放保存失败的行的文件,如果你没有指定这个参数,失败的文件将会被保存到源数据的目录中,然后文件名是源文件名加上`.failed`的后缀。
+  - example: `-fd ./failed/`
 
-注意,如果导出字段存在如下特殊字符:
+### Example
 
-1. `,` : 整个字段会被用`"`括起来。
-2. `"` : 整个字段会被用`"`括起来且`"`会被替换为`\"`。
+```sh
+# Unix/OS X>
+tools/import-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv -fd ./failed
+# or>
+tools/import-csv.sh -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv -fd ./failed
+# Windows>
+tools\import-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv
+# or>
+tools/import-csv.bat -h 127.0.0.1 -p 6667 -u root -pw root -f example-filename.csv -fd .\failed
+```
 
+### 注意
 
+注意,在导入数据前,需要特殊处理下列的字符:
 
+1. `,` :如果text类型的字段中包含`,`那么需要用`\`来进行转义。
+2. 你可以导入像`yyyy-MM-dd'T'HH:mm:ss`, `yyy-MM-dd HH:mm:ss`, 或者 `yyyy-MM-dd'T'HH:mm:ss.SSSZ`格式的时间。
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 81cc46d..2cd9ca0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1369,4 +1369,4 @@
             </properties>
         </profile>
     </profiles>
-</project>
+</project>
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/query/control/QueryTimeManager.java b/server/src/main/java/org/apache/iotdb/db/query/control/QueryTimeManager.java
index 59e646f..669fe1c 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/control/QueryTimeManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/control/QueryTimeManager.java
@@ -61,19 +61,20 @@ public class QueryTimeManager implements IService {
   }
 
   public void registerQuery(long queryId, long startTime, String sql, long timeout) {
-    final long finalTimeout = timeout == 0 ? config.getQueryTimeoutThreshold() : timeout;
     queryInfoMap.put(queryId, new QueryInfo(startTime, sql));
-    // submit a scheduled task to judge whether query is still running after timeout
-    ScheduledFuture<?> scheduledFuture =
-        executorService.schedule(
-            () -> {
-              killQuery(queryId);
-              logger.warn(
-                  String.format("Query is time out (%dms) with queryId %d", finalTimeout, queryId));
-            },
-            finalTimeout,
-            TimeUnit.MILLISECONDS);
-    queryScheduledTaskMap.put(queryId, scheduledFuture);
+    if (timeout != 0) {
+      // submit a scheduled task to judge whether query is still running after timeout
+      ScheduledFuture<?> scheduledFuture =
+          executorService.schedule(
+              () -> {
+                killQuery(queryId);
+                logger.warn(
+                    String.format("Query is time out (%dms) with queryId %d", timeout, queryId));
+              },
+              timeout,
+              TimeUnit.MILLISECONDS);
+      queryScheduledTaskMap.put(queryId, scheduledFuture);
+    }
   }
 
   public void killQuery(long queryId) {