You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ji...@apache.org on 2019/12/13 01:37:18 UTC

[incubator-iotdb] branch refactor_session_management updated: replace TS_SessionHandle with sessionId, TSOperationHandle with queryId

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

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


The following commit(s) were added to refs/heads/refactor_session_management by this push:
     new 1e1cdca  replace TS_SessionHandle with sessionId, TSOperationHandle with queryId
1e1cdca is described below

commit 1e1cdcac3609665c2ad0c95953302a13c07bac80
Author: jt2594838 <jt...@163.com>
AuthorDate: Fri Dec 13 09:36:59 2019 +0800

    replace TS_SessionHandle with sessionId, TSOperationHandle with queryId
---
 .../org/apache/iotdb/jdbc/IoTDBConnection.java     | 147 ++---
 .../apache/iotdb/jdbc/IoTDBDatabaseMetadata.java   |  30 +-
 .../jdbc/IoTDBPreparedInsertionStatement.java      |  15 +-
 .../apache/iotdb/jdbc/IoTDBPreparedStatement.java  |  66 +-
 .../org/apache/iotdb/jdbc/IoTDBQueryResultSet.java |  41 +-
 .../java/org/apache/iotdb/jdbc/IoTDBStatement.java |  67 ++-
 .../test/java/org/apache/iotdb/jdbc/BatchTest.java |  34 +-
 .../org/apache/iotdb/jdbc/IoTDBConnectionTest.java |  10 +-
 .../iotdb/jdbc/IoTDBDatabaseMetadataTest.java      |   4 +-
 .../iotdb/jdbc/IoTDBPreparedStatementTest.java     |  38 +-
 .../apache/iotdb/jdbc/IoTDBQueryResultSetTest.java |  56 +-
 .../org/apache/iotdb/jdbc/IoTDBStatementTest.java  |  20 +-
 .../org/apache/iotdb/db/conf/IoTDBConstant.java    |  17 +-
 .../engine/storagegroup/StorageGroupProcessor.java |   2 +-
 .../qp/executor/AbstractQueryProcessExecutor.java  |  22 +-
 .../iotdb/db/qp/executor/QueryProcessExecutor.java |  20 +-
 .../iotdb/db/query/context/QueryContext.java       |  10 +-
 .../db/query/control/QueryResourceManager.java     |  11 +-
 .../iotdb/db/query/executor/EngineQueryRouter.java |   4 +-
 .../resourceRelated/UnseqResourceMergeReader.java  |   2 +-
 .../UnseqResourceReaderByTimestamp.java            |   2 +-
 .../org/apache/iotdb/db/service/StaticResps.java   |  74 +++
 .../org/apache/iotdb/db/service/TSServiceImpl.java | 664 ++++++++++-----------
 .../apache/iotdb/db/integration/IoTDBCloseIT.java  |  12 +-
 .../db/integration/IoTDBMultiStatementsIT.java     |  12 +-
 .../iotdb/db/integration/IoTDBPreparedStmtIT.java  |   2 -
 .../db/integration/IoTDBSequenceDataQueryIT.java   |   9 +-
 .../iotdb/db/integration/IoTDBSeriesReaderIT.java  |  12 +-
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |  10 +-
 service-rpc/rpc-changelist.md                      |   5 +-
 service-rpc/src/main/thrift/rpc.thrift             |  20 +-
 31 files changed, 749 insertions(+), 689 deletions(-)

diff --git a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java
index 901c4b9..eec4021 100644
--- a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java
+++ b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java
@@ -51,10 +51,12 @@ import org.slf4j.LoggerFactory;
 
 
 public class IoTDBConnection implements Connection {
+
   private static final Logger logger = LoggerFactory.getLogger(IoTDBConnection.class);
-  private final TSProtocolVersion protocolVersion = TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V1;
-  public TSIService.Iface client = null;
-  TS_SessionHandle sessionHandle = null;
+  private static final TSProtocolVersion protocolVersion = TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V1;
+  private static final String METHOD_NOT_SUPPORTED = "Method not supported";
+  private TSIService.Iface client = null;
+  private long sessionId = -1;
   private IoTDBConnectionParams params;
   private boolean isClosed = true;
   private SQLWarning warningChain = null;
@@ -75,31 +77,31 @@ public class IoTDBConnection implements Connection {
 
     openTransport();
     if(Config.rpcThriftCompressionEnable) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
+      setClient(new TSIService.Client(new TCompactProtocol(transport)));
     }
     else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
+      setClient(new TSIService.Client(new TBinaryProtocol(transport)));
     }
     // open client session
     openSession();
     // Wrap the client with a thread-safe proxy to serialize the RPC calls
-    client = RpcUtils.newSynchronizedClient(client);
+    setClient(RpcUtils.newSynchronizedClient(getClient()));
     autoCommit = false;
   }
 
   @Override
   public boolean isWrapperFor(Class<?> arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public <T> T unwrap(Class<T> arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public void abort(Executor arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
@@ -112,9 +114,9 @@ public class IoTDBConnection implements Connection {
     if (isClosed) {
       return;
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionHandle);
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
     try {
-      client.closeSession(req);
+      getClient().closeSession(req);
     } catch (TException e) {
       throw new SQLException("Error occurs when closing session at server. Maybe server is down.", e);
     } finally {
@@ -127,32 +129,32 @@ public class IoTDBConnection implements Connection {
 
   @Override
   public void commit() throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public Array createArrayOf(String arg0, Object[] arg1) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public Blob createBlob() throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public Clob createClob() throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public NClob createNClob() throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public SQLXML createSQLXML() throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
@@ -160,7 +162,7 @@ public class IoTDBConnection implements Connection {
     if (isClosed) {
       throw new SQLException("Cannot create statement because connection is closed");
     }
-    return new IoTDBStatement(this, client, sessionHandle, zoneId);
+    return new IoTDBStatement(this, getClient(), sessionId, zoneId);
   }
 
   @Override
@@ -175,17 +177,17 @@ public class IoTDBConnection implements Connection {
       throw new SQLException(String.format("Statements with ResultSet type %d are not supported",
           resultSetType));
     }
-    return new IoTDBStatement(this, client, sessionHandle, zoneId);
+    return new IoTDBStatement(this, getClient(), sessionId, zoneId);
   }
 
   @Override
   public Statement createStatement(int arg0, int arg1, int arg2) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public Struct createStruct(String arg0, Object[] arg1) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
@@ -205,22 +207,22 @@ public class IoTDBConnection implements Connection {
 
   @Override
   public void setCatalog(String arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public Properties getClientInfo() throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public void setClientInfo(Properties arg0) throws SQLClientInfoException {
-    throw new SQLClientInfoException("Method not supported", null);
+    throw new SQLClientInfoException(METHOD_NOT_SUPPORTED, null);
   }
 
   @Override
   public String getClientInfo(String arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
@@ -231,7 +233,7 @@ public class IoTDBConnection implements Connection {
 
   @Override
   public void setHoldability(int arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
@@ -239,7 +241,7 @@ public class IoTDBConnection implements Connection {
     if (isClosed) {
       throw new SQLException("Cannot create statement because connection is closed");
     }
-    return new IoTDBDatabaseMetadata(this, client);
+    return new IoTDBDatabaseMetadata(this, getClient(), sessionId);
   }
 
   @Override
@@ -249,12 +251,12 @@ public class IoTDBConnection implements Connection {
 
   @Override
   public String getSchema() throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public void setSchema(String arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
@@ -264,17 +266,17 @@ public class IoTDBConnection implements Connection {
 
   @Override
   public void setTransactionIsolation(int arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public Map<String, Class<?>> getTypeMap() throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public void setTypeMap(Map<String, Class<?>> arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
@@ -294,73 +296,73 @@ public class IoTDBConnection implements Connection {
 
   @Override
   public void setReadOnly(boolean arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public boolean isValid(int arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public String nativeSQL(String arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public CallableStatement prepareCall(String arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public CallableStatement prepareCall(String arg0, int arg1, int arg2) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public CallableStatement prepareCall(String arg0, int arg1, int arg2, int arg3)
       throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql) throws SQLException {
     if (sql.equalsIgnoreCase("INSERT")) {
-      return new IoTDBPreparedInsertionStatement(this, client, sessionHandle, zoneId);
+      return new IoTDBPreparedInsertionStatement(this, getClient(), sessionId, zoneId);
     }
-    return new IoTDBPreparedStatement(this, client, sessionHandle, sql, zoneId);
+    return new IoTDBPreparedStatement(this, getClient(), sessionId, sql, zoneId);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql, String[] columnNames) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency)
       throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency,
       int resultSetHoldability) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public void releaseSavepoint(Savepoint arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
@@ -375,22 +377,34 @@ public class IoTDBConnection implements Connection {
 
   @Override
   public void setClientInfo(String arg0, String arg1) throws SQLClientInfoException {
-    throw new SQLClientInfoException("Method not supported", null);
+    throw new SQLClientInfoException(METHOD_NOT_SUPPORTED, null);
   }
 
   @Override
   public void setNetworkTimeout(Executor arg0, int arg1) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public Savepoint setSavepoint() throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
   }
 
   @Override
   public Savepoint setSavepoint(String arg0) throws SQLException {
-    throw new SQLException("Method not supported");
+    throw new SQLException(METHOD_NOT_SUPPORTED);
+  }
+
+  public TSIService.Iface getClient() {
+    return client;
+  }
+
+  public long getSessionId() {
+    return sessionId;
+  }
+
+  public void setClient(TSIService.Iface client) {
+    this.client = client;
   }
 
   private void openTransport() throws TTransportException {
@@ -406,24 +420,19 @@ public class IoTDBConnection implements Connection {
     openReq.setUsername(params.getUsername());
     openReq.setPassword(params.getPassword());
 
+    TSOpenSessionResp openResp = null;
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
-
+      openResp = client.openSession(openReq);
+      sessionId = openResp.getSessionId();
       // validate connection
-      try {
-        RpcUtils.verifySuccess(openResp.getStatus());
-      } catch (IoTDBRPCException e) {
-        // failed to connect, disconnect from the server
-        transport.close();
-        throw new IoTDBSQLException(e.getMessage(), openResp.getStatus());
-      }
+      RpcUtils.verifySuccess(openResp.getStatus());
+
       if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
         throw new TException(String
             .format("Protocol not supported, Client version is %d, but Server version is %d",
                 protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
       }
       setProtocol(openResp.getServerProtocolVersion());
-      sessionHandle = openResp.getSessionHandle();
 
       if (zoneId != null) {
         setTimeZone(zoneId.toString());
@@ -434,6 +443,10 @@ public class IoTDBConnection implements Connection {
     } catch (TException e) {
       throw new SQLException(String.format("Can not establish connection with %s.",
           params.getJdbcUriString()), e);
+    } catch (IoTDBRPCException e) {
+      // failed to connect, disconnect from the server
+      transport.close();
+      throw new IoTDBSQLException(e.getMessage(), openResp.getStatus());
     }
     isClosed = false;
   }
@@ -446,13 +459,13 @@ public class IoTDBConnection implements Connection {
           transport.close();
           openTransport();
           if(Config.rpcThriftCompressionEnable) {
-            client = new TSIService.Client(new TCompactProtocol(transport));
+            setClient(new TSIService.Client(new TCompactProtocol(transport)));
           }
           else {
-            client = new TSIService.Client(new TBinaryProtocol(transport));
+            setClient(new TSIService.Client(new TBinaryProtocol(transport)));
           }
           openSession();
-          client = RpcUtils.newSynchronizedClient(client);
+          setClient(RpcUtils.newSynchronizedClient(getClient()));
           flag = true;
           break;
         }
@@ -472,7 +485,7 @@ public class IoTDBConnection implements Connection {
       return zoneId.toString();
     }
 
-    TSGetTimeZoneResp resp = client.getTimeZone();
+    TSGetTimeZoneResp resp = getClient().getTimeZone(sessionId);
     try {
       RpcUtils.verifySuccess(resp.getStatus());
     } catch (IoTDBRPCException e) {
@@ -482,8 +495,8 @@ public class IoTDBConnection implements Connection {
   }
 
   public void setTimeZone(String zoneId) throws TException, IoTDBSQLException {
-    TSSetTimeZoneReq req = new TSSetTimeZoneReq(zoneId);
-    TSStatus resp = client.setTimeZone(req);
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp = getClient().setTimeZone(req);
     try {
       RpcUtils.verifySuccess(resp);
     } catch (IoTDBRPCException e) {
@@ -493,7 +506,7 @@ public class IoTDBConnection implements Connection {
   }
 
   public ServerProperties getServerProperties() throws TException {
-    return client.getProperties();
+    return getClient().getProperties();
   }
 
   private void setProtocol(TSProtocolVersion protocol) {
diff --git a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadata.java b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadata.java
index ae61f30..5e10b22 100644
--- a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadata.java
+++ b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadata.java
@@ -38,10 +38,12 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
   private static final Logger logger = LoggerFactory
           .getLogger(IoTDBDatabaseMetadata.class);
   private static final String METHOD_NOT_SUPPORTED_STRING = "Method not supported";
+  private long sessionId;
 
-  IoTDBDatabaseMetadata(IoTDBConnection connection, TSIService.Iface client) {
+  IoTDBDatabaseMetadata(IoTDBConnection connection, TSIService.Iface client, long sessionId) {
     this.connection = connection;
     this.client = client;
+    this.sessionId = sessionId;
   }
 
   @Override
@@ -52,7 +54,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
       return getColumnsFunc(catalog, schemaPattern);
     } catch (TException e) {
       boolean flag = connection.reconnect();
-      this.client = connection.client;
+      this.client = connection.getClient();
       if (flag) {
         try {
           return getColumnsFunc(catalog, schemaPattern);
@@ -77,7 +79,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
     TSFetchMetadataReq req;
     switch (catalog) {
       case Constant.CATALOG_COLUMN:
-        req = new TSFetchMetadataReq(Constant.GLOBAL_COLUMNS_REQ);
+        req = new TSFetchMetadataReq(sessionId, Constant.GLOBAL_COLUMNS_REQ);
         req.setColumnPath(schemaPattern);
         try {
           TSFetchMetadataResp resp = client.fetchMetadata(req);
@@ -91,7 +93,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
           throw new TException("Connection error when fetching column metadata", e);
         }
       case Constant.CATALOG_DEVICES:
-        req = new TSFetchMetadataReq(Constant.GLOBAL_SHOW_DEVICES_REQ);
+        req = new TSFetchMetadataReq(sessionId, Constant.GLOBAL_SHOW_DEVICES_REQ);
         req.setColumnPath(schemaPattern);
         try {
           TSFetchMetadataResp resp = client.fetchMetadata(req);
@@ -105,7 +107,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
           throw new TException("Connection error when fetching device metadata", e);
         }
       case Constant.CATALOG_CHILD_PATHS:
-        req = new TSFetchMetadataReq(Constant.GLOBAL_SHOW_CHILD_PATHS_REQ);
+        req = new TSFetchMetadataReq(sessionId, Constant.GLOBAL_SHOW_CHILD_PATHS_REQ);
         req.setColumnPath(schemaPattern);
         try {
             TSFetchMetadataResp resp = client.fetchMetadata(req);
@@ -119,7 +121,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
           throw new TException("Connection error when fetching child path metadata", e);
         }
       case Constant.CATALOG_STORAGE_GROUP:
-        req = new TSFetchMetadataReq(Constant.GLOBAL_SHOW_STORAGE_GROUP_REQ);
+        req = new TSFetchMetadataReq(sessionId, Constant.GLOBAL_SHOW_STORAGE_GROUP_REQ);
         try {
           TSFetchMetadataResp resp = client.fetchMetadata(req);
           try {
@@ -133,7 +135,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
           throw new TException("Connection error when fetching storage group metadata", e);
         }
       case Constant.CATALOG_TIMESERIES:
-        req = new TSFetchMetadataReq(Constant.GLOBAL_SHOW_TIMESERIES_REQ);
+        req = new TSFetchMetadataReq(sessionId, Constant.GLOBAL_SHOW_TIMESERIES_REQ);
         req.setColumnPath(schemaPattern);
         try {
           TSFetchMetadataResp resp = client.fetchMetadata(req);
@@ -148,7 +150,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
           throw new TException("Connection error when fetching timeseries metadata", e);
         }
       case Constant.COUNT_TIMESERIES:
-        req = new TSFetchMetadataReq(Constant.GLOBAL_COUNT_TIMESERIES_REQ);
+        req = new TSFetchMetadataReq(sessionId, Constant.GLOBAL_COUNT_TIMESERIES_REQ);
         req.setColumnPath(schemaPattern);
         try {
           TSFetchMetadataResp resp = client.fetchMetadata(req);
@@ -173,7 +175,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
       return getNodesFunc(catalog, schemaPattern, nodeLevel);
     } catch (TException e) {
       boolean flag = connection.reconnect();
-      this.client = connection.client;
+      this.client = connection.getClient();
       if (flag) {
         try {
           return getNodesFunc(catalog, schemaPattern, nodeLevel);
@@ -197,7 +199,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
     TSFetchMetadataReq req;
     switch (catalog) {
       case Constant.COUNT_NODES:
-        req = new TSFetchMetadataReq(Constant.GLOBAL_COUNT_NODES_REQ);
+        req = new TSFetchMetadataReq(sessionId, Constant.GLOBAL_COUNT_NODES_REQ);
         req.setNodeLevel(nodeLevel);
         req.setColumnPath(schemaPattern);
         try {
@@ -212,7 +214,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
           throw new TException("Connection error when fetching node metadata", e);
         }
       case Constant.COUNT_NODE_TIMESERIES:
-        req = new TSFetchMetadataReq(Constant.GLOBAL_COUNT_NODE_TIMESERIES_REQ);
+        req = new TSFetchMetadataReq(sessionId, Constant.GLOBAL_COUNT_NODE_TIMESERIES_REQ);
         req.setNodeLevel(nodeLevel);
         req.setColumnPath(schemaPattern);
         try {
@@ -1139,7 +1141,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
       logger.error("Failed to fetch metadata in json because: ", e);
     } catch (TException e) {
       boolean flag = connection.reconnect();
-      this.client = connection.client;
+      this.client = connection.getClient();
       if (flag) {
         try {
           return getMetadataInJsonFunc();
@@ -1165,7 +1167,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
       return getMetadataInJsonFunc();
     } catch (TException e) {
       boolean flag = connection.reconnect();
-      this.client = connection.client;
+      this.client = connection.getClient();
       if (flag) {
         try {
           return getMetadataInJsonFunc();
@@ -1181,7 +1183,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
   }
 
   private String getMetadataInJsonFunc() throws TException, IoTDBSQLException {
-    TSFetchMetadataReq req = new TSFetchMetadataReq("METADATA_IN_JSON");
+    TSFetchMetadataReq req = new TSFetchMetadataReq(sessionId, "METADATA_IN_JSON");
     TSFetchMetadataResp resp;
     resp = client.fetchMetadata(req);
     try {
diff --git a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedInsertionStatement.java b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedInsertionStatement.java
index 4423bc0..e83053c 100644
--- a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedInsertionStatement.java
+++ b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedInsertionStatement.java
@@ -27,25 +27,26 @@ import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
 import org.apache.iotdb.service.rpc.thrift.TSIService.Iface;
 import org.apache.iotdb.service.rpc.thrift.TSInsertionReq;
-import org.apache.iotdb.service.rpc.thrift.TS_SessionHandle;
 import org.apache.thrift.TException;
 
 public class IoTDBPreparedInsertionStatement extends IoTDBPreparedStatement {
 
   private TSInsertionReq req = new TSInsertionReq();
+  private long queryId;
 
-  public IoTDBPreparedInsertionStatement(IoTDBConnection connection,
-      Iface client,
-      TS_SessionHandle sessionHandle, ZoneId zoneId) throws SQLException {
-    super(connection, client, sessionHandle, zoneId);
-    req.setStmtId(stmtId);
+  IoTDBPreparedInsertionStatement(IoTDBConnection connection,
+      Iface client, long sessionId, ZoneId zoneId) throws SQLException {
+    super(connection, client, sessionId, zoneId);
+    req.setSessionId(sessionId);
   }
 
   @Override
   public boolean execute() throws SQLException {
-
     try {
       TSExecuteStatementResp resp = client.insert(req);
+      queryId = resp.getQueryId();
+      req.setQueryId(queryId);
+
       req.unsetDeviceId();
       req.unsetMeasurements();
       req.unsetTimestamp();
diff --git a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java
index 69425e8..a2f9c61 100644
--- a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java
+++ b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java
@@ -18,40 +18,56 @@
  */
 package org.apache.iotdb.jdbc;
 
-import org.apache.iotdb.service.rpc.thrift.TSIService.Iface;
-import org.apache.iotdb.service.rpc.thrift.TS_SessionHandle;
-
 import java.io.InputStream;
 import java.io.Reader;
 import java.math.BigDecimal;
 import java.net.URL;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
 import java.sql.Date;
-import java.sql.*;
+import java.sql.NClob;
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.Ref;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLXML;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.time.Instant;
 import java.time.ZoneId;
 import java.time.ZonedDateTime;
 import java.time.format.DateTimeFormatter;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.service.rpc.thrift.TSIService.Iface;
 
 public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedStatement {
 
   private String sql;
   private static final String METHOD_NOT_SUPPORTED_STRING = "Method not supported";
+  private long queryId;
 
   /**
    * save the SQL parameters as (paramLoc,paramValue) pairs.
    */
   private final Map<Integer, String> parameters = new HashMap<>();
 
-  public IoTDBPreparedStatement(IoTDBConnection connection, Iface client,
-      TS_SessionHandle sessionHandle, ZoneId zoneId) throws SQLException{
-    super(connection, client, sessionHandle, zoneId);
+  IoTDBPreparedStatement(IoTDBConnection connection, Iface client,
+      Long sessionId, ZoneId zoneId) throws SQLException{
+    super(connection, client, sessionId, zoneId);
   }
 
-  public IoTDBPreparedStatement(IoTDBConnection connection, Iface client,
-      TS_SessionHandle sessionHandle, String sql,
+  IoTDBPreparedStatement(IoTDBConnection connection, Iface client,
+      Long sessionId, String sql,
       ZoneId zoneId) throws SQLException {
-    super(connection, client, sessionHandle, zoneId);
+    super(connection, client, sessionId, zoneId);
     this.sql = sql;
   }
 
@@ -61,7 +77,7 @@ public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedSt
   }
 
   @Override
-  public void clearParameters() throws SQLException {
+  public void clearParameters() {
     this.parameters.clear();
   }
 
@@ -147,7 +163,7 @@ public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedSt
   }
 
   @Override
-  public void setBoolean(int parameterIndex, boolean x) throws SQLException {
+  public void setBoolean(int parameterIndex, boolean x) {
     this.parameters.put(parameterIndex, Boolean.toString(x));
   }
 
@@ -204,22 +220,22 @@ public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedSt
   }
 
   @Override
-  public void setDouble(int parameterIndex, double x) throws SQLException {
+  public void setDouble(int parameterIndex, double x) {
     this.parameters.put(parameterIndex, Double.toString(x));
   }
 
   @Override
-  public void setFloat(int parameterIndex, float x) throws SQLException {
+  public void setFloat(int parameterIndex, float x) {
     this.parameters.put(parameterIndex, Float.toString(x));
   }
 
   @Override
-  public void setInt(int parameterIndex, int x) throws SQLException {
+  public void setInt(int parameterIndex, int x) {
     this.parameters.put(parameterIndex, Integer.toString(x));
   }
 
   @Override
-  public void setLong(int parameterIndex, long x) throws SQLException {
+  public void setLong(int parameterIndex, long x) {
     this.parameters.put(parameterIndex, Long.toString(x));
   }
 
@@ -269,15 +285,15 @@ public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedSt
     if (x instanceof String) {
       setString(parameterIndex, (String) x);
     } else if (x instanceof Integer) {
-      setInt(parameterIndex, ((Integer) x).intValue());
+      setInt(parameterIndex, (Integer) x);
     } else if (x instanceof Long) {
-      setLong(parameterIndex, ((Long) x).longValue());
+      setLong(parameterIndex, (Long) x);
     } else if (x instanceof Float) {
-      setFloat(parameterIndex, ((Float) x).floatValue());
+      setFloat(parameterIndex, (Float) x);
     } else if (x instanceof Double) {
-      setDouble(parameterIndex, ((Double) x).doubleValue());
+      setDouble(parameterIndex, (Double) x);
     } else if (x instanceof Boolean) {
-      setBoolean(parameterIndex, ((Boolean) x).booleanValue());
+      setBoolean(parameterIndex, (Boolean) x);
     } else if (x instanceof Timestamp) {
       setTimestamp(parameterIndex, (Timestamp) x);
     } else {
@@ -321,7 +337,7 @@ public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedSt
   }
 
   @Override
-  public void setString(int parameterIndex, String x) throws SQLException {
+  public void setString(int parameterIndex, String x) {
     this.parameters.put(parameterIndex, "'" + x.replace("'", "\\'") + "'");
   }
 
@@ -336,7 +352,7 @@ public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedSt
   }
 
   @Override
-  public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException {
+  public void setTimestamp(int parameterIndex, Timestamp x) {
     ZonedDateTime zonedDateTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(x.getTime()),
         super.zoneId);
     this.parameters.put(parameterIndex, zonedDateTime
@@ -406,7 +422,7 @@ public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedSt
           break;
       }
     }
-    parts.add(sql.substring(off, sql.length()));
+    parts.add(sql.substring(off));
     return parts;
 
   }
diff --git a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBQueryResultSet.java b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBQueryResultSet.java
index d40b955..375b41d 100644
--- a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBQueryResultSet.java
+++ b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBQueryResultSet.java
@@ -42,12 +42,12 @@ public class IoTDBQueryResultSet implements ResultSet {
 
   private static final String TIMESTAMP_STR = "Time";
   private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
   private Statement statement = null;
   private String sql;
   private SQLWarning warningChain = null;
   private boolean isClosed = false;
   private TSIService.Iface client = null;
-  private TSOperationHandle operationHandle = null;
   private List<String> columnInfoList; // no deduplication
   private List<String> columnTypeList; // no deduplication
   private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
@@ -60,8 +60,9 @@ public class IoTDBQueryResultSet implements ResultSet {
   private byte[] time; // used to cache the current time value
   private byte[][] values; // used to cache the current row record value
   private byte[] currentBitmap; // used to cache the current bitmap for every column
-  private static final int flag = 0x80; // used to do `and` operation with bitmap to judge whether the value is null
+  private static final int FLAG = 0x80; // used to do `and` operation with bitmap to judge whether the value is null
 
+  private long sessionId;
   private long queryId;
   private boolean ignoreTimeStamp = false;
 
@@ -71,7 +72,7 @@ public class IoTDBQueryResultSet implements ResultSet {
 
   public IoTDBQueryResultSet(Statement statement, List<String> columnNameList,
       List<String> columnTypeList, boolean ignoreTimeStamp, TSIService.Iface client,
-      TSOperationHandle operationHandle, String sql, long queryId)
+      String sql, long queryId, long sessionId)
       throws SQLException {
     this.statement = statement;
     this.fetchSize = statement.getFetchSize();
@@ -99,9 +100,9 @@ public class IoTDBQueryResultSet implements ResultSet {
 
     this.ignoreTimeStamp = ignoreTimeStamp;
     this.client = client;
-    this.operationHandle = operationHandle;
     this.sql = sql;
     this.queryId = queryId;
+    this.sessionId = sessionId;
   }
 
   @Override
@@ -145,18 +146,19 @@ public class IoTDBQueryResultSet implements ResultSet {
       return;
     }
 
-    closeOperationHandle();
+    if (client != null) {
+      closeOperationHandle();
+    }
     client = null;
     isClosed = true;
   }
 
   private void closeOperationHandle() throws SQLException {
     try {
-      if (operationHandle != null) {
-        TSCloseOperationReq closeReq = new TSCloseOperationReq(operationHandle, queryId);
-        TSStatus closeResp = client.closeOperation(closeReq);
-        RpcUtils.verifySuccess(closeResp);
-      }
+      TSCloseOperationReq closeReq = new TSCloseOperationReq(sessionId);
+      closeReq.setQueryId(queryId);
+      TSStatus closeResp = client.closeOperation(closeReq);
+      RpcUtils.verifySuccess(closeResp);
     } catch (IoTDBRPCException e) {
       throw new SQLException("Error occurs for close opeation in server side becasuse " + e);
     } catch (TException e) {
@@ -254,8 +256,7 @@ public class IoTDBQueryResultSet implements ResultSet {
       return BytesUtils.bytesToBool(values[index]);
     }
     else {
-      throw new SQLException(
-          String.format("The value got by %s (column name) is NULL.", columnName));
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
     }
   }
 
@@ -343,7 +344,7 @@ public class IoTDBQueryResultSet implements ResultSet {
     }
     else {
       throw new SQLException(
-              String.format("The value got by %s (column name) is NULL.", columnName));
+              String.format(VALUE_IS_NULL, columnName));
     }
   }
 
@@ -381,7 +382,7 @@ public class IoTDBQueryResultSet implements ResultSet {
     }
     else {
       throw new SQLException(
-              String.format("The value got by %s (column name) is NULL.", columnName));
+              String.format(VALUE_IS_NULL, columnName));
     }
   }
 
@@ -404,7 +405,7 @@ public class IoTDBQueryResultSet implements ResultSet {
     }
     else {
       throw new SQLException(
-              String.format("The value got by %s (column name) is NULL.", columnName));
+              String.format(VALUE_IS_NULL, columnName));
     }
   }
 
@@ -425,12 +426,12 @@ public class IoTDBQueryResultSet implements ResultSet {
     }
     else {
       throw new SQLException(
-              String.format("The value got by %s (column name) is NULL.", columnName));
+              String.format(VALUE_IS_NULL, columnName));
     }
   }
 
   @Override
-  public ResultSetMetaData getMetaData() throws SQLException {
+  public ResultSetMetaData getMetaData() {
     return new IoTDBResultMetadata(columnInfoList, columnTypeList);
   }
 
@@ -595,7 +596,7 @@ public class IoTDBQueryResultSet implements ResultSet {
   }
 
   @Override
-  public int getType() throws SQLException {
+  public int getType() {
     return ResultSet.TYPE_FORWARD_ONLY;
   }
 
@@ -672,7 +673,7 @@ public class IoTDBQueryResultSet implements ResultSet {
   @Override
   public boolean next() throws SQLException {
     if ((tsQueryDataSet == null || !tsQueryDataSet.time.hasRemaining()) && !emptyResultSet) {
-      TSFetchResultsReq req = new TSFetchResultsReq(sql, fetchSize, queryId);
+      TSFetchResultsReq req = new TSFetchResultsReq(sessionId, sql, fetchSize, queryId);
       try {
         TSFetchResultsResp resp = client.fetchResults(req);
         try {
@@ -758,7 +759,7 @@ public class IoTDBQueryResultSet implements ResultSet {
   private boolean isNull(int index, int rowNum) {
     byte bitmap = currentBitmap[index];
     int shift = rowNum % 8;
-    return ((flag >>> shift) & bitmap) == 0;
+    return ((FLAG >>> shift) & bitmap) == 0;
   }
 
   @Override
diff --git a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java
index f726c3b..7739d62 100644
--- a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java
+++ b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java
@@ -39,9 +39,7 @@ import org.apache.iotdb.service.rpc.thrift.TSExecuteBatchStatementResp;
 import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
 import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
 import org.apache.iotdb.service.rpc.thrift.TSIService;
-import org.apache.iotdb.service.rpc.thrift.TSOperationHandle;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
-import org.apache.iotdb.service.rpc.thrift.TS_SessionHandle;
 import org.apache.thrift.TException;
 
 public class IoTDBStatement implements Statement {
@@ -60,8 +58,6 @@ public class IoTDBStatement implements Statement {
   private int fetchSize;
   private int queryTimeout = 10;
   protected TSIService.Iface client;
-  private TS_SessionHandle sessionHandle;
-  private TSOperationHandle operationHandle = null;
   private List<String> batchSQLList;
   /**
    * Keep state so we can fail certain calls made after close().
@@ -85,17 +81,19 @@ public class IoTDBStatement implements Statement {
    */
   private SQLWarning warningChain = null;
 
-  long stmtId = -1;
+  private long sessionId;
+  private long stmtId = -1;
+  private long queryId = -1;
 
   /**
    * Constructor of IoTDBStatement.
    */
   IoTDBStatement(IoTDBConnection connection, TSIService.Iface client,
-      TS_SessionHandle sessionHandle,
+      Long sessionId,
       int fetchSize, ZoneId zoneId) throws SQLException {
     this.connection = connection;
     this.client = client;
-    this.sessionHandle = sessionHandle;
+    this.sessionId = sessionId;
     this.fetchSize = fetchSize;
     this.batchSQLList = new ArrayList<>();
     this.zoneId = zoneId;
@@ -104,10 +102,10 @@ public class IoTDBStatement implements Statement {
 
   // only for test
   IoTDBStatement(IoTDBConnection connection, TSIService.Iface client,
-      TS_SessionHandle sessionHandle, ZoneId zoneId, long statementId) throws SQLException {
+      Long sessionId, ZoneId zoneId, long statementId) {
     this.connection = connection;
     this.client = client;
-    this.sessionHandle = sessionHandle;
+    this.sessionId = sessionId;
     this.fetchSize = Config.fetchSize;
     this.batchSQLList = new ArrayList<>();
     this.zoneId = zoneId;
@@ -115,9 +113,9 @@ public class IoTDBStatement implements Statement {
   }
 
   IoTDBStatement(IoTDBConnection connection, TSIService.Iface client,
-      TS_SessionHandle sessionHandle,
+      Long sessionId,
       ZoneId zoneId) throws SQLException {
-    this(connection, client, sessionHandle, Config.fetchSize, zoneId);
+    this(connection, client, sessionId, Config.fetchSize, zoneId);
   }
 
   @Override
@@ -145,8 +143,8 @@ public class IoTDBStatement implements Statement {
       return;
     }
     try {
-      if (operationHandle != null) {
-        TSCancelOperationReq closeReq = new TSCancelOperationReq(operationHandle);
+      if (queryId != -1) {
+        TSCancelOperationReq closeReq = new TSCancelOperationReq(sessionId, queryId);
         TSStatus closeResp = client.cancelOperation(closeReq);
         RpcUtils.verifySuccess(closeResp);
       }
@@ -171,11 +169,11 @@ public class IoTDBStatement implements Statement {
 
   private void closeClientOperation() throws SQLException {
     try {
-      if (operationHandle != null) {
-        TSCloseOperationReq closeReq = new TSCloseOperationReq(operationHandle, -1);
-        closeReq.setStmtId(stmtId);
+      if (sessionId != -1) {
+        TSCloseOperationReq closeReq = new TSCloseOperationReq(sessionId);
         TSStatus closeResp = client.closeOperation(closeReq);
         RpcUtils.verifySuccess(closeResp);
+        sessionId = -1;
       }
     } catch (Exception e) {
       throw new SQLException("Error occurs when closing statement.", e);
@@ -243,6 +241,7 @@ public class IoTDBStatement implements Statement {
   private boolean executeSQL(String sql) throws TException, SQLException {
     isCancelled = false;
     String sqlToLowerCase = sql.toLowerCase().trim();
+    // TODO: use normal query instead of metadata query
     if (sqlToLowerCase.startsWith(SHOW_TIMESERIES_COMMAND_LOWERCASE)) {
       if (sqlToLowerCase.equals(SHOW_TIMESERIES_COMMAND_LOWERCASE)) {
         DatabaseMetaData databaseMetaData = connection.getMetaData();
@@ -316,19 +315,18 @@ public class IoTDBStatement implements Statement {
         return true;
       }
     } else {
-      TSExecuteStatementReq execReq = new TSExecuteStatementReq(sessionHandle, sql, stmtId);
+      TSExecuteStatementReq execReq = new TSExecuteStatementReq(sessionId, sql, stmtId);
       TSExecuteStatementResp execResp = client.executeStatement(execReq);
-      operationHandle = execResp.getOperationHandle();
       try {
         RpcUtils.verifySuccess(execResp.getStatus());
       } catch (IoTDBRPCException e) {
         throw new IoTDBSQLException(e.getMessage(), execResp.getStatus());
       }
-      if (execResp.getOperationHandle().hasResultSet) {
+      if (execResp.isSetColumns()) {
+        queryId = execResp.getQueryId();
         this.resultSet = new IoTDBQueryResultSet(this,
             execResp.getColumns(), execResp.getDataTypeList(),
-            execResp.ignoreTimeStamp, client, operationHandle, sql,
-            operationHandle.getOperationId().getQueryId());
+            execResp.ignoreTimeStamp, client, sql, queryId, sessionId);
         return true;
       }
       return false;
@@ -358,7 +356,7 @@ public class IoTDBStatement implements Statement {
 
   private int[] executeBatchSQL() throws TException, SQLException {
     isCancelled = false;
-    TSExecuteBatchStatementReq execReq = new TSExecuteBatchStatementReq(sessionHandle,
+    TSExecuteBatchStatementReq execReq = new TSExecuteBatchStatementReq(sessionId,
         batchSQLList);
     TSExecuteBatchStatementResp execResp = client.executeBatchStatement(execReq);
     if (execResp.getStatus().getStatusType().getCode() == TSStatusCode.SUCCESS_STATUS
@@ -417,17 +415,16 @@ public class IoTDBStatement implements Statement {
 
   private ResultSet executeQuerySQL(String sql) throws TException, SQLException {
     isCancelled = false;
-    TSExecuteStatementReq execReq = new TSExecuteStatementReq(sessionHandle, sql, stmtId);
+    TSExecuteStatementReq execReq = new TSExecuteStatementReq(sessionId, sql, stmtId);
     TSExecuteStatementResp execResp = client.executeQueryStatement(execReq);
-    operationHandle = execResp.getOperationHandle();
+    queryId = execResp.getQueryId();
     try {
       RpcUtils.verifySuccess(execResp.getStatus());
     } catch (IoTDBRPCException e) {
       throw new IoTDBSQLException(e.getMessage(), execResp.getStatus());
     }
     this.resultSet = new IoTDBQueryResultSet(this, execResp.getColumns(),
-        execResp.getDataTypeList(), execResp.ignoreTimeStamp, client, operationHandle, sql,
-        operationHandle.getOperationId().getQueryId());
+        execResp.getDataTypeList(), execResp.ignoreTimeStamp, client, sql, queryId, sessionId);
     return resultSet;
   }
 
@@ -470,9 +467,11 @@ public class IoTDBStatement implements Statement {
   }
 
   private int executeUpdateSQL(String sql) throws TException, IoTDBSQLException {
-    TSExecuteStatementReq execReq = new TSExecuteStatementReq(sessionHandle, sql, stmtId);
+    TSExecuteStatementReq execReq = new TSExecuteStatementReq(sessionId, sql, stmtId);
     TSExecuteStatementResp execResp = client.executeUpdateStatement(execReq);
-    operationHandle = execResp.getOperationHandle();
+    if (execResp.isSetQueryId()) {
+      queryId = execResp.getQueryId();
+    }
     try {
       RpcUtils.verifySuccess(execResp.getStatus());
     } catch (IoTDBRPCException e) {
@@ -634,17 +633,17 @@ public class IoTDBStatement implements Statement {
   }
 
   private void reInit() {
-    this.client = connection.client;
-    this.sessionHandle = connection.sessionHandle;
+    this.client = connection.getClient();
+    this.sessionId = connection.getSessionId();
   }
 
   private void requestStmtId() throws SQLException {
     try {
-      this.stmtId = client.requestStatementId();
+      this.stmtId = client.requestStatementId(sessionId);
     } catch (TException e) {
       if (reConnect()) {
         try {
-          this.stmtId = client.requestStatementId();
+          this.stmtId = client.requestStatementId(sessionId);
         } catch (TException e2) {
           throw new SQLException(
               "Cannot get id for statement after reconnecting. please check server status",
@@ -663,4 +662,8 @@ public class IoTDBStatement implements Statement {
     reInit();
     return flag;
   }
+
+  public long getSessionId() {
+    return sessionId;
+  }
 }
diff --git a/jdbc/src/test/java/org/apache/iotdb/jdbc/BatchTest.java b/jdbc/src/test/java/org/apache/iotdb/jdbc/BatchTest.java
index 1566d49..f19edd9 100644
--- a/jdbc/src/test/java/org/apache/iotdb/jdbc/BatchTest.java
+++ b/jdbc/src/test/java/org/apache/iotdb/jdbc/BatchTest.java
@@ -18,14 +18,10 @@
  */
 package org.apache.iotdb.jdbc;
 
-import org.apache.iotdb.rpc.TSStatusCode;
-import org.apache.iotdb.service.rpc.thrift.*;
-import org.apache.thrift.TException;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.when;
 
 import java.sql.BatchUpdateException;
 import java.sql.SQLException;
@@ -33,11 +29,18 @@ import java.sql.Statement;
 import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.when;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteBatchStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteBatchStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.iotdb.service.rpc.thrift.TSStatusType;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
 
 public class BatchTest {
 
@@ -45,8 +48,7 @@ public class BatchTest {
   private IoTDBConnection connection;
   @Mock
   private TSIService.Iface client;
-  @Mock
-  private TS_SessionHandle sessHandle;
+  private long sessionId;
   @Mock
   private IoTDBStatement statement;
   private TSStatusType successStatus = new TSStatusType(TSStatusCode.SUCCESS_STATUS.getStatusCode(), "");
@@ -60,7 +62,7 @@ public class BatchTest {
   public void setUp() throws Exception {
     MockitoAnnotations.initMocks(this);
     when(connection.createStatement())
-        .thenReturn(new IoTDBStatement(connection, client, sessHandle, zoneID, 1L));
+        .thenReturn(new IoTDBStatement(connection, client, sessionId, zoneID, 1L));
 
   }
 
diff --git a/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBConnectionTest.java b/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBConnectionTest.java
index 76b26a2..787c48b 100644
--- a/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBConnectionTest.java
+++ b/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBConnectionTest.java
@@ -42,6 +42,7 @@ public class IoTDBConnectionTest {
   private IoTDBConnection connection = new IoTDBConnection();
   private TSStatusType successStatus = new TSStatusType(TSStatusCode.SUCCESS_STATUS.getStatusCode(), "");
   private TSStatus Status_SUCCESS = new TSStatus(successStatus);
+  private long sessionId;
 
   @Before
   public void setUp() throws Exception {
@@ -57,7 +58,7 @@ public class IoTDBConnectionTest {
     String timeZone = "Asia/Shanghai";
     when(client.setTimeZone(any(TSSetTimeZoneReq.class)))
         .thenReturn(new TSStatus(Status_SUCCESS));
-    connection.client = client;
+    connection.setClient(client);
     connection.setTimeZone(timeZone);
     assertEquals(connection.getTimeZone(), timeZone);
   }
@@ -65,8 +66,9 @@ public class IoTDBConnectionTest {
   @Test
   public void testGetTimeZone() throws IoTDBSQLException, TException {
     String timeZone = "GMT+:08:00";
-    when(client.getTimeZone()).thenReturn(new TSGetTimeZoneResp(Status_SUCCESS, timeZone));
-    connection.client = client;
+    sessionId = connection.getSessionId();
+    when(client.getTimeZone(sessionId)).thenReturn(new TSGetTimeZoneResp(Status_SUCCESS, timeZone));
+    connection.setClient(client);
     assertEquals(connection.getTimeZone(), timeZone);
   }
 
@@ -82,7 +84,7 @@ public class IoTDBConnectionTest {
     final String timestampPrecision = "ms";
     when(client.getProperties())
         .thenReturn(new ServerProperties(version, supportedAggregationTime, timestampPrecision));
-    connection.client = client;
+    connection.setClient(client);
     assertEquals(connection.getServerProperties().getVersion(), version);
     for (int i = 0; i < supportedAggregationTime.size(); i++) {
       assertEquals(connection.getServerProperties().getSupportedTimeAggregationOperations().get(i),
diff --git a/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadataTest.java b/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadataTest.java
index 43f79e2..735c149 100644
--- a/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadataTest.java
+++ b/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadataTest.java
@@ -63,13 +63,15 @@ public class IoTDBDatabaseMetadataTest {
   private TSStatus Status_SUCCESS = new TSStatus(successStatus);
 
   private DatabaseMetaData databaseMetaData;
+  private long sessionId;
 
   @Before
   public void before() throws Exception {
     MockitoAnnotations.initMocks(this);
-    when(connection.getMetaData()).thenReturn(new IoTDBDatabaseMetadata(connection, client));
+    when(connection.getMetaData()).thenReturn(new IoTDBDatabaseMetadata(connection, client, sessionId));
 
     when(client.fetchMetadata(any(TSFetchMetadataReq.class))).thenReturn(fetchMetadataResp);
+    when(client.closeOperation(any(TSCloseOperationReq.class))).thenReturn(Status_SUCCESS);
     when(fetchMetadataResp.getStatus()).thenReturn(Status_SUCCESS);
 
     databaseMetaData = connection.getMetaData();
diff --git a/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java b/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java
index 9702ed4..b052252 100644
--- a/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java
+++ b/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java
@@ -48,18 +48,16 @@ public class IoTDBPreparedStatementTest {
   @Mock
   private Iface client;
   @Mock
-  private TS_SessionHandle sessHandle;
   private TSStatusType successStatus = new TSStatusType(TSStatusCode.SUCCESS_STATUS.getStatusCode(), "");
   private TSStatus Status_SUCCESS = new TSStatus(successStatus);
-
-  @Mock
-  private TSOperationHandle tOperationHandle;
+  private long queryId;
+  private long sessionId;
 
   @Before
   public void before() throws Exception {
     MockitoAnnotations.initMocks(this);
     when(execStatementResp.getStatus()).thenReturn(Status_SUCCESS);
-    when(execStatementResp.getOperationHandle()).thenReturn(tOperationHandle);
+    when(execStatementResp.getQueryId()).thenReturn(queryId);
 
     when(client.executeStatement(any(TSExecuteStatementReq.class))).thenReturn(execStatementResp);
   }
@@ -68,7 +66,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void testNonParameterized() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 24 and time > 2017-11-1 0:13:00";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.execute();
 
@@ -84,7 +82,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void unusedArgument() throws SQLException {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 24 and time > 2017-11-1 0:13:00";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setString(1, "123");
     ps.execute();
@@ -94,7 +92,7 @@ public class IoTDBPreparedStatementTest {
   @Test(expected = SQLException.class)
   public void unsetArgument() throws SQLException {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 24 and time > ?";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.execute();
   }
@@ -103,7 +101,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void oneIntArgument() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < ? and time > 2017-11-1 0:13:00";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setInt(1, 123);
     ps.execute();
@@ -119,7 +117,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void oneLongArgument() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < ? and time > 2017-11-1 0:13:00";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setLong(1, 123);
     ps.execute();
@@ -135,7 +133,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void oneFloatArgument() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < ? and time > 2017-11-1 0:13:00";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setFloat(1, 123.133f);
     ps.execute();
@@ -151,7 +149,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void oneDoubleArgument() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < ? and time > 2017-11-1 0:13:00";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setDouble(1, 123.456);
     ps.execute();
@@ -167,7 +165,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void oneBooleanArgument() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < ? and time > 2017-11-1 0:13:00";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setBoolean(1, false);
     ps.execute();
@@ -183,7 +181,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void oneStringArgument() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < ? and time > 2017-11-1 0:13:00";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setString(1, "abcde");
     ps.execute();
@@ -199,7 +197,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void oneTimeLongArgument() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE time > ?";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setLong(1, 1233);
     ps.execute();
@@ -214,7 +212,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void oneTimeTimestampArgument() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE time > ?";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setTimestamp(1, Timestamp.valueOf("2017-11-01 00:13:00"));
     ps.execute();
@@ -230,7 +228,7 @@ public class IoTDBPreparedStatementTest {
   @Test
   public void escapingOfStringArgument() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE status = '134' and temperature = ?";
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setLong(1, 1333);
     ps.execute();
@@ -248,7 +246,7 @@ public class IoTDBPreparedStatementTest {
   public void pastingIntoEscapedQuery() throws Exception {
     String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE status = '\\044e' || temperature = ?";
 
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setDouble(1, -1323.0);
     ps.execute();
@@ -266,7 +264,7 @@ public class IoTDBPreparedStatementTest {
   public void testInsertStatement1() throws Exception {
     String sql = "INSERT INTO root.ln.wf01.wt01(timestamp,a,b,c,d,e,f) VALUES(?,?,?,?,?,?,?)";
 
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setLong(1, 12324);
     ps.setBoolean(2, false);
@@ -290,7 +288,7 @@ public class IoTDBPreparedStatementTest {
   public void testInsertStatement2() throws Exception {
     String sql = "INSERT INTO root.ln.wf01.wt01(timestamp,a,b,c,d,e,f) VALUES(?,?,?,?,?,?,?)";
 
-    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessHandle, sql,
+    IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql,
         zoneId);
     ps.setTimestamp(1, Timestamp.valueOf("2017-11-01 00:13:00"));
     ps.setBoolean(2, false);
diff --git a/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBQueryResultSetTest.java b/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBQueryResultSetTest.java
index 83a5044..e9d8132 100644
--- a/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBQueryResultSetTest.java
+++ b/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBQueryResultSetTest.java
@@ -18,14 +18,11 @@
  */
 package org.apache.iotdb.jdbc;
 
-import org.apache.iotdb.rpc.TSStatusCode;
-import org.apache.iotdb.service.rpc.thrift.*;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
@@ -39,9 +36,24 @@ import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.*;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.service.rpc.thrift.TSCloseOperationReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSFetchMetadataReq;
+import org.apache.iotdb.service.rpc.thrift.TSFetchMetadataResp;
+import org.apache.iotdb.service.rpc.thrift.TSFetchResultsReq;
+import org.apache.iotdb.service.rpc.thrift.TSFetchResultsResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSQueryDataSet;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.iotdb.service.rpc.thrift.TSStatusType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
 
 /*
     This class is designed to test the function of TsfileQueryResultSet.
@@ -93,17 +105,13 @@ public class IoTDBQueryResultSetTest {
 
   @Mock
   TSExecuteStatementResp execResp;
-  @Mock
-  TSOperationHandle operationHandle;
-  @Mock
-  TSHandleIdentifier handleIdentifier;
+  private long queryId;
+  private long sessionId;
   @Mock
   private IoTDBConnection connection;
   @Mock
   private TSIService.Iface client;
   @Mock
-  private TS_SessionHandle sessHandle;
-  @Mock
   private Statement statement;
   @Mock
   private TSFetchMetadataResp fetchMetadataResp;
@@ -119,16 +127,11 @@ public class IoTDBQueryResultSetTest {
   public void before() throws Exception {
     MockitoAnnotations.initMocks(this);
 
-    statement = new IoTDBStatement(connection, client, sessHandle, zoneID);
+    statement = new IoTDBStatement(connection, client, sessionId, zoneID);
 
     when(connection.isClosed()).thenReturn(false);
     when(client.executeStatement(any(TSExecuteStatementReq.class))).thenReturn(execResp);
-    operationHandle.hasResultSet = true;
-    operationHandle.operationId = handleIdentifier;
-    handleIdentifier.queryId = 1L;
-    when(execResp.getOperationHandle()).thenReturn(operationHandle);
-    when(operationHandle.getOperationId()).thenReturn(handleIdentifier);
-    when(handleIdentifier.getQueryId()).thenReturn(1L);
+    when(execResp.getQueryId()).thenReturn(queryId);
     when(execResp.getStatus()).thenReturn(Status_SUCCESS);
 
     when(client.fetchMetadata(any(TSFetchMetadataReq.class))).thenReturn(fetchMetadataResp);
@@ -137,8 +140,7 @@ public class IoTDBQueryResultSetTest {
     when(client.fetchResults(any(TSFetchResultsReq.class))).thenReturn(fetchResultsResp);
     when(fetchResultsResp.getStatus()).thenReturn(Status_SUCCESS);
 
-    TSStatus closeResp = new TSStatus();
-    closeResp = Status_SUCCESS;
+    TSStatus closeResp = Status_SUCCESS;
     when(client.closeOperation(any(TSCloseOperationReq.class))).thenReturn(closeResp);
   }
 
@@ -169,6 +171,8 @@ public class IoTDBQueryResultSetTest {
     when(execResp.getColumns()).thenReturn(columns);
     when(execResp.getDataTypeList()).thenReturn(dataTypeList);
     when(execResp.getOperationType()).thenReturn("QUERY");
+    when(execResp.isSetQueryId()).thenReturn(true);
+    when(execResp.getQueryId()).thenReturn(queryId);
     doReturn("FLOAT").doReturn("INT64").doReturn("INT32").doReturn("FLOAT").when(fetchMetadataResp)
         .getDataType();
 
diff --git a/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBStatementTest.java b/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBStatementTest.java
index 534a5e0..b7d7080 100644
--- a/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBStatementTest.java
+++ b/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBStatementTest.java
@@ -37,7 +37,6 @@ import org.apache.iotdb.service.rpc.thrift.TSFetchMetadataResp;
 import org.apache.iotdb.service.rpc.thrift.TSIService.Iface;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
 import org.apache.iotdb.service.rpc.thrift.TSStatusType;
-import org.apache.iotdb.service.rpc.thrift.TS_SessionHandle;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -53,8 +52,7 @@ public class IoTDBStatementTest {
   @Mock
   private Iface client;
 
-  @Mock
-  private TS_SessionHandle sessHandle;
+  private long sessionId;
 
   @Mock
   private TSFetchMetadataResp fetchMetadataResp;
@@ -66,7 +64,7 @@ public class IoTDBStatementTest {
   @Before
   public void setUp() throws Exception {
     MockitoAnnotations.initMocks(this);
-    when(connection.getMetaData()).thenReturn(new IoTDBDatabaseMetadata(connection, client));
+    when(connection.getMetaData()).thenReturn(new IoTDBDatabaseMetadata(connection, client, sessionId));
     when(connection.isClosed()).thenReturn(false);
     when(client.fetchMetadata(any(TSFetchMetadataReq.class))).thenReturn(fetchMetadataResp);
     when(fetchMetadataResp.getStatus()).thenReturn(Status_SUCCESS);
@@ -79,7 +77,7 @@ public class IoTDBStatementTest {
   @SuppressWarnings({"resource", "serial"})
   @Test
   public void testExecuteSQL1() throws SQLException {
-    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessHandle, zoneID);
+    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessionId, zoneID);
     List<List<String>> tsList = new ArrayList<>();
     tsList.add(new ArrayList<String>(4) {
       {
@@ -136,7 +134,7 @@ public class IoTDBStatementTest {
   @SuppressWarnings({"resource", "serial"})
   @Test
   public void testExecuteSQL2() throws SQLException {
-    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessHandle, zoneID);
+    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessionId, zoneID);
     List<List<String>> tsList = new ArrayList<>();
     tsList.add(new ArrayList<String>(4) {
       {
@@ -192,7 +190,7 @@ public class IoTDBStatementTest {
   @SuppressWarnings({"resource"})
   @Test
   public void testExecuteSQL3() throws SQLException {
-    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessHandle, zoneID);
+    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessionId, zoneID);
     Set<String> sgSet = new HashSet<>();
     sgSet.add("root.vehicle");
     when(fetchMetadataResp.getStorageGroups()).thenReturn(sgSet);
@@ -223,7 +221,7 @@ public class IoTDBStatementTest {
   @SuppressWarnings("resource")
   @Test
   public void testSetFetchSize1() throws SQLException {
-    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessHandle, zoneID);
+    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessionId, zoneID);
     stmt.setFetchSize(123);
     assertEquals(123, stmt.getFetchSize());
   }
@@ -231,7 +229,7 @@ public class IoTDBStatementTest {
   @SuppressWarnings("resource")
   @Test
   public void testSetFetchSize2() throws SQLException {
-    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessHandle, zoneID);
+    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessionId, zoneID);
     int initial = stmt.getFetchSize();
     stmt.setFetchSize(0);
     assertEquals(initial, stmt.getFetchSize());
@@ -241,14 +239,14 @@ public class IoTDBStatementTest {
   @Test
   public void testSetFetchSize3() throws SQLException {
     final int fetchSize = 10000;
-    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessHandle, fetchSize, zoneID);
+    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessionId, fetchSize, zoneID);
     assertEquals(fetchSize, stmt.getFetchSize());
   }
 
   @SuppressWarnings("resource")
   @Test(expected = SQLException.class)
   public void testSetFetchSize4() throws SQLException {
-    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessHandle, zoneID);
+    IoTDBStatement stmt = new IoTDBStatement(connection, client, sessionId, zoneID);
     stmt.setFetchSize(-1);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
index f632631..7bf9f64 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
@@ -55,16 +55,17 @@ public class IoTDBConstant {
   public static final int MIN_SUPPORTED_JDK_VERSION = 8;
 
   // show info
-  public static final String ITEM = "                             item";
-  public static final String PARAMETER = "                                          parameter";
-  public static final String VALUE = "value";
+  public static final String COLUMN_ITEM = "                             item";
+  public static final String COLUMN_PARAMETER = "                                          parameter";
+  public static final String COLUMN_VALUE = "value";
+  public static final String COLUMN_VERSION = "version        ";
 
-  public static final String ROLE = "role";
-  public static final String USER = "user";
-  public static final String PRIVILEGE = "privilege";
+  public static final String COLUMN_ROLE = "role";
+  public static final String COLUMN_USER = "user";
+  public static final String COLUMN_PRIVILEGE = "privilege";
 
-  public static final String STORAGE_GROUP = "storage group";
-  public static final String TTL = "ttl";
+  public static final String COLUMN_STORAGE_GROUP = "storage group";
+  public static final String COLUMN_TTL = "ttl";
 
   // data folder name
   public static final String SEQUENCE_FLODER_NAME = "sequence";
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index 7e19a27..6f42bb2 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -734,7 +734,7 @@ public class StorageGroupProcessor {
       // running merge
       // is null only in tests
       if (filePathsManager != null) {
-        filePathsManager.addUsedFilesForGivenJob(context.getJobId(), dataSource);
+        filePathsManager.addUsedFilesForGivenJob(context.getQueryId(), dataSource);
       }
       dataSource.setDataTTL(dataTTL);
       return dataSource;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/AbstractQueryProcessExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/AbstractQueryProcessExecutor.java
index a0fa7c1..9dc5afe 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/AbstractQueryProcessExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/AbstractQueryProcessExecutor.java
@@ -19,11 +19,11 @@
 package org.apache.iotdb.db.qp.executor;
 
 
-import static org.apache.iotdb.db.conf.IoTDBConstant.ITEM;
-import static org.apache.iotdb.db.conf.IoTDBConstant.PARAMETER;
-import static org.apache.iotdb.db.conf.IoTDBConstant.STORAGE_GROUP;
-import static org.apache.iotdb.db.conf.IoTDBConstant.TTL;
-import static org.apache.iotdb.db.conf.IoTDBConstant.VALUE;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ITEM;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PARAMETER;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_STORAGE_GROUP;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TTL;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_VALUE;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -99,8 +99,8 @@ public abstract class AbstractQueryProcessExecutor implements IQueryProcessExecu
 
   private QueryDataSet processShowTTLQuery(ShowTTLPlan showTTLPlan) {
     List<Path> paths = new ArrayList<>();
-    paths.add(new Path(STORAGE_GROUP));
-    paths.add(new Path(TTL));
+    paths.add(new Path(COLUMN_STORAGE_GROUP));
+    paths.add(new Path(COLUMN_TTL));
     List<TSDataType> dataTypes = new ArrayList<>();
     dataTypes.add(TSDataType.TEXT);
     dataTypes.add(TSDataType.INT64);
@@ -149,8 +149,8 @@ public abstract class AbstractQueryProcessExecutor implements IQueryProcessExecu
 
   private QueryDataSet processShowDynamicParameterQuery() {
     List<Path> paths = new ArrayList<>();
-    paths.add(new Path(PARAMETER));
-    paths.add(new Path(VALUE));
+    paths.add(new Path(COLUMN_PARAMETER));
+    paths.add(new Path(COLUMN_VALUE));
     List<TSDataType> dataTypes = new ArrayList<>();
     dataTypes.add(TSDataType.TEXT);
     dataTypes.add(TSDataType.TEXT);
@@ -177,8 +177,8 @@ public abstract class AbstractQueryProcessExecutor implements IQueryProcessExecu
 
   private QueryDataSet processShowFlushTaskInfo() {
     List<Path> paths = new ArrayList<>();
-    paths.add(new Path(ITEM));
-    paths.add(new Path(VALUE));
+    paths.add(new Path(COLUMN_ITEM));
+    paths.add(new Path(COLUMN_VALUE));
     List<TSDataType> dataTypes = new ArrayList<>();
     dataTypes.add(TSDataType.TEXT);
     dataTypes.add(TSDataType.TEXT);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
index 45b1a32..64d576b 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
@@ -18,9 +18,9 @@
  */
 package org.apache.iotdb.db.qp.executor;
 
-import static org.apache.iotdb.db.conf.IoTDBConstant.PRIVILEGE;
-import static org.apache.iotdb.db.conf.IoTDBConstant.ROLE;
-import static org.apache.iotdb.db.conf.IoTDBConstant.USER;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PRIVILEGE;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ROLE;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_USER;
 import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
 
 import java.io.File;
@@ -705,7 +705,7 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     int index = 0;
     List<Path> headerList = new ArrayList<>();
     List<TSDataType> typeList = new ArrayList<>();
-    headerList.add(new Path(ROLE));
+    headerList.add(new Path(COLUMN_ROLE));
     typeList.add(TSDataType.TEXT);
     ListDataSet dataSet = new ListDataSet(headerList, typeList);
     List<String> roleList = authorizer.listAllRoles();
@@ -723,7 +723,7 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     List<String> userList = authorizer.listAllUsers();
     List<Path> headerList = new ArrayList<>();
     List<TSDataType> typeList = new ArrayList<>();
-    headerList.add(new Path(USER));
+    headerList.add(new Path(COLUMN_USER));
     typeList.add(TSDataType.TEXT);
     int index = 0;
     ListDataSet dataSet = new ListDataSet(headerList, typeList);
@@ -745,7 +745,7 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     }
     List<Path> headerList = new ArrayList<>();
     List<TSDataType> typeList = new ArrayList<>();
-    headerList.add(new Path(USER));
+    headerList.add(new Path(COLUMN_USER));
     typeList.add(TSDataType.TEXT);
     ListDataSet dataSet = new ListDataSet(headerList, typeList);
     List<String> userList = authorizer.listAllUsers();
@@ -769,7 +769,7 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     if (user != null) {
       List<Path> headerList = new ArrayList<>();
       List<TSDataType> typeList = new ArrayList<>();
-      headerList.add(new Path(ROLE));
+      headerList.add(new Path(COLUMN_ROLE));
       typeList.add(TSDataType.TEXT);
       ListDataSet dataSet = new ListDataSet(headerList, typeList);
       int index = 0;
@@ -792,7 +792,7 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     if (role != null) {
       List<Path> headerList = new ArrayList<>();
       List<TSDataType> typeList = new ArrayList<>();
-      headerList.add(new Path(PRIVILEGE));
+      headerList.add(new Path(COLUMN_PRIVILEGE));
       typeList.add(TSDataType.TEXT);
       ListDataSet dataSet = new ListDataSet(headerList, typeList);
       int index = 0;
@@ -820,8 +820,8 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     }
     List<Path> headerList = new ArrayList<>();
     List<TSDataType> typeList = new ArrayList<>();
-    headerList.add(new Path(ROLE));
-    headerList.add(new Path(PRIVILEGE));
+    headerList.add(new Path(COLUMN_ROLE));
+    headerList.add(new Path(COLUMN_PRIVILEGE));
     typeList.add(TSDataType.TEXT);
     typeList.add(TSDataType.TEXT);
     ListDataSet dataSet = new ListDataSet(headerList, typeList);
diff --git a/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java b/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
index b603888..10f6cbc 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
@@ -44,15 +44,15 @@ public class QueryContext {
    */
   private Map<String, List<Modification>> fileModCache = new HashMap<>();
 
-  private long jobId;
+  private long queryId;
 
   private long queryTimeLowerBound = Long.MIN_VALUE;
 
   public QueryContext() {
   }
 
-  public QueryContext(long jobId) {
-    this.jobId = jobId;
+  public QueryContext(long queryId) {
+    this.queryId = queryId;
   }
 
   /**
@@ -86,8 +86,8 @@ public class QueryContext {
     return pathModifications;
   }
 
-  public long getJobId() {
-    return jobId;
+  public long getQueryId() {
+    return queryId;
   }
 
   public long getQueryTimeLowerBound() {
diff --git a/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java b/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
index 11da9e6..de3d8c9 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicLong;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.exception.StorageEngineException;
@@ -43,7 +42,6 @@ import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
 public class QueryResourceManager {
 
   private JobFileManager filePathsManager;
-  private AtomicLong maxJobId;
   /**
    * Record temporary files used for external sorting.
    *
@@ -53,7 +51,6 @@ public class QueryResourceManager {
 
   private QueryResourceManager() {
     filePathsManager = new JobFileManager();
-    maxJobId = new AtomicLong(0);
     externalSortFileMap = new ConcurrentHashMap<>();
   }
 
@@ -62,13 +59,11 @@ public class QueryResourceManager {
   }
 
   /**
-   * Assign a jobId for a new query job. When a query request is created firstly, this method must
+   * Register a new query. When a query request is created firstly, this method must
    * be invoked.
    */
-  public long assignJobId() {
-    long jobId = maxJobId.incrementAndGet();
-    filePathsManager.addJobId(jobId);
-    return jobId;
+  public void registerQuery(long queryId) {
+    filePathsManager.addJobId(queryId);
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java b/server/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java
index 267f915..375a1b9 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java
@@ -109,7 +109,7 @@ public class EngineQueryRouter implements IEngineQueryRouter {
           throws QueryFilterOptimizationException, StorageEngineException,
           QueryProcessException, IOException {
 
-    long nextJobId = context.getJobId();
+    long nextJobId = context.getQueryId();
 
     GlobalTimeExpression timeExpression = new GlobalTimeExpression(new GroupByFilter(unit, slidingStep, startTime, endTime, FilterType.GROUP_BY_FILTER));
 
@@ -139,7 +139,7 @@ public class EngineQueryRouter implements IEngineQueryRouter {
       QueryContext context)
       throws StorageEngineException, QueryProcessException, IOException {
 
-    long nextJobId = context.getJobId();
+    long nextJobId = context.getQueryId();
 
     FillEngineExecutor fillEngineExecutor = new FillEngineExecutor(nextJobId, fillPaths, queryTime,
         fillType);
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/resourceRelated/UnseqResourceMergeReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/resourceRelated/UnseqResourceMergeReader.java
index 8c61711..3c655d6 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/resourceRelated/UnseqResourceMergeReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/resourceRelated/UnseqResourceMergeReader.java
@@ -57,7 +57,7 @@ public class UnseqResourceMergeReader extends PriorityMergeReader {
   public UnseqResourceMergeReader(Path seriesPath, List<TsFileResource> unseqResources,
       QueryContext context, Filter filter) throws IOException {
     this.seriesPath = seriesPath;
-    this.queryId = context.getJobId();
+    this.queryId = context.getQueryId();
 
     List<ChunkReaderWrap> readerWrapList = new ArrayList<>();
     for (TsFileResource tsFileResource : unseqResources) {
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/resourceRelated/UnseqResourceReaderByTimestamp.java b/server/src/main/java/org/apache/iotdb/db/query/reader/resourceRelated/UnseqResourceReaderByTimestamp.java
index 139e859..be1651a 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/resourceRelated/UnseqResourceReaderByTimestamp.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/resourceRelated/UnseqResourceReaderByTimestamp.java
@@ -52,7 +52,7 @@ public class UnseqResourceReaderByTimestamp extends PriorityMergeReaderByTimesta
 
   public UnseqResourceReaderByTimestamp(Path seriesPath,
       List<TsFileResource> unseqResources, QueryContext context) throws IOException {
-    this.queryId = context.getJobId();
+    this.queryId = context.getQueryId();
     List<ChunkReaderWrap> chunkReaderWrapList = new ArrayList<>();
     for (TsFileResource tsFileResource : unseqResources) {
 
diff --git a/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java b/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java
new file mode 100644
index 0000000..7fb8403
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at      http://www.apache.org/licenses/LICENSE-2.0  Unless required by applicable law or ag [...]
+ */
+
+package org.apache.iotdb.db.service;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ITEM;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PARAMETER;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PRIVILEGE;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ROLE;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_STORAGE_GROUP;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TTL;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_USER;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_VALUE;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_VERSION;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+/**
+ * Static responses that won't change for all requests.
+ */
+class StaticResps {
+
+  private StaticResps() {
+    // enum-like class
+  }
+
+  static final TSExecuteStatementResp TTL_RESP = getNoTimeExecuteResp(
+      Arrays.asList(COLUMN_STORAGE_GROUP, COLUMN_TTL),
+      Arrays.asList(TSDataType.TEXT.toString(), TSDataType.INT64.toString()));
+
+  static final TSExecuteStatementResp FLUSH_INFO_RESP = getNoTimeExecuteResp(
+      Arrays.asList(COLUMN_ITEM, COLUMN_VALUE),
+      Arrays.asList(TSDataType.TEXT.toString(), TSDataType.TEXT.toString()));
+
+  static final TSExecuteStatementResp DYNAMIC_PARAMETER_RESP = getNoTimeExecuteResp(
+      Arrays.asList(COLUMN_PARAMETER, COLUMN_VALUE),
+      Arrays.asList(TSDataType.TEXT.toString(), TSDataType.TEXT.toString()));
+
+  static final TSExecuteStatementResp SHOW_VERSION_RESP = getNoTimeExecuteResp(
+      Collections.singletonList(COLUMN_VERSION),
+      Collections.singletonList(TSDataType.TEXT.toString()));
+
+  static final TSExecuteStatementResp LIST_ROLE_RESP = getNoTimeExecuteResp(
+      Collections.singletonList(COLUMN_ROLE),
+      Collections.singletonList(TSDataType.TEXT.toString()));
+
+  static final TSExecuteStatementResp LIST_USER_RESP = getNoTimeExecuteResp(
+      Collections.singletonList(COLUMN_USER),
+      Collections.singletonList(TSDataType.TEXT.toString()));
+
+  static final TSExecuteStatementResp LIST_USER_PRIVILEGE_RESP = getNoTimeExecuteResp(
+      Arrays.asList(COLUMN_ROLE, COLUMN_PRIVILEGE),
+      Arrays.asList(TSDataType.TEXT.toString(), TSDataType.TEXT.toString()));
+
+  static final TSExecuteStatementResp LIST_ROLE_PRIVILEGE_RESP = getNoTimeExecuteResp(
+      Collections.singletonList(COLUMN_PRIVILEGE),
+      Collections.singletonList(TSDataType.TEXT.toString()));
+
+  private static TSExecuteStatementResp getNoTimeExecuteResp(List<String> columns,
+      List<String> dataTypes) {
+    TSExecuteStatementResp resp =
+        TSServiceImpl.getTSExecuteStatementResp(TSServiceImpl.getStatus(TSStatusCode.SUCCESS_STATUS));
+    resp.setIgnoreTimeStamp(true);
+    resp.setColumns(columns);
+    resp.setDataTypeList(dataTypes);
+    return resp;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index b18e38c..8a12832 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -18,6 +18,25 @@
  */
 package org.apache.iotdb.db.service;
 
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PRIVILEGE;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ROLE;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_STORAGE_GROUP;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TTL;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_USER;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Vector;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 import org.antlr.v4.runtime.misc.ParseCancellationException;
@@ -48,14 +67,46 @@ import org.apache.iotdb.db.qp.physical.crud.BatchInsertPlan;
 import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
-import org.apache.iotdb.db.qp.physical.sys.*;
+import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.tools.watermark.GroupedLSBWatermarkEncoder;
 import org.apache.iotdb.db.tools.watermark.WatermarkEncoder;
 import org.apache.iotdb.db.utils.QueryDataSetUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
-import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.service.rpc.thrift.ServerProperties;
+import org.apache.iotdb.service.rpc.thrift.TSBatchInsertionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCancelOperationReq;
+import org.apache.iotdb.service.rpc.thrift.TSCloseOperationReq;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteBatchStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteBatchStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteInsertRowInBatchResp;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSFetchMetadataReq;
+import org.apache.iotdb.service.rpc.thrift.TSFetchMetadataResp;
+import org.apache.iotdb.service.rpc.thrift.TSFetchResultsReq;
+import org.apache.iotdb.service.rpc.thrift.TSFetchResultsResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertInBatchReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSProtocolVersion;
+import org.apache.iotdb.service.rpc.thrift.TSQueryDataSet;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.iotdb.service.rpc.thrift.TSStatusType;
 import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
@@ -68,15 +119,6 @@ import org.apache.thrift.server.ServerContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.time.ZoneId;
-import java.util.*;
-
-import static org.apache.iotdb.db.conf.IoTDBConstant.*;
-
 /**
  * Thrift RPC implementation at server side.
  */
@@ -87,12 +129,13 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   private static final String INFO_NOT_LOGIN = "{}: Not login.";
   private static final int MAX_SIZE = 200;
   private static final int DELETE_SIZE = 50;
+  private static final String ERROR_PARSING_SQL = "meet error while parsing SQL to physical plan: {}";
   public static Vector<SqlArgument> sqlArgumentsList = new Vector<>();
 
   protected QueryProcessor processor;
   // Record the username for every rpc connection. Username.get() is null if
   // login is failed.
-  protected Map<Long, String> usernameMap = new ConcurrentHashMap<>();
+  private Map<Long, String> usernameMap = new ConcurrentHashMap<>();
   private Map<Long, ZoneId> zoneIds = new ConcurrentHashMap<>();
   
   // The sessionId is unique in one IoTDB instance.
@@ -101,17 +144,21 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   private AtomicLong statementIdGenerator = new AtomicLong();
   // The queryIdGenerator is unique in one IoTDB instance for each operation that needs an id.
   private AtomicLong queryIdGenerator = new AtomicLong();
+
+  // (sessionId -> Set(statementId))
+  private Map<Long, Set<Long>> sessionId2StatementId = new ConcurrentHashMap<>();
   // (statement -> Set(queryId))
   private Map<Long, Set<Long>> statementId2QueryId = new ConcurrentHashMap<>();
+
   // (queryId -> PhysicalPlan)
   private Map<Long, PhysicalPlan> queryId2Plan = new ConcurrentHashMap<>();
   // (queryId -> QueryDataSet)
   private Map<Long, QueryDataSet> queryId2DataSet = new ConcurrentHashMap<>();
-  // (queryId -> QueryContext)
-  private Map<Long, QueryContext> contextMapLocal = new ConcurrentHashMap<>();
-  
+
   private IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-  
+
+  private ThreadLocal<Long> currSessionId = new ThreadLocal<>();
+
   public TSServiceImpl() {
     processor = new QueryProcessor(new QueryProcessExecutor());
   }
@@ -119,12 +166,12 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   public static TSDataType getSeriesType(String path) throws QueryProcessException {
     switch (path.toLowerCase()) {
       // authorization queries
-      case ROLE:
-      case USER:
-      case PRIVILEGE:
-      case STORAGE_GROUP:
+      case COLUMN_ROLE:
+      case COLUMN_USER:
+      case COLUMN_PRIVILEGE:
+      case COLUMN_STORAGE_GROUP:
         return TSDataType.TEXT;
-      case TTL:
+      case COLUMN_TTL:
         return TSDataType.INT64;
       default:
         // do nothing
@@ -175,79 +222,58 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       status = false;
     }
     TSStatus tsStatus;
+    long sessionId = -1;
     if (status) {
       tsStatus = getStatus(TSStatusCode.SUCCESS_STATUS, "Login successfully");
-      username.set(req.getUsername());
-      zoneIds.set(config.getZoneID());
-      initForOneSession();
+      sessionId = sessionIdGenerator.incrementAndGet();
+      usernameMap.put(sessionId, req.getUsername());
+      zoneIds.put(sessionId, config.getZoneID());
+      currSessionId.set(sessionId);
     } else {
       tsStatus = getStatus(TSStatusCode.WRONG_LOGIN_PASSWORD_ERROR);
     }
     TSOpenSessionResp resp = new TSOpenSessionResp(tsStatus,
         TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V1);
-    resp.setSessionHandle(
-        new TS_SessionHandle(new TSHandleIdentifier(ByteBuffer.wrap(req.getUsername().getBytes()),
-            ByteBuffer.wrap(req.getPassword().getBytes()), -1L)));
+    resp.setSessionId(sessionId);
     logger.info("{}: Login status: {}. User : {}", IoTDBConstant.GLOBAL_DB_NAME,
         tsStatus.getStatusType().getMessage(), req.getUsername());
 
     return resp;
   }
 
-  private void initForOneSession() {
-    queryId2Plan.set(new HashMap<>());
-    queryId2DataSet.set(new HashMap<>());
-    queryIdGenerator.set(0L);
-    statementIdGenerator.set(0L);
-    contextMapLocal.set(new HashMap<>());
-    statementId2QueryId.set(new HashMap<>());
-  }
-
   @Override
   public TSStatus closeSession(TSCloseSessionReq req) {
     logger.info("{}: receive close session", IoTDBConstant.GLOBAL_DB_NAME);
+    long sessionId = req.getSessionId();
     TSStatus tsStatus;
-    if (username.get() == null) {
+    if (usernameMap.remove(sessionId) == null) {
       tsStatus = getStatus(TSStatusCode.NOT_LOGIN_ERROR);
     } else {
       tsStatus = getStatus(TSStatusCode.SUCCESS_STATUS);
-      username.remove();
     }
-    if (zoneIds.get() != null) {
-      zoneIds.remove();
-    }
-    // clear the statementId counter
-    if (statementIdGenerator.get() != null) {
-      statementIdGenerator.remove();
-    }
-    // clear the queryId counter
-    if (queryIdGenerator.get() != null) {
-      queryIdGenerator.remove();
-    }
-    // clear all cached physical plans of the connection
-    if (queryId2Plan.get() != null) {
-      queryId2Plan.remove();
-    }
-    // clear all cached ResultSets of the connection
-    if (queryId2DataSet.get() != null) {
-      queryId2DataSet.remove();
-    }
-    // clear all cached query context of the connection
-    if (contextMapLocal.get() != null) {
-      try {
-        for (QueryContext context : contextMapLocal.get().values()) {
-          QueryResourceManager.getInstance().endQueryForGivenJob(context.getJobId());
+
+    zoneIds.remove(sessionId);
+    List<Exception> exceptions = new ArrayList<>();
+    Set<Long> statementIds = sessionId2StatementId.getOrDefault(sessionId, Collections.emptySet());
+    for (long statementId : statementIds) {
+      Set<Long> queryIds = statementId2QueryId.getOrDefault(statementId, Collections.emptySet());
+      for (long queryId : queryIds) {
+        queryId2Plan.remove(queryId);
+        queryId2DataSet.remove(queryId);
+
+        try {
+          QueryResourceManager.getInstance().endQueryForGivenJob(queryId);
+        } catch (StorageEngineException e) {
+          exceptions.add(e);
+          logger.error("Error in closeSession : ", e);
         }
-        contextMapLocal.remove();
-      } catch (StorageEngineException e) {
-        logger.error("Error in closeSession : ", e);
-        return new TSStatus(
-            getStatus(TSStatusCode.CLOSE_OPERATION_ERROR, "Error in closeOperation"));
       }
     }
-    // clear the statementId to queryId map
-    if (statementId2QueryId.get() != null) {
-      statementId2QueryId.remove();
+    if (!exceptions.isEmpty()) {
+      return new TSStatus(
+          getStatus(TSStatusCode.CLOSE_OPERATION_ERROR,
+              String.format("%d errors in closeOperation, see server logs for detail",
+                  exceptions.size())));
     }
 
     return new TSStatus(tsStatus);
@@ -255,26 +281,30 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
 
   @Override
   public TSStatus cancelOperation(TSCancelOperationReq req) {
-    return getStatus(TSStatusCode.SUCCESS_STATUS);
+    //TODO implement
+    return getStatus(TSStatusCode.QUERY_NOT_ALLOWED, "Cancellation is not implemented");
   }
 
   @Override
   public TSStatus closeOperation(TSCloseOperationReq req) {
     logger.info("{}: receive close operation", IoTDBConstant.GLOBAL_DB_NAME);
+    TSStatus status;
+    if (!checkLogin(req.getSessionId())) {
+      logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
+      return getStatus(TSStatusCode.NOT_LOGIN_ERROR);
+    }
     try {
       // statement close
-      if (req.isSetStmtId()) {
-        long stmtId = req.getStmtId();
-        Set<Long> queryIdSet = statementId2QueryId.get().get(stmtId);
+      if (req.isSetStatementId()) {
+        long stmtId = req.getStatementId();
+        Set<Long> queryIdSet = statementId2QueryId.remove(stmtId);
         if (queryIdSet != null) {
           for (long queryId : queryIdSet) {
             releaseQueryResource(queryId);
           }
-          statementId2QueryId.get().remove(stmtId);
         }
-      }
-      // ResultSet close
-      else {
+      } else {
+        // ResultSet close
         releaseQueryResource(req.queryId);
       }
 
@@ -291,18 +321,10 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   private void releaseQueryResource(long queryId) throws StorageEngineException {
 
     // remove the corresponding Physical Plan
-    if (queryId2Plan.get() != null) {
-      queryId2Plan.get().remove(queryId);
-    }
-    // remove the corresponding Dataset
-    if (queryId2DataSet.get() != null) {
-      queryId2DataSet.get().remove(queryId);
-    }
-    // remove the corresponding query context and query resource
-    if (contextMapLocal.get() != null && contextMapLocal.get().containsKey(queryId)) {
-      QueryResourceManager.getInstance()
-          .endQueryForGivenJob(contextMapLocal.get().remove(queryId).getJobId());
-    }
+    queryId2DataSet.remove(queryId);
+    queryId2Plan.remove(queryId);
+
+    QueryResourceManager.getInstance().endQueryForGivenJob(queryId);
   }
 
   /**
@@ -310,7 +332,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
    *
    * @param statusType status type
    */
-  private TSStatus getStatus(TSStatusCode statusType) {
+  static TSStatus getStatus(TSStatusCode statusType) {
     TSStatusType statusCodeAndMessage = new TSStatusType(statusType.getStatusCode(), "");
     return new TSStatus(statusCodeAndMessage);
   }
@@ -329,11 +351,12 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   @Override
   public TSFetchMetadataResp fetchMetadata(TSFetchMetadataReq req) {
     TSStatus status;
-    if (!checkLogin()) {
+    if (!checkLogin(req.getSessionId())) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
       status = getStatus(TSStatusCode.NOT_LOGIN_ERROR);
       return new TSFetchMetadataResp(status);
     }
+
     TSFetchMetadataResp resp = new TSFetchMetadataResp();
     try {
       switch (req.getType()) {
@@ -444,8 +467,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
    * @param statement command
    * @return true if the statement is ADMIN COMMAND
    */
-  private boolean execAdminCommand(String statement) throws StorageEngineException {
-    if (!"root".equals(username.get())) {
+  private boolean execAdminCommand(String statement, long sessionId) throws StorageEngineException {
+    if (!"root".equals(usernameMap.get(sessionId))) {
       return false;
     }
     if (statement == null) {
@@ -473,7 +496,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     long t1 = System.currentTimeMillis();
     List<Integer> result = new ArrayList<>();
     try {
-      if (!checkLogin()) {
+      if (!checkLogin(req.getSessionId())) {
         logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
         return getTSBatchExecuteStatementResp(getStatus(TSStatusCode.NOT_LOGIN_ERROR), null);
       }
@@ -485,7 +508,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       for (String statement : statements) {
         long t2 = System.currentTimeMillis();
         isAllSuccessful =
-            isAllSuccessful && executeStatementInBatch(statement, batchErrorMessage, result);
+            isAllSuccessful && executeStatementInBatch(statement, batchErrorMessage, result,
+                req.getSessionId());
         Measurement.INSTANCE.addOperationLatency(Operation.EXECUTE_ONE_SQL_IN_BATCH, t2);
       }
       if (isAllSuccessful) {
@@ -503,13 +527,13 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   // execute one statement of a batch. Currently, query is not allowed in a batch statement and
   // on finding queries in a batch, such query will be ignored and an error will be generated
   private boolean executeStatementInBatch(String statement, StringBuilder batchErrorMessage,
-      List<Integer> result) {
+      List<Integer> result, long sessionId) {
     try {
-      PhysicalPlan physicalPlan = processor.parseSQLToPhysicalPlan(statement, zoneIds.get());
+      PhysicalPlan physicalPlan = processor.parseSQLToPhysicalPlan(statement, zoneIds.get(sessionId));
       if (physicalPlan.isQuery()) {
         throw new QueryInBatchStatementException(statement);
       }
-      TSExecuteStatementResp resp = executeUpdateStatement(physicalPlan);
+      TSExecuteStatementResp resp = executeUpdateStatement(physicalPlan, sessionId);
       if (resp.getStatus().getStatusType().getCode() == TSStatusCode.SUCCESS_STATUS
           .getStatusCode()) {
         result.add(Statement.SUCCESS_NO_INFO);
@@ -551,21 +575,21 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     TSExecuteStatementResp resp;
     SqlArgument sqlArgument;
     try {
-      if (!checkLogin()) {
+      if (!checkLogin(req.getSessionId())) {
         logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
         return getTSExecuteStatementResp(getStatus(TSStatusCode.NOT_LOGIN_ERROR));
       }
       String statement = req.getStatement();
 
-      if (execAdminCommand(statement)) {
+      if (execAdminCommand(statement, req.getSessionId())) {
         return getTSExecuteStatementResp(
             getStatus(TSStatusCode.SUCCESS_STATUS, "ADMIN_COMMAND_SUCCESS"));
       }
 
       PhysicalPlan physicalPlan;
-      physicalPlan = processor.parseSQLToPhysicalPlan(statement, zoneIds.get());
+      physicalPlan = processor.parseSQLToPhysicalPlan(statement, zoneIds.get(req.getSessionId()));
       if (physicalPlan.isQuery()) {
-        resp = executeQueryStatement(req.statementId, physicalPlan);
+        resp = executeQueryStatement(req.statementId, physicalPlan, usernameMap.get(req.getSessionId()));
         long endTime = System.currentTimeMillis();
         sqlArgument = new SqlArgument(resp, physicalPlan, statement, startTime, endTime);
         sqlArgumentsList.add(sqlArgument);
@@ -574,7 +598,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
         }
         return resp;
       } else {
-        return executeUpdateStatement(physicalPlan);
+        return executeUpdateStatement(physicalPlan, req.getSessionId());
       }
     } catch (ParseCancellationException e) {
       logger.debug(e.getMessage());
@@ -584,11 +608,11 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       return getTSExecuteStatementResp(getStatus(TSStatusCode.METADATA_ERROR,
           "Check metadata error: " + e.getMessage()));
     } catch (QueryProcessException e) {
-      logger.info("meet error while parsing SQL to physical plan: {}", e.getMessage());
+      logger.info(ERROR_PARSING_SQL, e.getMessage());
       return getTSExecuteStatementResp(getStatus(TSStatusCode.SQL_PARSE_ERROR,
           "Statement format is not right: " + e.getMessage()));
     } catch (StorageEngineException e) {
-      logger.info("meet error while parsing SQL to physical plan: {}", e.getMessage());
+      logger.info(ERROR_PARSING_SQL, e.getMessage());
       return getTSExecuteStatementResp(getStatus(TSStatusCode.READ_ONLY_SYSTEM_ERROR,
           e.getMessage()));
     }
@@ -598,7 +622,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
    * @param plan must be a plan for Query: FillQueryPlan, AggregationPlan, GroupByPlan, some
    * AuthorPlan
    */
-  private TSExecuteStatementResp executeQueryStatement(long statementId, PhysicalPlan plan) {
+  private TSExecuteStatementResp executeQueryStatement(long statementId, PhysicalPlan plan,
+      String username) {
     long t1 = System.currentTimeMillis();
     try {
       TSExecuteStatementResp resp; // column headers
@@ -607,7 +632,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       } else if (plan instanceof ShowPlan) {
         resp = getShowQueryColumnHeaders((ShowPlan) plan);
       } else {
-        resp = getQueryColumnHeaders(plan);
+        resp = getQueryColumnHeaders(plan, username);
       }
       if (plan.getOperatorType() == OperatorType.AGGREGATION) {
         resp.setIgnoreTimeStamp(true);
@@ -616,17 +641,10 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       // generate the queryId for the operation
       long queryId = generateQueryId();
       // put it into the corresponding Set
-      Set<Long> queryIdSet = statementId2QueryId.get()
-          .computeIfAbsent(statementId, k -> new HashSet<>());
-      queryIdSet.add(queryId);
-
-      TSHandleIdentifier queryId = new TSHandleIdentifier(
-          ByteBuffer.wrap(username.get().getBytes()), ByteBuffer.wrap("PASS".getBytes()),
-          queryId);
-      TSOperationHandle operationHandle = new TSOperationHandle(queryId, true);
-      resp.setOperationHandle(operationHandle);
+      statementId2QueryId.computeIfAbsent(statementId, k -> new HashSet<>()).add(queryId);
 
-      queryId2Plan.get().put(queryId, plan);
+      resp.setQueryId(queryId);
+      queryId2Plan.put(queryId, plan);
       return resp;
     } catch (Exception e) {
       logger.error("{}: Internal server error: ", IoTDBConstant.GLOBAL_DB_NAME, e);
@@ -639,7 +657,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
 
   @Override
   public TSExecuteStatementResp executeQueryStatement(TSExecuteStatementReq req) {
-    if (!checkLogin()) {
+    if (!checkLogin(req.getSessionId())) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
       return getTSExecuteStatementResp(getStatus(TSStatusCode.NOT_LOGIN_ERROR));
     }
@@ -647,9 +665,9 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     String statement = req.getStatement();
     PhysicalPlan physicalPlan;
     try {
-      physicalPlan = processor.parseSQLToPhysicalPlan(statement, zoneIds.get());
+      physicalPlan = processor.parseSQLToPhysicalPlan(statement, zoneIds.get(req.getSessionId()));
     } catch (QueryProcessException | SQLParserException e) {
-      logger.info("meet error while parsing SQL to physical plan: {}", e.getMessage());
+      logger.info(ERROR_PARSING_SQL, e.getMessage());
       return getTSExecuteStatementResp(getStatus(TSStatusCode.SQL_PARSE_ERROR, e.getMessage()));
     }
 
@@ -657,7 +675,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       return getTSExecuteStatementResp(getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR,
           "Statement is not a query statement."));
     }
-    return executeQueryStatement(req.statementId, physicalPlan);
+    return executeQueryStatement(req.statementId, physicalPlan, usernameMap.get(req.getSessionId()));
   }
 
   private List<String> queryColumnsType(List<String> columns) throws QueryProcessException {
@@ -668,232 +686,164 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return columnTypes;
   }
 
-  private TSExecuteStatementResp getShowQueryColumnHeaders(ShowPlan showPlan) throws Exception {
+  private TSExecuteStatementResp getShowQueryColumnHeaders(ShowPlan showPlan) throws QueryProcessException {
     switch (showPlan.getShowContentType()) {
       case TTL:
-        return executeShowTTL();
+        return StaticResps.TTL_RESP;
       case FLUSH_TASK_INFO:
-        return executeShowFlushTaskInfo();
+        return StaticResps.FLUSH_INFO_RESP;
       case DYNAMIC_PARAMETER:
-        return executeShowDynamicParameter();
+        return StaticResps.DYNAMIC_PARAMETER_RESP;
       case VERSION:
-        return executeShowVersion();
+        return StaticResps.SHOW_VERSION_RESP;
       default:
         logger.error("Unsupported show content type: {}", showPlan.getShowContentType());
-        throw new Exception("Unsupported show content type:" + showPlan.getShowContentType());
+        throw new QueryProcessException("Unsupported show content type:" + showPlan.getShowContentType());
     }
   }
 
-  private TSExecuteStatementResp executeShowTTL() {
-    TSExecuteStatementResp resp =
-        getTSExecuteStatementResp(getStatus(TSStatusCode.SUCCESS_STATUS));
-    resp.setIgnoreTimeStamp(true);
-    List<String> columns = new ArrayList<>();
-    List<String> columnTypes = new ArrayList<>();
-    columns.add(STORAGE_GROUP);
-    columns.add(TTL);
-    columnTypes.add(TSDataType.TEXT.toString());
-    columnTypes.add(TSDataType.INT64.toString());
-    resp.setColumns(columns);
-    resp.setDataTypeList(columnTypes);
-    return resp;
-  }
-
-  private TSExecuteStatementResp executeShowFlushTaskInfo() {
-    TSExecuteStatementResp resp =
-        getTSExecuteStatementResp(getStatus(TSStatusCode.SUCCESS_STATUS));
-    resp.setIgnoreTimeStamp(true);
-    List<String> columns = new ArrayList<>();
-    List<String> columnTypes = new ArrayList<>();
-    columns.add(ITEM);
-    columns.add(VALUE);
-    columnTypes.add(TSDataType.TEXT.toString());
-    columnTypes.add(TSDataType.TEXT.toString());
-    resp.setColumns(columns);
-    resp.setDataTypeList(columnTypes);
-    return resp;
-  }
-
-  private TSExecuteStatementResp executeShowDynamicParameter() {
-    TSExecuteStatementResp resp =
-        getTSExecuteStatementResp(getStatus(TSStatusCode.SUCCESS_STATUS));
-    resp.setIgnoreTimeStamp(true);
-    List<String> columns = new ArrayList<>();
-    List<String> columnTypes = new ArrayList<>();
-    columns.add(PARAMETER);
-    columns.add(VALUE);
-    columnTypes.add(TSDataType.TEXT.toString());
-    columnTypes.add(TSDataType.TEXT.toString());
-    resp.setColumns(columns);
-    resp.setDataTypeList(columnTypes);
-    return resp;
-  }
-
-  private TSExecuteStatementResp executeShowVersion() {
-    TSExecuteStatementResp resp =
-        getTSExecuteStatementResp(getStatus(TSStatusCode.SUCCESS_STATUS));
-    resp.setIgnoreTimeStamp(true);
-    List<String> columns = new ArrayList<>();
-    List<String> columnTypes = new ArrayList<>();
-    columns.add("version        ");
-    columnTypes.add(TSDataType.TEXT.toString());
-    resp.setColumns(columns);
-    resp.setDataTypeList(columnTypes);
-    return resp;
-  }
-
   private TSExecuteStatementResp getAuthQueryColumnHeaders(PhysicalPlan plan) {
-    TSExecuteStatementResp resp = getTSExecuteStatementResp(getStatus(TSStatusCode.SUCCESS_STATUS));
-    resp.setIgnoreTimeStamp(true);
     AuthorPlan authorPlan = (AuthorPlan) plan;
-    List<String> columnsName = new ArrayList<>();
-    List<String> columnsType = new ArrayList<>();
     switch (authorPlan.getAuthorType()) {
       case LIST_ROLE:
       case LIST_USER_ROLES:
-        columnsName.add(ROLE);
-        columnsType.add(TSDataType.TEXT.toString());
-        break;
+        return StaticResps.LIST_ROLE_RESP;
       case LIST_USER:
       case LIST_ROLE_USERS:
-        columnsName.add(USER);
-        columnsType.add(TSDataType.TEXT.toString());
-        break;
+        return StaticResps.LIST_USER_RESP;
       case LIST_ROLE_PRIVILEGE:
-        columnsName.add(PRIVILEGE);
-        columnsType.add(TSDataType.TEXT.toString());
-        break;
+        return StaticResps.LIST_ROLE_PRIVILEGE_RESP;
       case LIST_USER_PRIVILEGE:
-        columnsName.add(ROLE);
-        columnsName.add(PRIVILEGE);
-        columnsType.add(TSDataType.TEXT.toString());
-        columnsType.add(TSDataType.TEXT.toString());
-        break;
+        return StaticResps.LIST_USER_PRIVILEGE_RESP;
       default:
         return getTSExecuteStatementResp(getStatus(TSStatusCode.SQL_PARSE_ERROR,
             String.format("%s is not an auth query", authorPlan.getAuthorType())));
     }
-    resp.setColumns(columnsName);
-    resp.setDataTypeList(columnsType);
-    return resp;
   }
 
 
   /**
    * get ResultSet schema
    */
-  private TSExecuteStatementResp getQueryColumnHeaders(PhysicalPlan plan)
+  private TSExecuteStatementResp getQueryColumnHeaders(PhysicalPlan physicalPlan, String username)
       throws AuthException, TException, QueryProcessException {
-    List<Path> paths = plan.getPaths();
+
     List<String> respColumns = new ArrayList<>();
+    List<String> columnsTypes = new ArrayList<>();
 
     // check permissions
-    if (!checkAuthorization(paths, plan)) {
+    if (!checkAuthorization(physicalPlan.getPaths(), physicalPlan, username)) {
       return getTSExecuteStatementResp(getStatus(TSStatusCode.NO_PERMISSION_ERROR));
     }
 
     TSExecuteStatementResp resp = getTSExecuteStatementResp(getStatus(TSStatusCode.SUCCESS_STATUS));
 
     // group by device query
-    if (((QueryPlan) plan).isGroupByDevice()) {
-      // set columns in TSExecuteStatementResp. Note this is without deduplication.
-      List<String> measurementColumns = ((QueryPlan) plan).getMeasurementColumnList();
-      respColumns.add(SQLConstant.GROUPBY_DEVICE_COLUMN_NAME);
-      respColumns.addAll(measurementColumns);
-      resp.setColumns(respColumns);
-
-      // get column types and do deduplication
-      List<String> columnsType = new ArrayList<>();
-      columnsType.add(TSDataType.TEXT.toString()); // the DEVICE column of GROUP_BY_DEVICE result
-      List<TSDataType> deduplicatedColumnsType = new ArrayList<>();
-      deduplicatedColumnsType.add(TSDataType.TEXT); // the DEVICE column of GROUP_BY_DEVICE result
-      List<String> deduplicatedMeasurementColumns = new ArrayList<>();
-      Set<String> tmpColumnSet = new HashSet<>();
-      Map<String, TSDataType> checker = ((QueryPlan) plan).getDataTypeConsistencyChecker();
-      for (String column : measurementColumns) {
-        TSDataType dataType = checker.get(column);
-        columnsType.add(dataType.toString());
-
-        if (!tmpColumnSet.contains(column)) {
-          // Note that this deduplication strategy is consistent with that of client IoTDBQueryResultSet.
-          tmpColumnSet.add(column);
-          deduplicatedMeasurementColumns.add(column);
-          deduplicatedColumnsType.add(dataType);
-        }
-      }
-
+    QueryPlan plan = (QueryPlan) physicalPlan;
+    if (plan.isGroupByDevice()) {
+      getGroupByDeviceQueryHeaders(plan, respColumns, columnsTypes);
       // set dataTypeList in TSExecuteStatementResp. Note this is without deduplication.
-      resp.setDataTypeList(columnsType);
-
-      // save deduplicated measurementColumn names and types in QueryPlan for the next stage to use.
-      // i.e., used by DeviceIterateDataSet constructor in `fetchResults` stage.
-      ((QueryPlan) plan).setMeasurementColumnList(deduplicatedMeasurementColumns);
-      ((QueryPlan) plan).setDataTypes(deduplicatedColumnsType);
-
-      // set these null since they are never used henceforth in GROUP_BY_DEVICE query processing.
-      ((QueryPlan) plan).setPaths(null);
-      ((QueryPlan) plan).setDataTypeConsistencyChecker(null);
-
+      resp.setColumns(respColumns);
+      resp.setDataTypeList(columnsTypes);
     } else {
-      // Restore column header of aggregate to func(column_name), only
-      // support single aggregate function for now
-      switch (plan.getOperatorType()) {
-        case QUERY:
-        case FILL:
-          for (Path p : paths) {
-            respColumns.add(p.getFullPath());
-          }
-          break;
-        case AGGREGATION:
-        case GROUPBY:
-          List<String> aggregations = plan.getAggregations();
-          if (aggregations.size() != paths.size()) {
-            for (int i = 1; i < paths.size(); i++) {
-              aggregations.add(aggregations.get(0));
-            }
-          }
-          for (int i = 0; i < paths.size(); i++) {
-            respColumns.add(aggregations.get(i) + "(" + paths.get(i).getFullPath() + ")");
-          }
-          break;
-        default:
-          throw new TException("unsupported query type: " + plan.getOperatorType());
-      }
-
+      getWideQueryHeaders(plan, respColumns, columnsTypes);
       resp.setColumns(respColumns);
       resp.setDataTypeList(queryColumnsType(respColumns));
     }
     return resp;
   }
 
+  // wide means not group by device
+  private void getWideQueryHeaders(QueryPlan plan, List<String> respColumns,
+  List<String> columnTypes) throws TException {
+    // Restore column header of aggregate to func(column_name), only
+    // support single aggregate function for now
+    List<Path> paths = plan.getPaths();
+    switch (plan.getOperatorType()) {
+      case QUERY:
+      case FILL:
+        for (Path p : paths) {
+          respColumns.add(p.getFullPath());
+        }
+        break;
+      case AGGREGATION:
+      case GROUPBY:
+        List<String> aggregations = plan.getAggregations();
+        if (aggregations.size() != paths.size()) {
+          for (int i = 1; i < paths.size(); i++) {
+            aggregations.add(aggregations.get(0));
+          }
+        }
+        for (int i = 0; i < paths.size(); i++) {
+          respColumns.add(aggregations.get(i) + "(" + paths.get(i).getFullPath() + ")");
+        }
+        break;
+      default:
+        throw new TException("unsupported query type: " + plan.getOperatorType());
+    }
+  }
+
+  private void getGroupByDeviceQueryHeaders(QueryPlan plan, List<String> respColumns,
+      List<String> columnTypes) {
+    // set columns in TSExecuteStatementResp. Note this is without deduplication.
+    List<String> measurementColumns = plan.getMeasurementColumnList();
+    respColumns.add(SQLConstant.GROUPBY_DEVICE_COLUMN_NAME);
+    respColumns.addAll(measurementColumns);
+
+    // get column types and do deduplication
+    columnTypes.add(TSDataType.TEXT.toString()); // the DEVICE column of GROUP_BY_DEVICE result
+    List<TSDataType> deduplicatedColumnsType = new ArrayList<>();
+    deduplicatedColumnsType.add(TSDataType.TEXT); // the DEVICE column of GROUP_BY_DEVICE result
+    List<String> deduplicatedMeasurementColumns = new ArrayList<>();
+    Set<String> tmpColumnSet = new HashSet<>();
+    Map<String, TSDataType> checker = plan.getDataTypeConsistencyChecker();
+    for (String column : measurementColumns) {
+      TSDataType dataType = checker.get(column);
+      columnTypes.add(dataType.toString());
+
+      if (!tmpColumnSet.contains(column)) {
+        // Note that this deduplication strategy is consistent with that of client IoTDBQueryResultSet.
+        tmpColumnSet.add(column);
+        deduplicatedMeasurementColumns.add(column);
+        deduplicatedColumnsType.add(dataType);
+      }
+    }
+
+    // save deduplicated measurementColumn names and types in QueryPlan for the next stage to use.
+    // i.e., used by DeviceIterateDataSet constructor in `fetchResults` stage.
+    plan.setMeasurementColumnList(deduplicatedMeasurementColumns);
+    plan.setDataTypes(deduplicatedColumnsType);
+
+    // set these null since they are never used henceforth in GROUP_BY_DEVICE query processing.
+    plan.setPaths(null);
+    plan.setDataTypeConsistencyChecker(null);
+  }
+
 
   @Override
   public TSFetchResultsResp fetchResults(TSFetchResultsReq req) {
     try {
-      if (!checkLogin()) {
+      if (!checkLogin(req.getSessionId())) {
         return getTSFetchResultsResp(getStatus(TSStatusCode.NOT_LOGIN_ERROR));
       }
 
-      if (!queryId2Plan.get().containsKey(req.queryId)) {
+      if (!queryId2Plan.containsKey(req.queryId)) {
         return getTSFetchResultsResp(
             getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, "Has not executed statement"));
       }
 
       QueryDataSet queryDataSet;
-      if (!queryId2DataSet.get().containsKey(req.queryId)) {
+      if (!queryId2DataSet.containsKey(req.queryId)) {
         queryDataSet = createNewDataSet(req);
       } else {
-        queryDataSet = queryId2DataSet.get().get(req.queryId);
+        queryDataSet = queryId2DataSet.get(req.queryId);
       }
 
       IAuthorizer authorizer;
-      try {
-        authorizer = LocalFileAuthorizer.getInstance();
-      } catch (AuthException e) {
-        throw new TException(e);
-      }
+      authorizer = LocalFileAuthorizer.getInstance();
+
       TSQueryDataSet result;
-      if (config.isEnableWatermark() && authorizer.isUserUseWaterMark(username.get())) {
+      if (config.isEnableWatermark() && authorizer.isUserUseWaterMark(usernameMap.get(req.getSessionId()))) {
         WatermarkEncoder encoder;
         if (config.getWatermarkMethodName().equals(IoTDBConfig.WATERMARK_GROUPED_LSB)) {
           encoder = new GroupedLSBWatermarkEncoder(config);
@@ -907,8 +857,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
         result = QueryDataSetUtils.convertQueryDataSetByFetchSize(queryDataSet, req.fetchSize);
       }
       boolean hasResultSet = (result.bufferForTime().limit() != 0);
-      if (!hasResultSet && queryId2DataSet.get() != null) {
-        queryId2DataSet.get().remove(req.queryId);
+      if (!hasResultSet) {
+        queryId2DataSet.remove(req.queryId);
       }
 
       TSFetchResultsResp resp = getTSFetchResultsResp(getStatus(TSStatusCode.SUCCESS_STATUS,
@@ -924,37 +874,27 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
 
   private QueryDataSet createNewDataSet(TSFetchResultsReq req)
       throws QueryProcessException, QueryFilterOptimizationException, StorageEngineException, IOException {
-    PhysicalPlan physicalPlan = queryId2Plan.get().get(req.queryId);
+    PhysicalPlan physicalPlan = queryId2Plan.get(req.queryId);
 
     QueryDataSet queryDataSet;
-    QueryContext context = new QueryContext(QueryResourceManager.getInstance().assignJobId());
-
-    initContextMap();
-    contextMapLocal.get().put(req.queryId, context);
+    QueryContext context = new QueryContext(req.getQueryId());
+    QueryResourceManager.getInstance().registerQuery(req.queryId);
 
     queryDataSet = processor.getExecutor().processQuery(physicalPlan, context);
 
-    queryId2DataSet.get().put(req.queryId, queryDataSet);
+    queryId2DataSet.put(req.queryId, queryDataSet);
     return queryDataSet;
   }
 
-  private void initContextMap() {
-    Map<Long, QueryContext> contextMap = contextMapLocal.get();
-    if (contextMap == null) {
-      contextMap = new HashMap<>();
-      contextMapLocal.set(contextMap);
-    }
-  }
-
   @Override
   public TSExecuteStatementResp executeUpdateStatement(TSExecuteStatementReq req) {
     try {
-      if (!checkLogin()) {
+      if (!checkLogin(req.getSessionId())) {
         logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
         return getTSExecuteStatementResp(getStatus(TSStatusCode.NOT_LOGIN_ERROR));
       }
       String statement = req.getStatement();
-      return executeUpdateStatement(statement);
+      return executeUpdateStatement(statement, req.getSessionId());
     } catch (Exception e) {
       logger.error("{}: server Internal Error: ", IoTDBConstant.GLOBAL_DB_NAME, e);
       return getTSExecuteStatementResp(
@@ -962,8 +902,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     }
   }
 
-  private TSExecuteStatementResp executeUpdateStatement(PhysicalPlan plan) {
-    TSStatus status = checkAuthority(plan);
+  private TSExecuteStatementResp executeUpdateStatement(PhysicalPlan plan, long sessionId) {
+    TSStatus status = checkAuthority(plan, sessionId);
     if (status != null) {
       return new TSExecuteStatementResp(status);
     }
@@ -971,12 +911,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     status = executePlan(plan);
     TSExecuteStatementResp resp = getTSExecuteStatementResp(status);
     long queryId = generateQueryId();
-    TSHandleIdentifier queryId = new TSHandleIdentifier(
-        ByteBuffer.wrap(username.get().getBytes()),
-        ByteBuffer.wrap("PASS".getBytes()), queryId);
-    TSOperationHandle operationHandle;
-    operationHandle = new TSOperationHandle(queryId, false);
-    resp.setOperationHandle(operationHandle);
+    resp.setQueryId(queryId);
     return resp;
   }
 
@@ -988,13 +923,13 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return processor.getExecutor().processNonQuery(plan);
   }
 
-  private TSExecuteStatementResp executeUpdateStatement(String statement) {
+  private TSExecuteStatementResp executeUpdateStatement(String statement, long sessionId) {
 
     PhysicalPlan physicalPlan;
     try {
-      physicalPlan = processor.parseSQLToPhysicalPlan(statement, zoneIds.get());
+      physicalPlan = processor.parseSQLToPhysicalPlan(statement, zoneIds.get(sessionId));
     } catch (QueryProcessException | SQLParserException e) {
-      logger.info("meet error while parsing SQL to physical plan: {}", e.getMessage());
+      logger.info(ERROR_PARSING_SQL, e.getMessage());
       return getTSExecuteStatementResp(getStatus(TSStatusCode.SQL_PARSE_ERROR, e.getMessage()));
     }
 
@@ -1003,7 +938,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
           "Statement is a query statement."));
     }
 
-    return executeUpdateStatement(physicalPlan);
+    return executeUpdateStatement(physicalPlan, sessionId);
   }
 
   /**
@@ -1011,27 +946,22 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
    *
    * @return true: If logged in; false: If not logged in
    */
-  private boolean checkLogin() {
-    return username.get() != null;
+  private boolean checkLogin(long sessionId) {
+    return usernameMap.get(sessionId) != null;
   }
 
-  private boolean checkAuthorization(List<Path> paths, PhysicalPlan plan) throws AuthException {
+  private boolean checkAuthorization(List<Path> paths, PhysicalPlan plan, String username) throws AuthException {
     String targetUser = null;
     if (plan instanceof AuthorPlan) {
       targetUser = ((AuthorPlan) plan).getUserName();
     }
-    return AuthorityChecker.check(username.get(), paths, plan.getOperatorType(), targetUser);
+    return AuthorityChecker.check(username, paths, plan.getOperatorType(), targetUser);
   }
 
-  private TSExecuteStatementResp getTSExecuteStatementResp(TSStatus status) {
+  static TSExecuteStatementResp getTSExecuteStatementResp(TSStatus status) {
     TSExecuteStatementResp resp = new TSExecuteStatementResp();
     TSStatus tsStatus = new TSStatus(status);
     resp.setStatus(tsStatus);
-    TSHandleIdentifier queryId = new TSHandleIdentifier(
-        ByteBuffer.wrap(username.get().getBytes()),
-        ByteBuffer.wrap("PASS".getBytes()), generateQueryId());
-    TSOperationHandle operationHandle = new TSOperationHandle(queryId, false);
-    resp.setOperationHandle(operationHandle);
     return resp;
   }
 
@@ -1052,16 +982,17 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   void handleClientExit() {
-    closeSession(null);
+    TSCloseSessionReq req = new TSCloseSessionReq(currSessionId.get());
+    closeSession(req);
   }
 
   @Override
-  public TSGetTimeZoneResp getTimeZone() {
+  public TSGetTimeZoneResp getTimeZone(long sessionId) {
     TSStatus tsStatus;
     TSGetTimeZoneResp resp;
     try {
       tsStatus = getStatus(TSStatusCode.SUCCESS_STATUS);
-      resp = new TSGetTimeZoneResp(tsStatus, zoneIds.get().toString());
+      resp = new TSGetTimeZoneResp(tsStatus, zoneIds.get(sessionId).toString());
     } catch (Exception e) {
       logger.error("meet error while generating time zone.", e);
       tsStatus = getStatus(TSStatusCode.GENERATE_TIME_ZONE_ERROR);
@@ -1075,7 +1006,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     TSStatus tsStatus;
     try {
       String timeZoneID = req.getTimeZone();
-      zoneIds.set(ZoneId.of(timeZoneID));
+      zoneIds.put(req.getSessionId(), ZoneId.of(timeZoneID));
       tsStatus = getStatus(TSStatusCode.SUCCESS_STATUS);
     } catch (Exception e) {
       logger.error("meet error while setting time zone.", e);
@@ -1099,14 +1030,22 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   @Override
   public TSExecuteStatementResp insert(TSInsertionReq req) {
     // TODO need to refactor this when implementing PreparedStatement
-    if (!checkLogin()) {
+    if (!checkLogin(req.getSessionId())) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
       return getTSExecuteStatementResp(getStatus(TSStatusCode.NOT_LOGIN_ERROR));
     }
 
-    long stmtId = req.getStmtId();
-    InsertPlan plan = (InsertPlan) queryId2Plan.get()
-        .computeIfAbsent(stmtId, k -> new InsertPlan());
+    long queryId;
+    InsertPlan plan;
+    if (req.isSetQueryId()) {
+      plan = (InsertPlan) queryId2Plan.get(req.getQueryId());
+      queryId = req.getQueryId();
+    } else {
+      queryId = generateQueryId();
+      plan = new InsertPlan();
+      queryId2Plan.put(queryId, plan);
+    }
+
 
     // the old parameter will be used if new parameter is not set
     if (req.isSetDeviceId()) {
@@ -1123,7 +1062,9 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     }
 
     try {
-      return executeUpdateStatement(plan);
+      TSExecuteStatementResp resp = executeUpdateStatement(plan, req.getSessionId());
+      resp.setQueryId(queryId);
+      return resp;
     } catch (Exception e) {
       logger.info("meet error while executing an insertion into {}", req.getDeviceId(), e);
       return getTSExecuteStatementResp(
@@ -1134,7 +1075,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   @Override
   public TSExecuteInsertRowInBatchResp insertRowInBatch(TSInsertInBatchReq req) {
     TSExecuteInsertRowInBatchResp resp = new TSExecuteInsertRowInBatchResp();
-    if (!checkLogin()) {
+    if (!checkLogin(req.getSessionId())) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
       resp.addToStatusList(getStatus(TSStatusCode.NOT_LOGIN_ERROR));
       return resp;
@@ -1146,7 +1087,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       plan.setTime(req.getTimestamps().get(i));
       plan.setMeasurements(req.getMeasurementsList().get(i).toArray(new String[0]));
       plan.setValues(req.getValuesList().get(i).toArray(new String[0]));
-      TSStatus status = checkAuthority(plan);
+      TSStatus status = checkAuthority(plan, req.getSessionId());
       if (status != null) {
         resp.addToStatusList(new TSStatus(status));
       } else {
@@ -1158,7 +1099,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSExecuteBatchStatementResp testInsertBatch(TSBatchInsertionReq req) throws TException {
+  public TSExecuteBatchStatementResp testInsertBatch(TSBatchInsertionReq req) {
     logger.debug("Test insert batch request receive.");
     TSExecuteBatchStatementResp resp = new TSExecuteBatchStatementResp();
     resp.setStatus(getStatus(TSStatusCode.SUCCESS_STATUS));
@@ -1167,14 +1108,13 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSStatus testInsertRow(TSInsertReq req) throws TException {
+  public TSStatus testInsertRow(TSInsertReq req) {
     logger.debug("Test insert row request receive.");
     return getStatus(TSStatusCode.SUCCESS_STATUS);
   }
 
   @Override
-  public TSExecuteInsertRowInBatchResp testInsertRowInBatch(TSInsertInBatchReq req)
-      throws TException {
+  public TSExecuteInsertRowInBatchResp testInsertRowInBatch(TSInsertInBatchReq req) {
     logger.debug("Test insert row in batch request receive.");
 
     TSExecuteInsertRowInBatchResp resp = new TSExecuteInsertRowInBatchResp();
@@ -1185,7 +1125,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
 
   @Override
   public TSStatus insertRow(TSInsertReq req) {
-    if (!checkLogin()) {
+    if (!checkLogin(req.getSessionId())) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
       return getStatus(TSStatusCode.NOT_LOGIN_ERROR);
     }
@@ -1196,7 +1136,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     plan.setMeasurements(req.getMeasurements().toArray(new String[0]));
     plan.setValues(req.getValues().toArray(new String[0]));
 
-    TSStatus status = checkAuthority(plan);
+    TSStatus status = checkAuthority(plan, req.getSessionId());
     if (status != null) {
       return new TSStatus(status);
     }
@@ -1205,7 +1145,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
 
   @Override
   public TSStatus deleteData(TSDeleteDataReq req) {
-    if (!checkLogin()) {
+    if (!checkLogin(req.getSessionId())) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
       return getStatus(TSStatusCode.NOT_LOGIN_ERROR);
     }
@@ -1218,7 +1158,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     }
     plan.addPaths(paths);
 
-    TSStatus status = checkAuthority(plan);
+    TSStatus status = checkAuthority(plan, req.getSessionId());
     if (status != null) {
       return new TSStatus(status);
     }
@@ -1229,7 +1169,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   public TSExecuteBatchStatementResp insertBatch(TSBatchInsertionReq req) {
     long t1 = System.currentTimeMillis();
     try {
-      if (!checkLogin()) {
+      if (!checkLogin(req.getSessionId())) {
         logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
         return getTSBatchExecuteStatementResp(getStatus(TSStatusCode.NOT_LOGIN_ERROR), null);
       }
@@ -1244,7 +1184,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       batchInsertPlan.setDataTypes(req.types);
 
       boolean isAllSuccessful = true;
-      TSStatus status = checkAuthority(batchInsertPlan);
+      TSStatus status = checkAuthority(batchInsertPlan, req.getSessionId());
       if (status != null) {
         return new TSExecuteBatchStatementResp(status);
       }
@@ -1276,14 +1216,14 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSStatus setStorageGroup(String storageGroup) {
-    if (!checkLogin()) {
+  public TSStatus setStorageGroup(long sessionId, String storageGroup) {
+    if (!checkLogin(sessionId)) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
       return getStatus(TSStatusCode.NOT_LOGIN_ERROR);
     }
 
     SetStorageGroupPlan plan = new SetStorageGroupPlan(new Path(storageGroup));
-    TSStatus status = checkAuthority(plan);
+    TSStatus status = checkAuthority(plan, sessionId);
     if (status != null) {
       return new TSStatus(status);
     }
@@ -1291,8 +1231,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSStatus deleteStorageGroups(List<String> storageGroups) {
-    if (!checkLogin()) {
+  public TSStatus deleteStorageGroups(long sessionId, List<String> storageGroups) {
+    if (!checkLogin(sessionId)) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
       return getStatus(TSStatusCode.NOT_LOGIN_ERROR);
     }
@@ -1301,7 +1241,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       storageGroupList.add(new Path(storageGroup));
     }
     DeleteStorageGroupPlan plan = new DeleteStorageGroupPlan(storageGroupList);
-    TSStatus status = checkAuthority(plan);
+    TSStatus status = checkAuthority(plan, sessionId);
     if (status != null) {
       return new TSStatus(status);
     }
@@ -1310,14 +1250,14 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
 
   @Override
   public TSStatus createTimeseries(TSCreateTimeseriesReq req) {
-    if (!checkLogin()) {
+    if (!checkLogin(req.getSessionId())) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
       return getStatus(TSStatusCode.NOT_LOGIN_ERROR);
     }
     CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan(new Path(req.getPath()),
         TSDataType.values()[req.getDataType()], TSEncoding.values()[req.getEncoding()],
         CompressionType.values()[req.compressor], new HashMap<>());
-    TSStatus status = checkAuthority(plan);
+    TSStatus status = checkAuthority(plan, req.getSessionId());
     if (status != null) {
       return new TSStatus(status);
     }
@@ -1325,8 +1265,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSStatus deleteTimeseries(List<String> paths) {
-    if (!checkLogin()) {
+  public TSStatus deleteTimeseries(long sessionId, List<String> paths) {
+    if (!checkLogin(sessionId)) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
       return getStatus(TSStatusCode.NOT_LOGIN_ERROR);
     }
@@ -1335,7 +1275,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       pathList.add(new Path(path));
     }
     DeleteTimeSeriesPlan plan = new DeleteTimeSeriesPlan(pathList);
-    TSStatus status = checkAuthority(plan);
+    TSStatus status = checkAuthority(plan, sessionId);
     if (status != null) {
       return new TSStatus(status);
     }
@@ -1343,16 +1283,16 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public long requestStatementId() {
-    long statementId = statementIdGenerator.get();
-    statementIdGenerator.set(statementId + 1);
+  public long requestStatementId(long sessionId) {
+    long statementId = statementIdGenerator.incrementAndGet();
+    sessionId2StatementId.computeIfAbsent(sessionId, s -> new HashSet<>()).add(statementId);
     return statementId;
   }
 
-  private TSStatus checkAuthority(PhysicalPlan plan) {
+  private TSStatus checkAuthority(PhysicalPlan plan, long sessionId) {
     List<Path> paths = plan.getPaths();
     try {
-      if (!checkAuthorization(paths, plan)) {
+      if (!checkAuthorization(paths, plan, usernameMap.get(sessionId))) {
         return getStatus(TSStatusCode.NO_PERMISSION_ERROR, plan.getOperatorType().toString());
       }
     } catch (AuthException e) {
@@ -1376,9 +1316,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   private long generateQueryId() {
-    long queryId = queryIdGenerator.get();
-    queryIdGenerator.set(queryId + 1);
-    return queryId;
+    return queryIdGenerator.incrementAndGet();
   }
 }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCloseIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCloseIT.java
index 08306fe..d700432 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCloseIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCloseIT.java
@@ -18,6 +18,13 @@
  */
 package org.apache.iotdb.db.integration;
 
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
@@ -29,10 +36,6 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.sql.*;
-
-import static org.junit.Assert.fail;
-
 /**
  * Notice that, all test begins with "IoTDB" is integration test. All test which will start the IoTDB server should be
  * defined as integration test.
@@ -136,7 +139,6 @@ public class IoTDBCloseIT {
       Statement statement1 = connection.createStatement();
       Statement statement2 = connection.createStatement();
       statement1.setFetchSize(10);
-      statement1.close();
       boolean hasResultSet1 = statement1.execute(selectSql);
       Assert.assertTrue(hasResultSet1);
       ResultSet resultSet1 = statement1.getResultSet();
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java
index 7e39ae4..9c35a36 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java
@@ -18,6 +18,13 @@
  */
 package org.apache.iotdb.db.integration;
 
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
@@ -29,10 +36,6 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.sql.*;
-
-import static org.junit.Assert.fail;
-
 /**
  * Notice that, all test begins with "IoTDB" is integration test. All test which will start the IoTDB server should be
  * defined as integration test.
@@ -135,7 +138,6 @@ public class IoTDBMultiStatementsIT {
          Statement statement1 = connection.createStatement();
          Statement statement2 = connection.createStatement()) {
       statement1.setFetchSize(10);
-      statement1.close();
       boolean hasResultSet1 = statement1.execute(selectSql);
       Assert.assertTrue(hasResultSet1);
       ResultSet resultSet1 = statement1.getResultSet();
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBPreparedStmtIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBPreparedStmtIT.java
index d79a7a5..bdd054b 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBPreparedStmtIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBPreparedStmtIT.java
@@ -85,8 +85,6 @@ public class IoTDBPreparedStmtIT {
 
   @Test
   public void testPreparedInsertion() throws SQLException {
-
-
     try (Connection connection = DriverManager
         .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
         IoTDBPreparedInsertionStatement statement =
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
index 17f32c6..abd237a 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
@@ -175,7 +175,8 @@ public class IoTDBSequenceDataQueryIT {
     queryExpression.addSelectedPath(new Path(Constant.d1s1));
     queryExpression.setExpression(null);
 
-    TEST_QUERY_JOB_ID = QueryResourceManager.getInstance().assignJobId();
+    TEST_QUERY_JOB_ID = 1;
+    QueryResourceManager.getInstance().registerQuery(TEST_QUERY_JOB_ID);
     TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
     QueryDataSet queryDataSet = engineExecutor.query(queryExpression, TEST_QUERY_CONTEXT);
 
@@ -200,7 +201,8 @@ public class IoTDBSequenceDataQueryIT {
 
     GlobalTimeExpression globalTimeExpression = new GlobalTimeExpression(TimeFilter.gtEq(800L));
     queryExpression.setExpression(globalTimeExpression);
-    TEST_QUERY_JOB_ID = QueryResourceManager.getInstance().assignJobId();
+    TEST_QUERY_JOB_ID = 1;
+    QueryResourceManager.getInstance().registerQuery(TEST_QUERY_JOB_ID);
     TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
     QueryDataSet queryDataSet = engineExecutor.query(queryExpression, TEST_QUERY_CONTEXT);
 
@@ -236,7 +238,8 @@ public class IoTDBSequenceDataQueryIT {
         ValueFilter.gtEq(14));
     queryExpression.setExpression(singleSeriesExpression);
 
-    TEST_QUERY_JOB_ID = QueryResourceManager.getInstance().assignJobId();
+    TEST_QUERY_JOB_ID = 1;
+    QueryResourceManager.getInstance().registerQuery(TEST_QUERY_JOB_ID);
     TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
     QueryDataSet queryDataSet = engineExecutor.query(queryExpression, TEST_QUERY_CONTEXT);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
index 3c2b305..35c89ac 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
@@ -246,7 +246,8 @@ public class IoTDBSeriesReaderIT {
     queryExpression.addSelectedPath(new Path(Constant.d1s1));
     queryExpression.setExpression(null);
 
-    TEST_QUERY_JOB_ID = QueryResourceManager.getInstance().assignJobId();
+    TEST_QUERY_JOB_ID = 1;
+    QueryResourceManager.getInstance().registerQuery(TEST_QUERY_JOB_ID);
     TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
     QueryDataSet queryDataSet = engineExecutor.query(queryExpression, TEST_QUERY_CONTEXT);
 
@@ -274,7 +275,8 @@ public class IoTDBSeriesReaderIT {
         ValueFilter.gtEq(20));
     queryExpression.setExpression(singleSeriesExpression);
 
-    TEST_QUERY_JOB_ID = QueryResourceManager.getInstance().assignJobId();
+    TEST_QUERY_JOB_ID = 1;
+    QueryResourceManager.getInstance().registerQuery(TEST_QUERY_JOB_ID);
     TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
     QueryDataSet queryDataSet = engineExecutor.query(queryExpression, TEST_QUERY_CONTEXT);
 
@@ -302,7 +304,8 @@ public class IoTDBSeriesReaderIT {
     SingleSeriesExpression expression = new SingleSeriesExpression(path, TimeFilter.gt(22987L));
     queryExpression.setExpression(expression);
 
-    TEST_QUERY_JOB_ID = QueryResourceManager.getInstance().assignJobId();
+    TEST_QUERY_JOB_ID = 1;
+    QueryResourceManager.getInstance().registerQuery(TEST_QUERY_JOB_ID);
     TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
     QueryDataSet queryDataSet = engineExecutor.query(queryExpression, TEST_QUERY_CONTEXT);
 
@@ -331,7 +334,8 @@ public class IoTDBSeriesReaderIT {
         ValueFilter.lt(111));
     queryExpression.setExpression(singleSeriesExpression);
 
-    TEST_QUERY_JOB_ID = QueryResourceManager.getInstance().assignJobId();
+    TEST_QUERY_JOB_ID = 1;
+    QueryResourceManager.getInstance().registerQuery(TEST_QUERY_JOB_ID);
     TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
     QueryDataSet queryDataSet = engineExecutor.query(queryExpression, TEST_QUERY_CONTEXT);
 
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java b/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
index b9634a2..c0d5c6a 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
@@ -25,10 +25,10 @@ import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.db.auth.authorizer.IAuthorizer;
 import org.apache.iotdb.db.auth.authorizer.LocalFileAuthorizer;
 import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.cache.DeviceMetaDataCache;
 import org.apache.iotdb.db.engine.cache.TsFileMetaDataCache;
@@ -59,7 +59,7 @@ public class EnvironmentUtils {
   private static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
   private static DirectoryManager directoryManager = DirectoryManager.getInstance();
 
-  public static long TEST_QUERY_JOB_ID = QueryResourceManager.getInstance().assignJobId();
+  public static long TEST_QUERY_JOB_ID = 1;
   public static QueryContext TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
 
   private static long oldTsFileThreshold = config.getTsFileSizeThreshold();
@@ -141,7 +141,7 @@ public class EnvironmentUtils {
   /**
    * disable memory control</br> this function should be called before all code in the setup
    */
-  public static void envSetUp() throws StartupException, IOException {
+  public static void envSetUp() throws StartupException {
     IoTDBDescriptor.getInstance().getConfig().setEnableParameterAdapter(false);
     MManager.getInstance().init();
     IoTDBConfigDynamicAdapter.getInstance().setInitialized(true);
@@ -164,7 +164,8 @@ public class EnvironmentUtils {
     MultiFileLogNodeManager.getInstance().start();
     FlushManager.getInstance().start();
     MergeManager.getINSTANCE().start();
-    TEST_QUERY_JOB_ID = QueryResourceManager.getInstance().assignJobId();
+    TEST_QUERY_JOB_ID ++;
+    QueryResourceManager.getInstance().registerQuery(TEST_QUERY_JOB_ID);
     TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
   }
 
@@ -183,6 +184,7 @@ public class EnvironmentUtils {
     createDir(config.getWalFolder());
     // create query
     createDir(config.getQueryDir());
+    createDir(TestConstant.OUTPUT_DATA_DIR);
     // create data
     for (String dataDir : config.getDataDirs()) {
       createDir(dataDir);
diff --git a/service-rpc/rpc-changelist.md b/service-rpc/rpc-changelist.md
index 53e38c1..296f983 100644
--- a/service-rpc/rpc-changelist.md
+++ b/service-rpc/rpc-changelist.md
@@ -99,11 +99,10 @@ Last Updated on November 12th, 2019 by Tian Jiang.
 
 | Latest Changes                                               | Related Committers                 |
 | ------------------------------------------------------------ | ---------------------------------- |
-|                            |                       |
+| Add parameter sessionId in getTimeZone, getProperties, setStorageGroup, createTimeseries... | Tian Jiang|
 
 ## 3. Update
 
 | Latest Changes                                               | Related Committers     |
 | ------------------------------------------------------------ | ---------------------- |
-| Replace TS_SessionHandles with SessionIds, TSOperationHandle with queryIds  | Tian 
-Jiang  |
\ No newline at end of file
+| Replace TS_SessionHandles with SessionIds, TSOperationHandle with queryIds  | Tian Jiang  |
\ No newline at end of file
diff --git a/service-rpc/src/main/thrift/rpc.thrift b/service-rpc/src/main/thrift/rpc.thrift
index 5c4fe5d..951d508 100644
--- a/service-rpc/src/main/thrift/rpc.thrift
+++ b/service-rpc/src/main/thrift/rpc.thrift
@@ -101,10 +101,9 @@ struct TSExecuteInsertRowInBatchResp{
 }
 
 struct TSExecuteBatchStatementResp{
-  1: required i64 sessionId
-	2: required TSStatus status
+	1: required TSStatus status
   // For each value in result, Statement.SUCCESS_NO_INFO represents success, Statement.EXECUTE_FAILED represents fail otherwise.
-	3: optional list<i32> result
+	2: optional list<i32> result
 }
 
 struct TSExecuteBatchStatementReq{
@@ -133,7 +132,8 @@ struct TSCancelOperationReq {
 // CloseOperation()
 struct TSCloseOperationReq {
   1: required i64 sessionId
-  2: required i64 queryId
+  2: optional i64 queryId
+  3: optional i64 statementId
 }
 
 struct TSFetchResultsReq{
@@ -187,7 +187,7 @@ struct TSInsertionReq {
     3: optional list<string> measurements
     4: optional list<string> values
     5: optional i64 timestamp
-    6: required i64 stmtId
+    6: optional i64 queryId
 }
 
 // for session
@@ -274,7 +274,7 @@ service TSIService {
 
 	TSStatus closeOperation(1:TSCloseOperationReq req);
 
-	TSGetTimeZoneResp getTimeZone();
+	TSGetTimeZoneResp getTimeZone(1:i64 sessionId);
 
 	TSStatus setTimeZone(1:TSSetTimeZoneReq req);
 
@@ -282,13 +282,13 @@ service TSIService {
 
 	TSExecuteStatementResp insert(1:TSInsertionReq req);
 
-	TSStatus setStorageGroup(1:string storageGroup);
+	TSStatus setStorageGroup(1:i64 sessionId, 2:string storageGroup);
 
 	TSStatus createTimeseries(1:TSCreateTimeseriesReq req);
 
-  TSStatus deleteTimeseries(1:list<string> path)
+  TSStatus deleteTimeseries(1:i64 sessionId, 2:list<string> path)
 
-  TSStatus deleteStorageGroups(1:list<string> storageGroup);
+  TSStatus deleteStorageGroups(1:i64 sessionId, 2:list<string> storageGroup);
 
   TSExecuteBatchStatementResp insertBatch(1:TSBatchInsertionReq req);
 
@@ -304,5 +304,5 @@ service TSIService {
 
 	TSStatus deleteData(1:TSDeleteDataReq req);
 
-	i64 requestStatementId();
+	i64 requestStatementId(1:i64 sessionId);
 }