You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2020/06/10 07:38:48 UTC

[incubator-iotdb] branch ShowTimeseriesSort created (now 16d6082)

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

jackietien pushed a change to branch ShowTimeseriesSort
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git.


      at 16d6082  resolve conflicts twice

This branch includes the following new commits:

     new 542a59b  init
     new 88e2118  show timeseries with sort
     new 8bea063  resolve conflicts
     new 16d6082  resolve conflicts twice

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[incubator-iotdb] 04/04: resolve conflicts twice

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jackietien pushed a commit to branch ShowTimeseriesSort
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit 16d6082583a1ca848f51f80f579a95676318c3d7
Merge: 8bea063 a7ac8c1
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Wed Jun 10 15:35:29 2020 +0800

    resolve conflicts twice

 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |   4 +-
 .../db/qp/physical/crud/AlignByDevicePlan.java     |  24 +++--
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |  62 ++++++++---
 .../db/query/dataset/AlignByDeviceDataSet.java     |  11 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |   2 +-
 .../IoTDBAggregationAlignByDeviceIT.java           | 113 +++++++++++++++++++++
 6 files changed, 186 insertions(+), 30 deletions(-)



[incubator-iotdb] 02/04: show timeseries with sort

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jackietien pushed a commit to branch ShowTimeseriesSort
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit 88e2118e4637dd36671a4acf6f447710ca8c3d26
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Wed Jun 10 15:29:43 2020 +0800

    show timeseries with sort
---
 .../org/apache/iotdb/db/qp/strategy/SqlBase.g4     |  14 +-
 .../org/apache/iotdb/db/metadata/MManager.java     |  38 ++-
 .../java/org/apache/iotdb/db/metadata/MTree.java   |  71 ++++-
 .../db/qp/logical/sys/ShowTimeSeriesOperator.java  |  10 +
 .../db/qp/physical/sys/ShowTimeSeriesPlan.java     |  15 +-
 .../iotdb/db/qp/strategy/LogicalGenerator.java     |  15 +-
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |  72 ++----
 .../integration/IoTDBSortedShowTimeseriesIT.java   | 287 +++++++++++++++++++++
 8 files changed, 453 insertions(+), 69 deletions(-)

diff --git a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4 b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
index 7d5b9f2..3f18bd5 100644
--- a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
+++ b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
@@ -69,7 +69,7 @@ statement
     | SHOW FLUSH TASK INFO #showFlushTaskInfo
     | SHOW DYNAMIC PARAMETER #showDynamicParameter
     | SHOW VERSION #showVersion
-    | SHOW TIMESERIES prefixPath? showWhereClause? limitClause? #showTimeseries
+    | SHOW TIMESERIES prefixPath? showWhereClause? orderByHeatClause? limitClause? #showTimeseries
     | SHOW STORAGE GROUP #showStorageGroup
     | SHOW CHILD PATHS prefixPath? #showChildPaths
     | SHOW DEVICES prefixPath? #showDevices
@@ -168,6 +168,10 @@ containsExpression
     : name=ID OPERATOR_CONTAINS value=propertyValue
     ;
 
+orderByHeatClause
+    : ORDER BY HEAT
+    ;
+
 orExpression
     : andExpression (OPERATOR_OR andExpression)*
     ;
@@ -876,6 +880,14 @@ TRUE
 FALSE
     : F A L S E
     ;
+
+ORDER
+    : O R D E R
+    ;
+
+HEAT
+    : H E A T
+    ;
 //============================
 // End of the keywords list
 //============================
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 19e3a0a..1b89926 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -18,6 +18,8 @@
  */
 package org.apache.iotdb.db.metadata;
 
+import static java.util.stream.Collectors.toList;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
@@ -36,7 +38,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.TreeSet;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBConstant;
@@ -772,7 +773,9 @@ public class MManager {
       if (value2Node.isEmpty()) {
         throw new MetadataException("The key " + plan.getKey() + " is not a tag.");
       }
-      Set<LeafMNode> allMatchedNodes = new TreeSet<>(Comparator.comparing(MNode::getFullPath));
+
+      List<LeafMNode> allMatchedNodes = new ArrayList<>();
+
       if (plan.isContains()) {
         for (Entry<String, Set<LeafMNode>> entry : value2Node.entrySet()) {
           String tagValue = entry.getKey();
@@ -788,6 +791,18 @@ public class MManager {
           }
         }
       }
+
+      // if ordered by heat, we sort all the timeseries by the descending order of the last insert timestamp
+      if (plan.isOrderByHeat()) {
+        allMatchedNodes = allMatchedNodes.stream().sorted(
+            Comparator.comparingLong(MTree::getLastTimeStamp).reversed()
+                .thenComparing(MNode::getFullPath)).collect(toList());
+      } else {
+        // otherwise, we just sort them by the alphabetical order
+        allMatchedNodes = allMatchedNodes.stream().sorted(Comparator.comparing(MNode::getFullPath))
+            .collect(toList());
+      }
+
       List<ShowTimeSeriesResult> res = new LinkedList<>();
       String[] prefixNodes = MetaUtils.getNodeNames(plan.getPath().getFullPath());
       int curOffset = -1;
@@ -811,7 +826,7 @@ public class MManager {
                 getStorageGroupName(leaf.getFullPath()), measurementSchema.getType().toString(),
                 measurementSchema.getEncodingType().toString(),
                 measurementSchema.getCompressor().toString(), pair.left));
-            if (limit != 0 || offset != 0) {
+            if (limit != 0) {
               count++;
             }
           } catch (IOException e) {
@@ -850,8 +865,13 @@ public class MManager {
   public List<ShowTimeSeriesResult> showTimeseries(ShowTimeSeriesPlan plan)
       throws MetadataException {
     lock.readLock().lock();
+    List<String[]> ans;
     try {
-      List<String[]> ans = mtree.getAllMeasurementSchema(plan);
+      if (plan.isOrderByHeat()) {
+        ans = mtree.getAllMeasurementSchemaByHeatOrder(plan);
+      } else {
+        ans = mtree.getAllMeasurementSchema(plan);
+      }
       List<ShowTimeSeriesResult> res = new LinkedList<>();
       for (String[] ansString : ans) {
         long tagFileOffset = Long.parseLong(ansString[6]);
@@ -972,7 +992,7 @@ public class MManager {
 
   /**
    * get device node, if the storage group is not set, create it when autoCreateSchema is true
-   *
+   * <p>
    * (we develop this method as we need to get the node's lock after we get the lock.writeLock())
    *
    * <p>!!!!!!Attention!!!!! must call the return node's readUnlock() if you call this method.
@@ -1573,15 +1593,17 @@ public class MManager {
   }
 
   /**
-   * if the path is in local mtree, nothing needed to do (because mtree is in the memory);
-   * Otherwise cache the path to mRemoteSchemaCache
+   * if the path is in local mtree, nothing needed to do (because mtree is in the memory); Otherwise
+   * cache the path to mRemoteSchemaCache
+   *
    * @param path
    * @param schema
    */
   public void cacheSchema(String path, MeasurementSchema schema) {
     // check schema is in local
     try {
-      ShowTimeSeriesPlan tempPlan = new ShowTimeSeriesPlan(new Path(path), false, null, null, 0, 0);
+      ShowTimeSeriesPlan tempPlan = new ShowTimeSeriesPlan(new Path(path), false, null, null, 0, 0,
+          false);
       List<String[]> schemas = mtree.getAllMeasurementSchema(tempPlan);
       if (schemas.isEmpty()) {
         mRemoteSchemaCache.put(path, schema);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index 6b944ec..530594e 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@ -18,8 +18,10 @@
  */
 package org.apache.iotdb.db.metadata;
 
+import static java.util.stream.Collectors.toList;
 import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
 import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_WILDCARD;
+import static org.apache.iotdb.db.query.executor.LastQueryExecutor.calculateLastPairForOneSeries;
 
 import com.alibaba.fastjson.JSON;
 import com.alibaba.fastjson.JSONObject;
@@ -27,6 +29,9 @@ import com.alibaba.fastjson.serializer.SerializerFeature;
 import java.io.Serializable;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -38,6 +43,7 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.regex.Pattern;
+import java.util.stream.Stream;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
@@ -52,10 +58,12 @@ import org.apache.iotdb.db.metadata.mnode.LeafMNode;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
@@ -612,6 +620,7 @@ public class MTree implements Serializable {
 
   /**
    * Traverse the MTree to get the count of timeseries in the given level.
+   *
    * @param targetLevel Record the distance to the target level, 0 means the target level.
    */
   private int getCountInGivenLevel(MNode node, int targetLevel) {
@@ -628,6 +637,34 @@ public class MTree implements Serializable {
   }
 
   /**
+   * Get all time series schema under the given path order by insert frequency
+   *
+   * <p>result: [name, alias, storage group, dataType, encoding, compression, offset]
+   */
+  List<String[]> getAllMeasurementSchemaByHeatOrder(ShowTimeSeriesPlan plan)
+      throws MetadataException {
+    String[] nodes = MetaUtils.getNodeNames(plan.getPath().getFullPath());
+    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
+      throw new IllegalPathException(plan.getPath().getFullPath());
+    }
+    List<String[]> allMatchedNodes = new ArrayList<>();
+
+    findPath(root, nodes, 1, "", allMatchedNodes, false);
+
+    Stream<String[]> sortedStream = allMatchedNodes.stream().sorted(
+        Comparator.comparingLong((String[] s) -> Long.parseLong(s[7])).reversed()
+            .thenComparing((String[] array) -> array[0]));
+
+    // no limit
+    if (plan.getLimit() == 0) {
+      return sortedStream.collect(toList());
+    } else {
+      return sortedStream.skip(plan.getOffset()).limit(plan.getLimit()).collect(toList());
+    }
+  }
+
+
+  /**
    * Get all time series schema under the given path
    *
    * <p>result: [name, alias, storage group, dataType, encoding, compression, offset]
@@ -643,10 +680,10 @@ public class MTree implements Serializable {
     curOffset.set(-1);
     count.set(0);
     if (offset.get() != 0 || limit.get() != 0) {
-      res = new ArrayList<>(limit.get());
+      res = new LinkedList<>();
       findPath(root, nodes, 1, "", res, true);
     } else {
-      res = new ArrayList<>();
+      res = new LinkedList<>();
       findPath(root, nodes, 1, "", res, false);
     }
     // avoid memory leaks
@@ -660,10 +697,11 @@ public class MTree implements Serializable {
   /**
    * Iterate through MTree to fetch metadata info of all leaf nodes under the given seriesPath
    *
-   * @param timeseriesSchemaList List<timeseriesSchema>
+   * @param timeseriesSchemas Collection<timeseriesSchema> result: [name, alias, storage group,
+   *                          dataType, encoding, compression, offset, lastTimeStamp]
    */
   private void findPath(MNode node, String[] nodes, int idx, String parent,
-      List<String[]> timeseriesSchemaList, boolean hasLimit) throws MetadataException {
+      Collection<String[]> timeseriesSchemas, boolean hasLimit) throws MetadataException {
     if (node instanceof LeafMNode) {
       if (nodes.length <= idx) {
         if (hasLimit) {
@@ -679,7 +717,7 @@ public class MTree implements Serializable {
           nodeName = node.getName();
         }
         String nodePath = parent + nodeName;
-        String[] tsRow = new String[7];
+        String[] tsRow = new String[8];
         tsRow[0] = nodePath;
         tsRow[1] = ((LeafMNode) node).getAlias();
         MeasurementSchema measurementSchema = ((LeafMNode) node).getSchema();
@@ -688,7 +726,8 @@ public class MTree implements Serializable {
         tsRow[4] = measurementSchema.getEncodingType().toString();
         tsRow[5] = measurementSchema.getCompressor().toString();
         tsRow[6] = String.valueOf(((LeafMNode) node).getOffset());
-        timeseriesSchemaList.add(tsRow);
+        tsRow[7] = String.valueOf(getLastTimeStamp((LeafMNode) node));
+        timeseriesSchemas.add(tsRow);
 
         if (hasLimit) {
           count.set(count.get() + 1);
@@ -700,7 +739,7 @@ public class MTree implements Serializable {
     if (!nodeReg.contains(PATH_WILDCARD)) {
       if (node.hasChild(nodeReg)) {
         findPath(node.getChild(nodeReg), nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR,
-            timeseriesSchemaList, hasLimit);
+            timeseriesSchemas, hasLimit);
       }
     } else {
       for (MNode child : node.getChildren().values()) {
@@ -708,7 +747,22 @@ public class MTree implements Serializable {
           continue;
         }
         findPath(child, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR,
-            timeseriesSchemaList, hasLimit);
+            timeseriesSchemas, hasLimit);
+      }
+    }
+  }
+
+  static long getLastTimeStamp(LeafMNode node) {
+    TimeValuePair last = node.getCachedLast();
+    if (last != null) {
+      return node.getCachedLast().getTimestamp();
+    } else {
+      try {
+        last = calculateLastPairForOneSeries(new Path(node.getFullPath()),
+            node.getSchema().getType(), new QueryContext(-1), Collections.emptySet());
+        return last.getTimestamp();
+      } catch (Exception e) {
+        return Long.MIN_VALUE;
       }
     }
   }
@@ -847,6 +901,7 @@ public class MTree implements Serializable {
 
   /**
    * Get all paths under the given level.
+   *
    * @param targetLevel Record the distance to the target level, 0 means the target level.
    */
   private void findNodes(MNode node, String path, List<String> res, int targetLevel) {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowTimeSeriesOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowTimeSeriesOperator.java
index d999772..e2988c7 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowTimeSeriesOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowTimeSeriesOperator.java
@@ -29,6 +29,8 @@ public class ShowTimeSeriesOperator extends ShowOperator {
   private String value;
   private int limit = 0;
   private int offset = 0;
+  // if is true, the result will be sorted according to the inserting frequency of the timeseries
+  private boolean orderByHeat;
 
   public ShowTimeSeriesOperator(int tokeIntType, Path path) {
     super(tokeIntType);
@@ -78,4 +80,12 @@ public class ShowTimeSeriesOperator extends ShowOperator {
   public void setOffset(int offset) {
     this.offset = offset;
   }
+
+  public boolean isOrderByHeat() {
+    return orderByHeat;
+  }
+
+  public void setOrderByHeat(boolean orderByHeat) {
+    this.orderByHeat = orderByHeat;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowTimeSeriesPlan.java
index 92e764c..e01ad3c 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowTimeSeriesPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowTimeSeriesPlan.java
@@ -33,6 +33,8 @@ public class ShowTimeSeriesPlan extends ShowPlan {
   private String value;
   private int limit = 0;
   private int offset = 0;
+  // if is true, the result will be sorted according to the inserting frequency of the timeseries
+  private boolean orderByHeat;
 
   public ShowTimeSeriesPlan(Path path) {
     super(ShowContentType.TIMESERIES);
@@ -40,7 +42,7 @@ public class ShowTimeSeriesPlan extends ShowPlan {
   }
 
   public ShowTimeSeriesPlan(Path path, boolean isContains, String key, String value, int limit,
-      int offset) {
+      int offset, boolean orderByHeat) {
     super(ShowContentType.TIMESERIES);
     this.path = path;
     this.isContains = isContains;
@@ -48,6 +50,7 @@ public class ShowTimeSeriesPlan extends ShowPlan {
     this.value = value;
     this.limit = limit;
     this.offset = offset;
+    this.orderByHeat = orderByHeat;
   }
 
   public ShowTimeSeriesPlan() {
@@ -78,6 +81,14 @@ public class ShowTimeSeriesPlan extends ShowPlan {
     return offset;
   }
 
+  public boolean isOrderByHeat() {
+    return orderByHeat;
+  }
+
+  public void setOrderByHeat(boolean orderByHeat) {
+    this.orderByHeat = orderByHeat;
+  }
+
   @Override
   public void serialize(DataOutputStream outputStream) throws IOException {
     outputStream.write(PhysicalPlanType.SHOW_TIMESERIES.ordinal());
@@ -89,6 +100,7 @@ public class ShowTimeSeriesPlan extends ShowPlan {
 
     outputStream.writeInt(limit);
     outputStream.writeInt(offset);
+    outputStream.writeBoolean(orderByHeat);
   }
 
   @Override
@@ -100,5 +112,6 @@ public class ShowTimeSeriesPlan extends ShowPlan {
 
     limit = buffer.getInt();
     limit = buffer.getInt();
+    orderByHeat = buffer.get() == 1;
   }
 }
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index 43e4c4a..7afd647 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
@@ -27,7 +27,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import org.antlr.v4.runtime.tree.TerminalNode;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.runtime.SQLParserException;
@@ -47,7 +46,6 @@ import org.apache.iotdb.db.qp.logical.sys.AlterTimeSeriesOperator;
 import org.apache.iotdb.db.qp.logical.sys.AlterTimeSeriesOperator.AlterType;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator.AuthorType;
-import org.apache.iotdb.db.qp.logical.sys.LoadConfigurationOperator.LoadConfigurationOperatorType;
 import org.apache.iotdb.db.qp.logical.sys.ClearCacheOperator;
 import org.apache.iotdb.db.qp.logical.sys.CountOperator;
 import org.apache.iotdb.db.qp.logical.sys.CreateTimeSeriesOperator;
@@ -56,6 +54,7 @@ import org.apache.iotdb.db.qp.logical.sys.DeleteStorageGroupOperator;
 import org.apache.iotdb.db.qp.logical.sys.DeleteTimeSeriesOperator;
 import org.apache.iotdb.db.qp.logical.sys.FlushOperator;
 import org.apache.iotdb.db.qp.logical.sys.LoadConfigurationOperator;
+import org.apache.iotdb.db.qp.logical.sys.LoadConfigurationOperator.LoadConfigurationOperatorType;
 import org.apache.iotdb.db.qp.logical.sys.LoadDataOperator;
 import org.apache.iotdb.db.qp.logical.sys.LoadFilesOperator;
 import org.apache.iotdb.db.qp.logical.sys.MergeOperator;
@@ -165,14 +164,11 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.StringContainer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * This class is a listener and you can get an operator which is a logical plan.
  */
 public class LogicalGenerator extends SqlBaseBaseListener {
-  private static Logger logger = LoggerFactory.getLogger(LogicalGenerator.class);
 
   private RootOperator initializedOperator = null;
   private ZoneId zoneId;
@@ -1058,7 +1054,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
     List<String> measurementList = new ArrayList<>();
     for (NodeNameWithoutStarContext nodeNameWithoutStar : nodeNamesWithoutStar) {
       String measurement = nodeNameWithoutStar.getText();
-      if (measurement.contains("\"") || measurement.contains("\'")) {
+      if (measurement.contains("\"") || measurement.contains("'")) {
         measurement = measurement.substring(1, measurement.length() - 1);
       }
       measurementList.add(measurement);
@@ -1360,6 +1356,13 @@ public class LogicalGenerator extends SqlBaseBaseListener {
     operator.setValue(value);
   }
 
+  @Override
+  public void enterOrderByHeatClause(SqlBaseParser.OrderByHeatClauseContext ctx) {
+    super.enterOrderByHeatClause(ctx);
+    ShowTimeSeriesOperator operator = (ShowTimeSeriesOperator) initializedOperator;
+    operator.setOrderByHeat(true);
+  }
+
   private FilterOperator parseOrExpression(OrExpressionContext ctx) {
     if (ctx.andExpression().size() == 1) {
       return parseAndExpression(ctx.andExpression(0));
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index 644b99b..b9b630a 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -98,6 +98,7 @@ import org.slf4j.Logger;
  * Used to convert logical operator to physical plan
  */
 public class PhysicalGenerator {
+
   private static Logger logger = LoggerFactory.getLogger(PhysicalGenerator.class);
 
   public PhysicalPlan transformToPhysicalPlan(Operator operator) throws QueryProcessException {
@@ -106,13 +107,8 @@ public class PhysicalGenerator {
       case AUTHOR:
         AuthorOperator author = (AuthorOperator) operator;
         try {
-          return new AuthorPlan(
-              author.getAuthorType(),
-              author.getUserName(),
-              author.getRoleName(),
-              author.getPassWord(),
-              author.getNewPassword(),
-              author.getPrivilegeList(),
+          return new AuthorPlan(author.getAuthorType(), author.getUserName(), author.getRoleName(),
+              author.getPassWord(), author.getNewPassword(), author.getPrivilegeList(),
               author.getNodeName());
         } catch (AuthException e) {
           throw new QueryProcessException(e.getMessage());
@@ -145,26 +141,17 @@ public class PhysicalGenerator {
             }
           }
         }
-        return new CreateTimeSeriesPlan(
-            createOperator.getPath(),
-            createOperator.getDataType(),
-            createOperator.getEncoding(),
-            createOperator.getCompressor(),
-            createOperator.getProps(),
-            createOperator.getTags(),
-            createOperator.getAttributes(),
-            createOperator.getAlias());
+        return new CreateTimeSeriesPlan(createOperator.getPath(), createOperator.getDataType(),
+            createOperator.getEncoding(), createOperator.getCompressor(), createOperator.getProps(),
+            createOperator.getTags(), createOperator.getAttributes(), createOperator.getAlias());
       case DELETE_TIMESERIES:
         DeleteTimeSeriesOperator deletePath = (DeleteTimeSeriesOperator) operator;
         return new DeleteTimeSeriesPlan(deletePath.getDeletePathList());
       case ALTER_TIMESERIES:
         AlterTimeSeriesOperator alterTimeSeriesOperator = (AlterTimeSeriesOperator) operator;
-        return new AlterTimeSeriesPlan(
-            alterTimeSeriesOperator.getPath(),
-            alterTimeSeriesOperator.getAlterType(),
-            alterTimeSeriesOperator.getAlterMap(),
-            alterTimeSeriesOperator.getAlias(),
-            alterTimeSeriesOperator.getTagsMap(),
+        return new AlterTimeSeriesPlan(alterTimeSeriesOperator.getPath(),
+            alterTimeSeriesOperator.getAlterType(), alterTimeSeriesOperator.getAlterMap(),
+            alterTimeSeriesOperator.getAlias(), alterTimeSeriesOperator.getTagsMap(),
             alterTimeSeriesOperator.getAttributesMap());
       case DELETE:
         DeleteDataOperator delete = (DeleteDataOperator) operator;
@@ -178,11 +165,8 @@ public class PhysicalGenerator {
               "For Insert command, cannot specified more than one seriesPath: " + paths);
         }
 
-        return new InsertPlan(
-            paths.get(0).getFullPath(),
-            insert.getTime(),
-            insert.getMeasurementList(),
-            insert.getValueList());
+        return new InsertPlan(paths.get(0).getFullPath(), insert.getTime(),
+            insert.getMeasurementList(), insert.getValueList());
       case MERGE:
         if (operator.getTokenIntType() == SQLConstant.TOK_FULL_MERGE) {
           return new MergePlan(OperatorType.FULL_MERGE);
@@ -225,10 +209,10 @@ public class PhysicalGenerator {
             return new ShowPlan(ShowContentType.VERSION);
           case SQLConstant.TOK_TIMESERIES:
             ShowTimeSeriesOperator showTimeSeriesOperator = (ShowTimeSeriesOperator) operator;
-            return new ShowTimeSeriesPlan(
-                showTimeSeriesOperator.getPath(), showTimeSeriesOperator.isContains(),
-                showTimeSeriesOperator.getKey(), showTimeSeriesOperator.getValue(),
-                showTimeSeriesOperator.getLimit(), showTimeSeriesOperator.getOffset());
+            return new ShowTimeSeriesPlan(showTimeSeriesOperator.getPath(),
+                showTimeSeriesOperator.isContains(), showTimeSeriesOperator.getKey(),
+                showTimeSeriesOperator.getValue(), showTimeSeriesOperator.getLimit(),
+                showTimeSeriesOperator.getOffset(), showTimeSeriesOperator.isOrderByHeat());
           case SQLConstant.TOK_STORAGE_GROUP:
             return new ShowPlan(ShowContentType.STORAGE_GROUP);
           case SQLConstant.TOK_DEVICES:
@@ -256,19 +240,15 @@ public class PhysicalGenerator {
                     "not supported operator type %s in show operation.", operator.getType()));
         }
       case LOAD_FILES:
-        return new OperateFilePlan(
-            ((LoadFilesOperator) operator).getFile(),
-            OperatorType.LOAD_FILES,
-            ((LoadFilesOperator) operator).isAutoCreateSchema(),
+        return new OperateFilePlan(((LoadFilesOperator) operator).getFile(),
+            OperatorType.LOAD_FILES, ((LoadFilesOperator) operator).isAutoCreateSchema(),
             ((LoadFilesOperator) operator).getSgLevel());
       case REMOVE_FILE:
-        return new OperateFilePlan(
-            ((RemoveFileOperator) operator).getFile(), OperatorType.REMOVE_FILE);
+        return new OperateFilePlan(((RemoveFileOperator) operator).getFile(),
+            OperatorType.REMOVE_FILE);
       case MOVE_FILE:
-        return new OperateFilePlan(
-            ((MoveFileOperator) operator).getFile(),
-            ((MoveFileOperator) operator).getTargetDir(),
-            OperatorType.MOVE_FILE);
+        return new OperateFilePlan(((MoveFileOperator) operator).getFile(),
+            ((MoveFileOperator) operator).getTargetDir(), OperatorType.MOVE_FILE);
       case CLEAR_CACHE:
         return new ClearCachePlan();
       default:
@@ -341,7 +321,8 @@ public class PhysicalGenerator {
 
       if (queryOperator.getLevel() >= 0) {
         for (int i = 0; i < queryOperator.getSelectOperator().getAggregations().size(); i++) {
-          if (!SQLConstant.COUNT.equals(queryOperator.getSelectOperator().getAggregations().get(i))) {
+          if (!SQLConstant.COUNT
+              .equals(queryOperator.getSelectOperator().getAggregations().get(i))) {
             throw new QueryProcessException("group by level only support count now.");
           }
         }
@@ -357,12 +338,13 @@ public class PhysicalGenerator {
       ((FillQueryPlan) queryPlan).setFillType(queryOperator.getFillTypes());
     } else if (queryOperator.hasAggregation()) {
       queryPlan = new AggregationPlan();
-      ((AggregationPlan)queryPlan).setLevel(queryOperator.getLevel());
+      ((AggregationPlan) queryPlan).setLevel(queryOperator.getLevel());
       ((AggregationPlan) queryPlan)
           .setAggregations(queryOperator.getSelectOperator().getAggregations());
       if (queryOperator.getLevel() >= 0) {
         for (int i = 0; i < queryOperator.getSelectOperator().getAggregations().size(); i++) {
-          if (!SQLConstant.COUNT.equals(queryOperator.getSelectOperator().getAggregations().get(i))) {
+          if (!SQLConstant.COUNT
+              .equals(queryOperator.getSelectOperator().getAggregations().get(i))) {
             throw new QueryProcessException("group by level only support count now.");
           }
         }
@@ -387,7 +369,7 @@ public class PhysicalGenerator {
       } else if (queryPlan instanceof FillQueryPlan) {
         alignByDevicePlan.setFillQueryPlan((FillQueryPlan) queryPlan);
       } else if (queryPlan instanceof AggregationPlan) {
-        if (((AggregationPlan)queryPlan).getLevel() >= 0) {
+        if (((AggregationPlan) queryPlan).getLevel() >= 0) {
           throw new QueryProcessException("group by level does not support align by device now.");
         }
         alignByDevicePlan.setAggregationPlan((AggregationPlan) queryPlan);
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSortedShowTimeseriesIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSortedShowTimeseriesIT.java
new file mode 100644
index 0000000..3460199
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSortedShowTimeseriesIT.java
@@ -0,0 +1,287 @@
+/*
+ * 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.db.integration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IoTDBSortedShowTimeseriesIT {
+
+  private static String[] sqls = new String[]{
+      "SET STORAGE GROUP TO root.turbine",
+
+      "create timeseries root.turbine.d0.s0(temperature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(unit=f, description='turbine this is a test1') attributes(H_Alarm=100, M_Alarm=50)",
+
+      "create timeseries root.turbine.d0.s1(power) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(unit=kw, description='turbine this is a test2') attributes(H_Alarm=99.9, M_Alarm=44.4)",
+
+      "create timeseries root.turbine.d0.s2(cpu) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(unit=cores, description='turbine this is a cpu') attributes(H_Alarm=99.9, M_Alarm=44.4)",
+
+      "create timeseries root.turbine.d0.s3(gpu0) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(unit=cores, description='turbine this is a gpu') attributes(H_Alarm=99.9, M_Alarm=44.4)",
+
+      "create timeseries root.turbine.d0.s4(tpu0) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(unit=cores, description='turbine this is a tpu') attributes(H_Alarm=99.9, M_Alarm=44.4)",
+
+      "create timeseries root.turbine.d1.s0(status) with datatype=INT32, encoding=RLE " +
+          "tags(description='turbine this is a test3') attributes(H_Alarm=9, M_Alarm=5)",
+
+      "create timeseries root.turbine.d2.s0(temperature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(unit=f, description='turbine d2 this is a test1') attributes(MaxValue=100, MinValue=1)",
+
+      "create timeseries root.turbine.d2.s1(power) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(unit=kw, description='turbine d2 this is a test2') attributes(MaxValue=99.9, MinValue=44.4)",
+
+      "create timeseries root.turbine.d2.s3(status) with datatype=INT32, encoding=RLE " +
+          "tags(description='turbine d2 this is a test3') attributes(MaxValue=9, MinValue=5)",
+
+      "create timeseries root.ln.d0.s0(temperature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(unit=c, description='ln this is a test1') attributes(H_Alarm=1000, M_Alarm=500)",
+
+      "create timeseries root.ln.d0.s1(power) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(unit=w, description='ln this is a test2') attributes(H_Alarm=9.9, M_Alarm=4.4)",
+
+      "create timeseries root.ln.d1.s0(status) with datatype=INT32, encoding=RLE " +
+          "tags(description='ln this is a test3') attributes(H_Alarm=90, M_Alarm=50)",
+
+      "insert into root.turbine.d0(timestamp,s0) values(1, 1)",
+      "insert into root.turbine.d0(timestamp,s1) values(2, 2)",
+      "insert into root.turbine.d0(timestamp,s2) values(3, 3)",
+      "insert into root.turbine.d0(timestamp,s3) values(4, 4)",
+      "insert into root.turbine.d0(timestamp,s4) values(5, 5)",
+      "insert into root.turbine.d1(timestamp,s0) values(1, 11)",
+      "insert into root.turbine.d2(timestamp,s0,s1,s3) values(6,6,6,6)"
+  };
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.envSetUp();
+    createSchema();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void showTimeseriesOrderByHeatTest1() throws ClassNotFoundException {
+    String[] retArray1 = new String[]{
+        "root.turbine.d0.s0,temperature,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a test1,100,50,null,null,f",
+        "root.turbine.d0.s1,power,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a test2,99.9,44.4,null,null,kw",
+        "root.turbine.d0.s2,cpu,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a cpu,99.9,44.4,null,null,cores",
+        "root.turbine.d0.s3,gpu0,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a gpu,99.9,44.4,null,null,cores",
+        "root.turbine.d0.s4,tpu0,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a tpu,99.9,44.4,null,null,cores",
+        "root.turbine.d1.s0,status,root.turbine,INT32,RLE,SNAPPY,turbine this is a test3,9,5,null,null,null",
+        "root.turbine.d2.s0,temperature,root.turbine,FLOAT,RLE,SNAPPY,turbine d2 this is a test1,null,null,100,1,f",
+        "root.turbine.d2.s1,power,root.turbine,FLOAT,RLE,SNAPPY,turbine d2 this is a test2,null,null,99.9,44.4,kw",
+        "root.turbine.d2.s3,status,root.turbine,INT32,RLE,SNAPPY,turbine d2 this is a test3,null,null,9,5,null",
+        "root.ln.d0.s0,temperature,root.ln,FLOAT,RLE,SNAPPY,ln this is a test1,1000,500,null,null,c",
+        "root.ln.d0.s1,power,root.ln,FLOAT,RLE,SNAPPY,ln this is a test2,9.9,4.4,null,null,w",
+        "root.ln.d1.s0,status,root.ln,INT32,RLE,SNAPPY,ln this is a test3,90,50,null,null,null"
+    };
+
+    String[] retArray2 = new String[]{
+        "root.turbine.d2.s0,temperature,root.turbine,FLOAT,RLE,SNAPPY,turbine d2 this is a test1,null,null,100,1,f",
+        "root.turbine.d2.s1,power,root.turbine,FLOAT,RLE,SNAPPY,turbine d2 this is a test2,null,null,99.9,44.4,kw",
+        "root.turbine.d2.s3,status,root.turbine,INT32,RLE,SNAPPY,turbine d2 this is a test3,null,null,9,5,null",
+        "root.turbine.d0.s4,tpu0,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a tpu,99.9,44.4,null,null,cores",
+        "root.turbine.d0.s3,gpu0,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a gpu,99.9,44.4,null,null,cores",
+        "root.turbine.d0.s2,cpu,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a cpu,99.9,44.4,null,null,cores",
+        "root.turbine.d0.s1,power,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a test2,99.9,44.4,null,null,kw",
+        "root.turbine.d0.s0,temperature,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a test1,100,50,null,null,f",
+        "root.turbine.d1.s0,status,root.turbine,INT32,RLE,SNAPPY,turbine this is a test3,9,5,null,null,null",
+        "root.ln.d0.s0,temperature,root.ln,FLOAT,RLE,SNAPPY,ln this is a test1,1000,500,null,null,c",
+        "root.ln.d0.s1,power,root.ln,FLOAT,RLE,SNAPPY,ln this is a test2,9.9,4.4,null,null,w",
+        "root.ln.d1.s0,status,root.ln,INT32,RLE,SNAPPY,ln this is a test3,90,50,null,null,null",
+    };
+
+    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()) {
+
+      boolean hasResultSet = statement.execute("show timeseries");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+            + "," + resultSet.getString("alias")
+            + "," + resultSet.getString("storage group")
+            + "," + resultSet.getString("dataType")
+            + "," + resultSet.getString("encoding")
+            + "," + resultSet.getString("compression")
+            + "," + resultSet.getString("description")
+            + "," + resultSet.getString("H_Alarm")
+            + "," + resultSet.getString("M_Alarm")
+            + "," + resultSet.getString("MaxValue")
+            + "," + resultSet.getString("MinValue")
+            + "," + resultSet.getString("unit");
+
+        assertEquals(retArray1[count], ans);
+        count++;
+      }
+      assertEquals(retArray1.length, count);
+
+      hasResultSet = statement.execute("show timeseries order by heat");
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+            + "," + resultSet.getString("alias")
+            + "," + resultSet.getString("storage group")
+            + "," + resultSet.getString("dataType")
+            + "," + resultSet.getString("encoding")
+            + "," + resultSet.getString("compression")
+            + "," + resultSet.getString("description")
+            + "," + resultSet.getString("H_Alarm")
+            + "," + resultSet.getString("M_Alarm")
+            + "," + resultSet.getString("MaxValue")
+            + "," + resultSet.getString("MinValue")
+            + "," + resultSet.getString("unit");
+
+        System.out.println("\"" + ans + "\",");
+        assertEquals(retArray2[count], ans);
+        count++;
+      }
+      assertEquals(retArray2.length, count);
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void showTimeseriesOrderByHeatWithLimitTest() throws ClassNotFoundException {
+
+    String[] retArray = new String[]{
+        "root.turbine.d2.s0,temperature,root.turbine,FLOAT,RLE,SNAPPY,turbine d2 this is a test1,null,null,100,1,f",
+        "root.turbine.d2.s1,power,root.turbine,FLOAT,RLE,SNAPPY,turbine d2 this is a test2,null,null,99.9,44.4,kw",
+        "root.turbine.d2.s3,status,root.turbine,INT32,RLE,SNAPPY,turbine d2 this is a test3,null,null,9,5,null",
+        "root.turbine.d0.s4,tpu0,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a tpu,99.9,44.4,null,null,cores",
+        "root.turbine.d0.s3,gpu0,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a gpu,99.9,44.4,null,null,cores",
+    };
+
+    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()) {
+
+      boolean hasResultSet = statement.execute("show timeseries order by heat limit 5");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+            + "," + resultSet.getString("alias")
+            + "," + resultSet.getString("storage group")
+            + "," + resultSet.getString("dataType")
+            + "," + resultSet.getString("encoding")
+            + "," + resultSet.getString("compression")
+            + "," + resultSet.getString("description")
+            + "," + resultSet.getString("H_Alarm")
+            + "," + resultSet.getString("M_Alarm")
+            + "," + resultSet.getString("MaxValue")
+            + "," + resultSet.getString("MinValue")
+            + "," + resultSet.getString("unit");
+
+        assertEquals(retArray[count], ans);
+        count++;
+      }
+      assertEquals(retArray.length, count);
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void showTimeseriesOrderByHeatWithWhereTest() throws ClassNotFoundException {
+
+    String[] retArray = new String[]{
+        "root.turbine.d0.s4,tpu0,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a tpu,99.9,44.4,cores",
+        "root.turbine.d0.s3,gpu0,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a gpu,99.9,44.4,cores",
+        "root.turbine.d0.s2,cpu,root.turbine,FLOAT,RLE,SNAPPY,turbine this is a cpu,99.9,44.4,cores",
+    };
+
+    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()) {
+
+      boolean hasResultSet = statement.execute("show timeseries where unit=cores order by heat");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+            + "," + resultSet.getString("alias")
+            + "," + resultSet.getString("storage group")
+            + "," + resultSet.getString("dataType")
+            + "," + resultSet.getString("encoding")
+            + "," + resultSet.getString("compression")
+            + "," + resultSet.getString("description")
+            + "," + resultSet.getString("H_Alarm")
+            + "," + resultSet.getString("M_Alarm")
+            + "," + resultSet.getString("unit");
+
+        assertEquals(retArray[count], ans);
+        count++;
+      }
+      assertEquals(retArray.length, count);
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  private void createSchema() throws ClassNotFoundException {
+    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()) {
+
+      for (String sql : sqls) {
+        statement.execute(sql);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+}


[incubator-iotdb] 03/04: resolve conflicts

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jackietien pushed a commit to branch ShowTimeseriesSort
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit 8bea06346acfdd01c4ca36da2c5abb5be0bdbbdd
Merge: 88e2118 6a927f4
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Wed Jun 10 15:34:26 2020 +0800

    resolve conflicts

 docs/UserGuide/Server/Config Manual.md             |  55 ++++++
 docs/zh/UserGuide/Server/Config Manual.md          |  58 ++++++
 .../resources/conf/iotdb-engine.properties         |   3 +
 .../org/apache/iotdb/db/qp/strategy/SqlBase.g4     |   4 +
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  18 ++
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |   2 +
 .../iotdb/db/engine/merge/task/MergeTask.java      |   4 +-
 .../engine/storagegroup/StorageGroupProcessor.java |   6 +-
 .../org/apache/iotdb/db/metadata/MManager.java     |  81 ++++----
 .../java/org/apache/iotdb/db/metadata/MTree.java   | 110 ++++++-----
 .../iotdb/db/metadata/mnode/InternalMNode.java     |   2 +-
 .../{LeafMNode.java => MeasurementMNode.java}      |  57 +-----
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  10 +-
 .../iotdb/db/query/executor/LastQueryExecutor.java |   6 +-
 .../apache/iotdb/db/utils/TypeInferenceUtils.java  |   6 +-
 .../iotdb/db/integration/IoTDBAddSubDeviceIT.java  | 203 +++++++++++++++++++++
 .../iotdb/db/integration/IoTDBInsertNaNIT.java     | 139 ++++++++++++++
 .../apache/iotdb/db/integration/IoTDBLastIT.java   |  12 +-
 .../iotdb/db/metadata/MManagerAdvancedTest.java    |  14 +-
 .../iotdb/db/metadata/MManagerImproveTest.java     |   4 +-
 .../org/apache/iotdb/db/metadata/MTreeTest.java    |  77 ++++----
 .../java/org/apache/iotdb/db/qp/PlannerTest.java   |  13 ++
 22 files changed, 679 insertions(+), 205 deletions(-)

diff --cc server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
index 3f18bd5,d6ef3dd..21a84f2
--- a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
+++ b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
@@@ -881,13 -878,6 +882,14 @@@ FALS
      : F A L S E
      ;
  
 +ORDER
 +    : O R D E R
 +    ;
 +
 +HEAT
 +    : H E A T
 +    ;
++
  //============================
  // End of the keywords list
  //============================
diff --cc server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 1b89926,ec46b77..67694dc
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@@ -773,11 -782,9 +783,10 @@@ public class MManager 
        if (value2Node.isEmpty()) {
          throw new MetadataException("The key " + plan.getKey() + " is not a tag.");
        }
 -      Set<MeasurementMNode> allMatchedNodes = new TreeSet<>(Comparator.comparing(MNode::getFullPath));
 +
-       List<LeafMNode> allMatchedNodes = new ArrayList<>();
- 
++      List<MeasurementMNode> allMatchedNodes = new ArrayList<>();
        if (plan.isContains()) {
-         for (Entry<String, Set<LeafMNode>> entry : value2Node.entrySet()) {
+         for (Entry<String, Set<MeasurementMNode>> entry : value2Node.entrySet()) {
            String tagValue = entry.getKey();
            if (tagValue.contains(plan.getValue())) {
              allMatchedNodes.addAll(entry.getValue());
diff --cc server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index 530594e,bfb2398..df27cd7
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@@ -18,32 -18,9 +18,34 @@@
   */
  package org.apache.iotdb.db.metadata;
  
 +import static java.util.stream.Collectors.toList;
 +import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
 +import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_WILDCARD;
 +import static org.apache.iotdb.db.query.executor.LastQueryExecutor.calculateLastPairForOneSeries;
 +
 +import com.alibaba.fastjson.JSON;
 +import com.alibaba.fastjson.JSONObject;
 +import com.alibaba.fastjson.serializer.SerializerFeature;
 +import java.io.Serializable;
 +import java.util.ArrayDeque;
 +import java.util.ArrayList;
- import java.util.Collection;
 +import java.util.Collections;
 +import java.util.Comparator;
 +import java.util.Deque;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Queue;
 +import java.util.Set;
 +import java.util.TreeSet;
 +import java.util.regex.Pattern;
 +import java.util.stream.Stream;
+ import com.alibaba.fastjson.JSON;
+ import com.alibaba.fastjson.JSONObject;
+ import com.alibaba.fastjson.serializer.SerializerFeature;
  import org.apache.iotdb.db.conf.IoTDBConstant;
  import org.apache.iotdb.db.conf.IoTDBDescriptor;
  import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
@@@ -697,37 -657,34 +713,36 @@@ public class MTree implements Serializa
    /**
     * Iterate through MTree to fetch metadata info of all leaf nodes under the given seriesPath
     *
-    * @param timeseriesSchemas Collection<timeseriesSchema> result: [name, alias, storage group,
 -   * @param timeseriesSchemaList List<timeseriesSchema>
++   * @param timeseriesSchemaList Collection<timeseriesSchema> result: [name, alias, storage group,
 +   *                          dataType, encoding, compression, offset, lastTimeStamp]
     */
    private void findPath(MNode node, String[] nodes, int idx, String parent,
-       Collection<String[]> timeseriesSchemas, boolean hasLimit) throws MetadataException {
-     if (node instanceof LeafMNode) {
-       if (nodes.length <= idx) {
-         if (hasLimit) {
-           curOffset.set(curOffset.get() + 1);
-           if (curOffset.get() < offset.get() || count.get().intValue() == limit.get().intValue()) {
-             return;
-           }
+       List<String[]> timeseriesSchemaList, boolean hasLimit) throws MetadataException {
+     if (node instanceof MeasurementMNode && nodes.length <= idx) {
+       if (hasLimit) {
+         curOffset.set(curOffset.get() + 1);
+         if (curOffset.get() < offset.get() || count.get().intValue() == limit.get().intValue()) {
+           return;
          }
-         String nodeName;
-         if (node.getName().contains(TsFileConstant.PATH_SEPARATOR)) {
-           nodeName = "\"" + node + "\"";
-         } else {
+       }
+       String nodeName;
+       if (node.getName().contains(TsFileConstant.PATH_SEPARATOR)) {
+         nodeName = "\"" + node + "\"";
+       } else {
            nodeName = node.getName();
          }
          String nodePath = parent + nodeName;
 -        String[] tsRow = new String[7];
 +        String[] tsRow = new String[8];
          tsRow[0] = nodePath;
-         tsRow[1] = ((LeafMNode) node).getAlias();
-         MeasurementSchema measurementSchema = ((LeafMNode) node).getSchema();
+         tsRow[1] = ((MeasurementMNode) node).getAlias();
+         MeasurementSchema measurementSchema = ((MeasurementMNode) node).getSchema();
          tsRow[2] = getStorageGroupName(nodePath);
          tsRow[3] = measurementSchema.getType().toString();
          tsRow[4] = measurementSchema.getEncodingType().toString();
          tsRow[5] = measurementSchema.getCompressor().toString();
-         tsRow[6] = String.valueOf(((LeafMNode) node).getOffset());
-         tsRow[7] = String.valueOf(getLastTimeStamp((LeafMNode) node));
-         timeseriesSchemas.add(tsRow);
+         tsRow[6] = String.valueOf(((MeasurementMNode) node).getOffset());
 -        timeseriesSchemaList.add(tsRow);
++        tsRow[7] = String.valueOf(getLastTimeStamp((MeasurementMNode) node));
++      timeseriesSchemaList.add(tsRow);
  
          if (hasLimit) {
            count.set(count.get() + 1);
@@@ -752,21 -707,6 +765,21 @@@
      }
    }
  
-   static long getLastTimeStamp(LeafMNode node) {
++  static long getLastTimeStamp(MeasurementMNode node) {
 +    TimeValuePair last = node.getCachedLast();
 +    if (last != null) {
 +      return node.getCachedLast().getTimestamp();
 +    } else {
 +      try {
 +        last = calculateLastPairForOneSeries(new Path(node.getFullPath()),
 +            node.getSchema().getType(), new QueryContext(-1), Collections.emptySet());
 +        return last.getTimestamp();
 +      } catch (Exception e) {
 +        return Long.MIN_VALUE;
 +      }
 +    }
 +  }
 +
    /**
     * Get child node path in the next level of the given path.
     *


[incubator-iotdb] 01/04: init

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jackietien pushed a commit to branch ShowTimeseriesSort
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit 542a59bfb3bb097dae3471d476d69626b443c2a4
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Mon Jun 8 19:58:12 2020 +0800

    init
---
 .../src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index cce6140..9effca0 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -967,11 +967,11 @@ public class PlanExecutor implements IPlanExecutor {
       }
     } else if (deviceNode != null) {
       // device and measurement exists in MTree
-      LeafMNode measurementNode = (LeafMNode) MManager.getInstance().getChild(deviceNode, measurement);
+      LeafMNode measurementNode = (LeafMNode) mManager.getChild(deviceNode, measurement);
       measurementSchema = measurementNode.getSchema();
     } else {
       // device in not in MTree, try the cache
-      measurementSchema = MManager.getInstance().getSeriesSchema(deviceId, measurement);
+      measurementSchema = mManager.getSeriesSchema(deviceId, measurement);
     }
     return measurementSchema;
   }