You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2020/08/28 08:54:35 UTC

[GitHub] [incubator-iotdb] haimeiguo opened a new pull request #1656: Add raw data query in session

haimeiguo opened a new pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656


   


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

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



[GitHub] [incubator-iotdb] haimeiguo commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
haimeiguo commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r479844972



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,27 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+          throws StatementExecutionException, IoTDBConnectionException {
+
+    String statement = "raw data query";
+    TSRawDataQueryReq execReq = new TSRawDataQueryReq(sessionId, paths, startTime, endTime, statementId, "");

Review comment:
       Hi Houliang, thank you for pointing out. I will pass the statement string to parameter. We will need statement and statementId in closeSession() when executing multiple queries. 




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

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



[GitHub] [incubator-iotdb] mychaow commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r479056695



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,27 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)

Review comment:
       and change all other "time" in the code to the constant string, like in BasicOperatorType




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

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



[GitHub] [incubator-iotdb] neuyilan commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r479604627



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,27 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+          throws StatementExecutionException, IoTDBConnectionException {
+
+    String statement = "raw data query";
+    TSRawDataQueryReq execReq = new TSRawDataQueryReq(sessionId, paths, startTime, endTime, statementId, "");

Review comment:
       as the statement parameter is empty, why we need the parameter?

##########
File path: thrift/src/main/thrift/rpc.thrift
##########
@@ -222,6 +222,16 @@ struct TSDeleteDataReq {
     4: required i64 endTime
 }
 
+struct TSRawDataQueryReq {
+    1: required i64 sessionId
+    2: required list<string> paths
+    3: optional i32 fetchSize
+    4: required i64 startTime
+    5: required i64 endTime
+    6: required i64 statementId
+    7: required string statement

Review comment:
       What is the statement used for? can you give some examples?

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,27 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+          throws StatementExecutionException, IoTDBConnectionException {
+
+    String statement = "raw data query";

Review comment:
       please make the statement is a constant field to avoid too many local variables to created and GC. 




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

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



[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r484674664



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,37 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime
+   * time interval include startTime and exclude endTime
+   * @param paths
+   * @param startTime included
+   * @param endTime excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
+
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+          throws StatementExecutionException, IoTDBConnectionException {
+
+    TSRawDataQueryReq execReq = new TSRawDataQueryReq(sessionId, paths, startTime, endTime);
+    execReq.setFetchSize(fetchSize);
+
+    TSExecuteStatementResp execResp;
+    try {
+      execResp = client.executeRawDataQuery(execReq);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+
+    RpcUtils.verifySuccess(execResp.getStatus());
+    return new SessionDataSet("", execResp.getColumns(), execResp.getDataTypeList(),

Review comment:
       It's ok, but it seems that the client-0.10 can not connect to server-0.11 already...




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

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



[GitHub] [incubator-iotdb] haimeiguo commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
haimeiguo commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r481819256



##########
File path: example/session/src/main/java/org/apache/iotdb/SessionExample.java
##########
@@ -352,6 +353,24 @@ private static void query() throws IoTDBConnectionException, StatementExecutionE
     dataSet.closeOperationHandle();
   }
 
+  private static void rawDataQuery() throws IoTDBConnectionException, StatementExecutionException {
+    List<String> paths = new ArrayList<>();
+    paths.add("root.sg1.d1.s1");
+    paths.add("root.sg1.d1.s2");
+    paths.add("root.sg1.d1.s3");
+    long startTime = 10L;
+    long endTime = 200L;
+
+    SessionDataSet dataSet;
+    dataSet = session.executeRawDataQuery(paths, startTime, endTime);
+    System.out.println(dataSet.getColumnNames());
+    dataSet.setFetchSize(1024); // default is 10000

Review comment:
       removed. Thank you! 




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

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



[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r482893778



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,37 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime
+   * time interval include startTime and exclude endTime
+   * @param paths
+   * @param startTime included
+   * @param endTime excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
+
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+          throws StatementExecutionException, IoTDBConnectionException {
+
+    TSRawDataQueryReq execReq = new TSRawDataQueryReq(sessionId, paths, startTime, endTime);
+    execReq.setFetchSize(fetchSize);
+
+    TSExecuteStatementResp execResp;
+    try {
+      execResp = client.executeRawDataQuery(execReq);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+
+    RpcUtils.verifySuccess(execResp.getStatus());
+    return new SessionDataSet("", execResp.getColumns(), execResp.getDataTypeList(),

Review comment:
       It seems that the statement(sql) in SessionDataSet is not needed. We could remove the statement field in TSFetchResultsReq in rpc.thrift.




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

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



[GitHub] [incubator-iotdb] haimeiguo commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
haimeiguo commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r481993941



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,38 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime
+   * time interval include startTime and exclude endTime
+   * @param paths
+   * @param startTime included
+   * @param endTime excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
+
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+          throws StatementExecutionException, IoTDBConnectionException {
+
+    String statement = SessionUtils.rawDataQuery;

Review comment:
       Got it. Thank you Jialin for your comments!
   




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

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



[GitHub] [incubator-iotdb] haimeiguo commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
haimeiguo commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r479065618



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/Planner.java
##########
@@ -65,6 +72,46 @@ public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr, ZoneId zoneId)
     return physicalGenerator.transformToPhysicalPlan(operator);
   }
 
+  /**
+   * convert raw data query to physical plan directly
+   */
+  public PhysicalPlan rawDataQueryToPhysicalPlan(List<String> paths, long startTime, long endTime)

Review comment:
       Got it, thank you Chao for your comments!




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

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



[GitHub] [incubator-iotdb] neuyilan commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
neuyilan commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r480983309



##########
File path: example/session/src/main/java/org/apache/iotdb/SessionExample.java
##########
@@ -352,6 +353,24 @@ private static void query() throws IoTDBConnectionException, StatementExecutionE
     dataSet.closeOperationHandle();
   }
 
+  private static void rawDataQuery() throws IoTDBConnectionException, StatementExecutionException {
+    List<String> paths = new ArrayList<>();
+    paths.add("root.sg1.d1.s1");
+    paths.add("root.sg1.d1.s2");
+    paths.add("root.sg1.d1.s3");
+    long startTime = 10L;
+    long endTime = 200L;
+
+    SessionDataSet dataSet;
+    dataSet = session.executeRawDataQuery(paths, startTime, endTime);
+    System.out.println(dataSet.getColumnNames());
+    dataSet.setFetchSize(1024); // default is 10000

Review comment:
       It's better not comment on the end of the line




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

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



[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r481785697



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/Planner.java
##########
@@ -64,6 +77,50 @@ public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr, ZoneId zoneId)
     return physicalGenerator.transformToPhysicalPlan(operator);
   }
 
+  /**
+   * convert raw data query to physical plan directly
+   */
+  public PhysicalPlan rawDataQueryReqToPhysicalPlan(TSRawDataQueryReq rawDataQueryReq)
+      throws QueryProcessException, IllegalPathException {
+    List<String> paths = rawDataQueryReq.getPaths();
+    long startTime = rawDataQueryReq.getStartTime();
+    long endTime = rawDataQueryReq.getEndTime();
+
+    //construct query operator and set its global time filter
+    QueryOperator queryOp = new QueryOperator(SQLConstant.TOK_QUERY);
+    FromOperator fromOp = new FromOperator(SQLConstant.TOK_FROM);
+    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT);
+
+    //iterate the path list and add it to from operator
+    for (String p : paths) {
+      PartialPath path = new PartialPath(p);
+      fromOp.addPrefixTablePath(path);
+    }
+    selectOp.addSelectPath(new PartialPath(""));
+
+    queryOp.setSelectOperator(selectOp);
+    queryOp.setFromOperator(fromOp);
+
+    //set time filter operator
+    FilterOperator filterOp = new FilterOperator(SQLConstant.KW_AND);
+    PartialPath timePath = new PartialPath(TIME);
+    filterOp.setSinglePath(timePath);
+    Set<PartialPath> pathSet = new HashSet<>();
+    pathSet.add(timePath);
+    filterOp.setIsSingle(true);
+    filterOp.setPathSet(pathSet);
+
+    BasicFunctionOperator left = new BasicFunctionOperator(SQLConstant.GREATERTHANOREQUALTO, timePath, Long.toString(startTime));
+    BasicFunctionOperator right = new BasicFunctionOperator(SQLConstant.LESSTHAN, timePath, Long.toString(endTime));

Review comment:
       ```suggestion
       BasicFunctionOperator right = new BasicFunctionOperator(SQLConstant.LESSTHANOREQUALTO, timePath, Long.toString(endTime));
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/Planner.java
##########
@@ -64,6 +77,50 @@ public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr, ZoneId zoneId)
     return physicalGenerator.transformToPhysicalPlan(operator);
   }
 
+  /**
+   * convert raw data query to physical plan directly
+   */
+  public PhysicalPlan rawDataQueryReqToPhysicalPlan(TSRawDataQueryReq rawDataQueryReq)
+      throws QueryProcessException, IllegalPathException {
+    List<String> paths = rawDataQueryReq.getPaths();
+    long startTime = rawDataQueryReq.getStartTime();
+    long endTime = rawDataQueryReq.getEndTime();
+
+    //construct query operator and set its global time filter
+    QueryOperator queryOp = new QueryOperator(SQLConstant.TOK_QUERY);
+    FromOperator fromOp = new FromOperator(SQLConstant.TOK_FROM);
+    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT);
+
+    //iterate the path list and add it to from operator
+    for (String p : paths) {
+      PartialPath path = new PartialPath(p);
+      fromOp.addPrefixTablePath(path);
+    }
+    selectOp.addSelectPath(new PartialPath(""));
+
+    queryOp.setSelectOperator(selectOp);
+    queryOp.setFromOperator(fromOp);
+
+    //set time filter operator
+    FilterOperator filterOp = new FilterOperator(SQLConstant.KW_AND);
+    PartialPath timePath = new PartialPath(TIME);
+    filterOp.setSinglePath(timePath);
+    Set<PartialPath> pathSet = new HashSet<>();
+    pathSet.add(timePath);
+    filterOp.setIsSingle(true);
+    filterOp.setPathSet(pathSet);
+
+    BasicFunctionOperator left = new BasicFunctionOperator(SQLConstant.GREATERTHANOREQUALTO, timePath, Long.toString(startTime));
+    BasicFunctionOperator right = new BasicFunctionOperator(SQLConstant.LESSTHAN, timePath, Long.toString(endTime));

Review comment:
       ```suggestion
       BasicFunctionOperator right = new BasicFunctionOperator(SQLConstant.LESSTHANOREQUALTO, timePath, Long.toString(endTime));
   ```

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,38 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime
+   * time interval include startTime and exclude endTime
+   * @param paths
+   * @param startTime included
+   * @param endTime excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
+
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+          throws StatementExecutionException, IoTDBConnectionException {
+
+    String statement = SessionUtils.rawDataQuery;

Review comment:
       In JDBC or executeQueryStatement in Session, we send a SQL to the server, the SQL is used in a web metric(SqlArgument), which is disabled by default. The SQL is not important.
   
   We could remove the statement field in the rpc. if we need it, we could reconstruct a sql in the server based on other parameters.

##########
File path: thrift/src/main/thrift/rpc.thrift
##########
@@ -238,6 +238,16 @@ struct TSDeleteDataReq {
     4: required i64 endTime
 }
 
+struct TSRawDataQueryReq {
+    1: required i64 sessionId
+    2: required list<string> paths
+    3: optional i32 fetchSize
+    4: required i64 startTime
+    5: required i64 endTime
+    6: required i64 statementId

Review comment:
       The statementId is generated by the server, not the client. So this field is not needed.
   
   You could refer to generateQueryId(boolean isDataQuery) in TSServiceImpl.
   

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,38 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime
+   * time interval include startTime and exclude endTime
+   * @param paths
+   * @param startTime included
+   * @param endTime excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
+
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+          throws StatementExecutionException, IoTDBConnectionException {
+
+    String statement = SessionUtils.rawDataQuery;

Review comment:
       In the old version, we send a sql to the server, the sql is used in a web metric(SqlArgument), this is disable by default, not important.
   
   We could remove the statement arg in rpc, it's a redundant information, even we need it, we could construct a sql in server based on other parameters.

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionUtils.java
##########
@@ -28,6 +28,8 @@
 
 public class SessionUtils {
 
+  public static final String rawDataQuery = "raw data query";

Review comment:
       this could be removed




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

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



[GitHub] [incubator-iotdb] mychaow commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r482737196



##########
File path: thrift/src/main/thrift/rpc.thrift
##########
@@ -238,6 +238,16 @@ struct TSDeleteDataReq {
     4: required i64 endTime
 }
 
+struct TSRawDataQueryReq {
+    1: required i64 sessionId
+    2: required list<string> paths
+    3: optional i32 fetchSize
+    4: required i64 startTime
+    5: required i64 endTime
+    6: required i64 statementId

Review comment:
       we need statementId to record queryId and release resource when close session.




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

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



[GitHub] [incubator-iotdb] haimeiguo closed pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
haimeiguo closed pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656


   


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

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



[GitHub] [incubator-iotdb] mychaow commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r478991909



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/Planner.java
##########
@@ -65,6 +72,46 @@ public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr, ZoneId zoneId)
     return physicalGenerator.transformToPhysicalPlan(operator);
   }
 
+  /**
+   * convert raw data query to physical plan directly
+   */
+  public PhysicalPlan rawDataQueryToPhysicalPlan(List<String> paths, long startTime, long endTime)

Review comment:
       rawDataQueryToPhysicalPlan  ----> rawDataQueryReqToPhysicalPlan
   and use the class rawDataQueryReq as parameter, maybe we will support more complicated query req

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/Planner.java
##########
@@ -65,6 +72,46 @@ public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr, ZoneId zoneId)
     return physicalGenerator.transformToPhysicalPlan(operator);
   }
 
+  /**
+   * convert raw data query to physical plan directly
+   */
+  public PhysicalPlan rawDataQueryToPhysicalPlan(List<String> paths, long startTime, long endTime)
+          throws QueryProcessException {
+    //construct query operator and set its global time filter
+    QueryOperator queryOp = new QueryOperator(SQLConstant.TOK_QUERY);
+    FromOperator fromOp = new FromOperator(SQLConstant.TOK_FROM);
+    SelectOperator selectOp = new SelectOperator(SQLConstant.TOK_SELECT);
+
+    //iterate the path list and add it to from operator
+    for (String p : paths) {
+      Path path = new Path(p);
+      fromOp.addPrefixTablePath(path);
+    }
+    selectOp.addSelectPath(new Path(""));
+
+    queryOp.setSelectOperator(selectOp);
+    queryOp.setFromOperator(fromOp);
+
+    //set time filter operator
+    FilterOperator filterOp = new FilterOperator(SQLConstant.KW_AND);
+    Path timePath = new Path("time");

Review comment:
       "time" should be defined as a constant string in IoTDBConsant

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,27 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)

Review comment:
       add comments,  like the query is select * from paths where time >= startTime and time < endTime, time interval include startTime and exclude endTime




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

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



[GitHub] [incubator-iotdb] mychaow commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r479857825



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/Planner.java
##########
@@ -94,7 +100,7 @@ public PhysicalPlan rawDataQueryToPhysicalPlan(List<String> paths, long startTim
 
     //set time filter operator
     FilterOperator filterOp = new FilterOperator(SQLConstant.KW_AND);
-    Path timePath = new Path("time");
+    Path timePath = new Path(TIME);

Review comment:
        BasicOperatorType also has the raw string "time", it's too ugly, you could replace all this "time" by constant TIME

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -83,6 +83,7 @@
   private ZoneId zoneId;
   private long statementId;
   private int fetchSize;
+  private String rawDataQuery = "raw data query";

Review comment:
       use final to define it as constant, and move the string to other utils class, you may need other query strings later.




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

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



[GitHub] [incubator-iotdb] mychaow commented on a change in pull request #1656: Add raw data query Interface in Session

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #1656:
URL: https://github.com/apache/incubator-iotdb/pull/1656#discussion_r482963956



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -913,6 +914,37 @@ public SessionDataSet executeQueryStatement(String sql)
         execResp.isIgnoreTimeStamp());
   }
 
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime
+   * time interval include startTime and exclude endTime
+   * @param paths
+   * @param startTime included
+   * @param endTime excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
+
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+          throws StatementExecutionException, IoTDBConnectionException {
+
+    TSRawDataQueryReq execReq = new TSRawDataQueryReq(sessionId, paths, startTime, endTime);
+    execReq.setFetchSize(fetchSize);
+
+    TSExecuteStatementResp execResp;
+    try {
+      execResp = client.executeRawDataQuery(execReq);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+
+    RpcUtils.verifySuccess(execResp.getStatus());
+    return new SessionDataSet("", execResp.getColumns(), execResp.getDataTypeList(),

Review comment:
       > It seems that the statement(sql) in SessionDataSet is not needed. We could remove the statement field in TSFetchResultsReq in rpc.thrift.
   
   we should not remove the field, because TSFetchResults has the field in the previous release 0.10. User may use old client to query data from new release, we should keep our system backward compatible.




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

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