You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2020/09/04 07:27:22 UTC

[GitHub] [incubator-iotdb] LebronAl opened a new pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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


   


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

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



[GitHub] [incubator-iotdb] LebronAl removed a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl removed a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-689985530


   # Benchmark Parameter
   CLIENT_NUMBER=1
   GROUP_NUMBER=1
   DEVICE_NUMBER=20
   SENSOR_NUMBER=20
   BATCH_SIZE=50
   LOOP=1000
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   # Cluster config
   default_replica_num=1
   enablePartition=false
   is_enable_raft_log_persistence=true
   is_use_async_applier=true
   two nodes's cluster(34,35),benchmark local
   # Cluster_new Version(df8e0891cb75e0c6ff61ce10aedbaa8b99b66a31 add documents)
   ## forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92681984-bdce1b80-f361-11ea-96d6-c683b7a738a6.png)
   ## non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92682072-fcfc6c80-f361-11ea-8905-06ceae1a8548.png)
   # Cluster_new_cache_leader Version(229779f adjust close cache leader logic)
   ## Close cache leader
   ###  forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683902-bfe6a900-f366-11ea-9dc0-99bda47dce0d.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683361-47331d00-f365-11ea-918b-e18cc3cc03f5.png)
   ## Open cache leader
   ### little forward(benchmark writes data to 34, sg belongs to 35,internal forward first write for each deviceId + cache leader)
   ![image](https://user-images.githubusercontent.com/32640567/92683546-bf99de00-f365-11ea-9515-be3466697bff.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683752-48187e80-f366-11ea-8b1c-8e2b714dbba4.png)
   


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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       Actually I designed this on purpose. If the first call to the `forwardPlan` function returns StatusUtils.NO_LEADER. We should block this request until the new leader is elected. So the following code should be called. This is why I put this return statement in the scope of `if`. Of course,  If the second call to the `forwardPlan` function returns StatusUtils.NO_LEADER too, then maybe just return will be better~




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

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



[GitHub] [incubator-iotdb] LebronAl commented on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-693760479


   # Benchmark Parameter
   IS_CLIENT_BIND=False
   CLIENT_NUMBER=20
   GROUP_NUMBER=20
   DEVICE_NUMBER=200
   SENSOR_NUMBER=500
   BATCH_SIZE=50
   LOOP=200
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   
   # Cluster config
   default_replica_num=2
   enablePartition=false
   is_enable_raft_log_persistence=false
   is_use_async_applier=true
   three nodes's cluster(33,34,35),benchmark(32)
   
   # Cluster_new
   ## Version
   ef2c8a2 fix method name
   ## Result
   ![image](https://user-images.githubusercontent.com/32640567/93409838-5ddbf580-f8ca-11ea-9501-f533102cc296.png)
   ![image](https://user-images.githubusercontent.com/32640567/93409796-443aae00-f8ca-11ea-9376-0ae6af6bef61.png)
   ![image](https://user-images.githubusercontent.com/32640567/93409814-51579d00-f8ca-11ea-8811-5b1d733f1d73.png)
   ![image](https://user-images.githubusercontent.com/32640567/93409822-574d7e00-f8ca-11ea-9422-0003742e2d86.png)
   
   # Cluster_new_cache_leader
   ## Version
   b2e92ed Merge branch 'cluster_new' of github.com:Apache/incubator-iotdb into cluster_new_cache_leader
   ## Result
   ![image](https://user-images.githubusercontent.com/32640567/93410603-d98a7200-f8cb-11ea-96c5-88fb8b622f45.png)
   ![image](https://user-images.githubusercontent.com/32640567/93410549-bf509400-f8cb-11ea-9f7d-67898362c7bd.png)
   ![image](https://user-images.githubusercontent.com/32640567/93410570-c8d9fc00-f8cb-11ea-98c7-3ef46e780375.png)
   ![image](https://user-images.githubusercontent.com/32640567/93410590-d099a080-f8cb-11ea-92eb-2961d584d610.png)


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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~Anything can be discussed here~




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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I copy this from old session's implemention.I will delete it anyway.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       OK, I intend to update `endPointToSessionConnection ` first and then update `metaSessionConnection` from `endPointToSessionConnection `

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       Actually I designed this on purpose. If the first call to the `forwardPlan` function returns StatusUtils.NO_LEADER. We should block this request until the new leader is elected. So the following code should be called. This is why I put this return statement in the scope of `if`. Of course,  If the second call to the `forwardPlan` function returns StatusUtils.NO_LEADER too, then maybe just return will be better~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~ Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       Actually I designed this on purpose.If the last element of the tmpStatus do not set the RedirectNode, which means the coordinator can process at lease one plan locally, so the final allRedirect value is false, the server will not recomment client a new Node to redirect.
   BTW, maybe in our user's most scenarios, forwarding the plan to second Node is better, I will fix this.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       copy that!

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I copy this from old session's implemention.I will delete it anyway.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       OK, I intend to update `endPointToSessionConnection ` first and then update `metaSessionConnection` from `endPointToSessionConnection `

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       Actually I designed this on purpose. If the first call to the `forwardPlan` function returns StatusUtils.NO_LEADER. We should block this request until the new leader is elected. So the following code should be called. This is why I put this return statement in the scope of `if`. Of course,  If the second call to the `forwardPlan` function returns StatusUtils.NO_LEADER too, then maybe just return will be better~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~ Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       Actually I designed this on purpose.If the last element of the tmpStatus do not set the RedirectNode, which means the coordinator can process at lease one plan locally, so the final allRedirect value is false, the server will not recomment client a new Node to redirect.
   BTW, maybe in our user's most scenarios, forwarding the plan to second Node is better, I will fix this.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       copy that!

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I copy this from old session's implemention.I will delete it anyway.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       OK, I intend to update `endPointToSessionConnection ` first and then update `metaSessionConnection` from `endPointToSessionConnection `

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       Actually I designed this on purpose. If the first call to the `forwardPlan` function returns StatusUtils.NO_LEADER. We should block this request until the new leader is elected. So the following code should be called. This is why I put this return statement in the scope of `if`. Of course,  If the second call to the `forwardPlan` function returns StatusUtils.NO_LEADER too, then maybe just return will be better~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~ Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       Actually I designed this on purpose.If the last element of the tmpStatus do not set the RedirectNode, which means the coordinator can process at lease one plan locally, so the final allRedirect value is false, the server will not recomment client a new Node to redirect.
   BTW, maybe in our user's most scenarios, forwarding the plan to second Node is better, I will fix this.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       copy that!




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

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



[GitHub] [incubator-iotdb] LebronAl edited a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-689985530


   # Benchmark Parameter
   CLIENT_NUMBER=1
   GROUP_NUMBER=1
   DEVICE_NUMBER=20
   SENSOR_NUMBER=20
   BATCH_SIZE=50
   LOOP=1000
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   # Cluster config
   default_replica_num=1
   enablePartition=false
   two nodes's cluster(34,35)
   # Cluster_new's latest Version
   ## forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92681984-bdce1b80-f361-11ea-96d6-c683b7a738a6.png)
   ## non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92682072-fcfc6c80-f361-11ea-8905-06ceae1a8548.png)
   # Cluster_new_cache_leader's latest Version
   ## Close cache leader
   ###  forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683902-bfe6a900-f366-11ea-9dc0-99bda47dce0d.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683361-47331d00-f365-11ea-918b-e18cc3cc03f5.png)
   ## Open cache leader
   ### little forward(benchmark writes data to 34, sg belongs to 35,internal forward first write for each deviceId + cache leader)
   ![image](https://user-images.githubusercontent.com/32640567/92683546-bf99de00-f365-11ea-9515-be3466697bff.png)
   ### non-forward(benchmark writes data to 34, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683752-48187e80-f366-11ea-8b1c-8e2b714dbba4.png)
   


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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       ok~




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

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



[GitHub] [incubator-iotdb] LebronAl commented on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-689985530


   # Benchmark Parameter
   CLIENT_NUMBER=1
   GROUP_NUMBER=1
   DEVICE_NUMBER=20
   SENSOR_NUMBER=20
   BATCH_SIZE=50
   LOOP=1000
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   # Cluster config
   default_replica_num=1
   enablePartition=false
   two nodes's cluster(34,35)
   # Cluster_new's latest Version
   ## forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92681984-bdce1b80-f361-11ea-96d6-c683b7a738a6.png)
   ## non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92682072-fcfc6c80-f361-11ea-8905-06ceae1a8548.png)
   # Cluster_new_cache_leader's latest Version
   ## Close cache leader
   ###  forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683902-bfe6a900-f366-11ea-9dc0-99bda47dce0d.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683361-47331d00-f365-11ea-918b-e18cc3cc03f5.png)
   ## Open cache leader
   ### little forward(benchmark writes data to 34, sg belongs to 35,internal forward first write for each deviceId + cache leader)
   ![image](https://user-images.githubusercontent.com/32640567/92683546-bf99de00-f365-11ea-9515-be3466697bff.png)
   ### non-fowward(benchmark writes data to 34, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683752-48187e80-f366-11ea-8b1c-8e2b714dbba4.png)
   


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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I copy this from old session's implemention.I will delete it anyway.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       OK, I intend to update `endPointToSessionConnection ` first and then update `metaSessionConnection` from `endPointToSessionConnection `

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       Actually I designed this on purpose. If the first call to the `forwardPlan` function returns StatusUtils.NO_LEADER. We should block this request until the new leader is elected. So the following code should be called. This is why I put this return statement in the scope of `if`. Of course,  If the second call to the `forwardPlan` function returns StatusUtils.NO_LEADER too, then maybe just return will be better~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~ Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       Actually I designed this on purpose.If the last element of the tmpStatus do not set the RedirectNode, which means the coordinator can process at lease one plan locally, so the final allRedirect value is false, the server will not recomment client a new Node to redirect.
   BTW, maybe in our user's most scenarios, forwarding the plan to second Node is better, I will fix this.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       copy that!




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

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



[GitHub] [incubator-iotdb] jt2594838 merged pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
jt2594838 merged pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691


   


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

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



[GitHub] [incubator-iotdb] neuyilan commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above, outside of the if statement

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       One more thing:
   if the last element of the _tmpStatus_ not set the RedirectNode, the final allRedirect value is false,
   This situation should be considered. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       the _return result;_ statements should be outside of the _if (!StatusUtils.NO_LEADER.equals(result)) {_ statements? 
   Otherwise, the plan will be executed twice.
   
   ```suggestion
         if (!StatusUtils.NO_LEADER.equals(result)) {
           result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
         }
         return result; 
   ```
   
   

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       I got it, better add more comments here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above, outside of the if statement

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       One more thing:
   if the last element of the _tmpStatus_ not set the RedirectNode, the final allRedirect value is false,
   This situation should be considered. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       the _return result;_ statements should be outside of the _if (!StatusUtils.NO_LEADER.equals(result)) {_ statements? 
   Otherwise, the plan will be executed twice.
   
   ```suggestion
         if (!StatusUtils.NO_LEADER.equals(result)) {
           result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
         }
         return result; 
   ```
   
   

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       I got it, better add more comments here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above, outside of the if statement

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       One more thing:
   if the last element of the _tmpStatus_ not set the RedirectNode, the final allRedirect value is false,
   This situation should be considered. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       the _return result;_ statements should be outside of the _if (!StatusUtils.NO_LEADER.equals(result)) {_ statements? 
   Otherwise, the plan will be executed twice.
   
   ```suggestion
         if (!StatusUtils.NO_LEADER.equals(result)) {
           result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
         }
         return result; 
   ```
   
   

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       I got it, better add more comments here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above, outside of the if statement

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       One more thing:
   if the last element of the _tmpStatus_ not set the RedirectNode, the final allRedirect value is false,
   This situation should be considered. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       the _return result;_ statements should be outside of the _if (!StatusUtils.NO_LEADER.equals(result)) {_ statements? 
   Otherwise, the plan will be executed twice.
   
   ```suggestion
         if (!StatusUtils.NO_LEADER.equals(result)) {
           result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
         }
         return result; 
   ```
   
   

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       I got it, better add more comments here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above, outside of the if statement

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       One more thing:
   if the last element of the _tmpStatus_ not set the RedirectNode, the final allRedirect value is false,
   This situation should be considered. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       the _return result;_ statements should be outside of the _if (!StatusUtils.NO_LEADER.equals(result)) {_ statements? 
   Otherwise, the plan will be executed twice.
   
   ```suggestion
         if (!StatusUtils.NO_LEADER.equals(result)) {
           result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
         }
         return result; 
   ```
   
   

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       I got it, better add more comments here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above, outside of the if statement

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       One more thing:
   if the last element of the _tmpStatus_ not set the RedirectNode, the final allRedirect value is false,
   This situation should be considered. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       the _return result;_ statements should be outside of the _if (!StatusUtils.NO_LEADER.equals(result)) {_ statements? 
   Otherwise, the plan will be executed twice.
   
   ```suggestion
         if (!StatusUtils.NO_LEADER.equals(result)) {
           result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
         }
         return result; 
   ```
   
   

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       I got it, better add more comments here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above, outside of the if statement

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       One more thing:
   if the last element of the _tmpStatus_ not set the RedirectNode, the final allRedirect value is false,
   This situation should be considered. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       the _return result;_ statements should be outside of the _if (!StatusUtils.NO_LEADER.equals(result)) {_ statements? 
   Otherwise, the plan will be executed twice.
   
   ```suggestion
         if (!StatusUtils.NO_LEADER.equals(result)) {
           result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
         }
         return result; 
   ```
   
   

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       I got it, better add more comments here~




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

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



[GitHub] [incubator-iotdb] LebronAl edited a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-689985530


   # Benchmark Parameter
   CLIENT_NUMBER=1
   GROUP_NUMBER=1
   DEVICE_NUMBER=20
   SENSOR_NUMBER=20
   BATCH_SIZE=50
   LOOP=1000
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   # Cluster config
   default_replica_num=1
   enablePartition=false
   is_enable_raft_log_persistence=true
   is_use_async_applier=true
   two nodes's cluster(34,35),benchmark local
   # Cluster_new Version(df8e0891cb75e0c6ff61ce10aedbaa8b99b66a31 add documents)
   ## forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92681984-bdce1b80-f361-11ea-96d6-c683b7a738a6.png)
   ## non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92682072-fcfc6c80-f361-11ea-8905-06ceae1a8548.png)
   # Cluster_new_cache_leader Version(229779f adjust close cache leader logic)
   ## Close cache leader
   ###  forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683902-bfe6a900-f366-11ea-9dc0-99bda47dce0d.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683361-47331d00-f365-11ea-918b-e18cc3cc03f5.png)
   ## Open cache leader
   ### little forward(benchmark writes data to 34, sg belongs to 35,internal forward first write for each deviceId + cache leader)
   ![image](https://user-images.githubusercontent.com/32640567/92683546-bf99de00-f365-11ea-9515-be3466697bff.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683752-48187e80-f366-11ea-8b1c-8e2b714dbba4.png)
   


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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       Actually I designed this on purpose.If the last element of the tmpStatus do not set the RedirectNode, which means the coordinator can process at lease one plan locally, so the final allRedirect value is false, the server will not recomment client a new Node to redirect.
   BTW, maybe in our user's most scenarios, forwarding the plan to second Node is better, I will fix this.




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

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



[GitHub] [incubator-iotdb] neuyilan commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       I got it, better add more comments here~




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

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



[GitHub] [incubator-iotdb] neuyilan commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?




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

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



[GitHub] [incubator-iotdb] LebronAl edited a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-689985530


   # Benchmark Parameter
   CLIENT_NUMBER=1
   GROUP_NUMBER=1
   DEVICE_NUMBER=20
   SENSOR_NUMBER=20
   BATCH_SIZE=50
   LOOP=1000
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   # Cluster config
   default_replica_num=1
   enablePartition=false
   two nodes's cluster(34,35)
   # Cluster_new's latest Version(df8e0891cb75e0c6ff61ce10aedbaa8b99b66a31 add documents)
   ## forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92681984-bdce1b80-f361-11ea-96d6-c683b7a738a6.png)
   ## non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92682072-fcfc6c80-f361-11ea-8905-06ceae1a8548.png)
   # Cluster_new_cache_leader's latest Version
   ## Close cache leader
   ###  forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683902-bfe6a900-f366-11ea-9dc0-99bda47dce0d.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683361-47331d00-f365-11ea-918b-e18cc3cc03f5.png)
   ## Open cache leader
   ### little forward(benchmark writes data to 34, sg belongs to 35,internal forward first write for each deviceId + cache leader)
   ![image](https://user-images.githubusercontent.com/32640567/92683546-bf99de00-f365-11ea-9515-be3466697bff.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683752-48187e80-f366-11ea-8b1c-8e2b714dbba4.png)
   


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

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



[GitHub] [incubator-iotdb] LebronAl commented on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-692695905


   # Benchmark Parameter
   CLIENT_NUMBER=20
   GROUP_NUMBER=20
   DEVICE_NUMBER=200
   SENSOR_NUMBER=500
   BATCH_SIZE=50
   LOOP=200
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   
   # Cluster config
   default_replica_num=2
   enablePartition=false
   is_enable_raft_log_persistence=false
   is_use_async_applier=true
   three nodes's cluster(33,34,35),benchmark(32)
   
   # Cluster_new
   ## Version
   ef2c8a2a4  fix method name
   ## Result
   ![image](https://user-images.githubusercontent.com/32640567/93212925-d3927500-f795-11ea-9fb3-8f1704873997.png)
   ![image](https://user-images.githubusercontent.com/32640567/93212874-bfe70e80-f795-11ea-9459-e07c54a5c4f4.png)
   ![image](https://user-images.githubusercontent.com/32640567/93212885-c6758600-f795-11ea-96f5-1bed48c3e958.png)
   ![image](https://user-images.githubusercontent.com/32640567/93212910-cd9c9400-f795-11ea-9410-630b27f8fc6f.png)
   # Cluster_new_cache_leader
   ## Version
   b2e92ed68 Merge branch 'cluster_new' of github.com:Apache/incubator-iotdb into cluster_new_cache_leader
   ## Result
   ![image](https://user-images.githubusercontent.com/32640567/93211566-0176ba00-f794-11ea-9ad2-485e12fbeb0f.png)
   ![image](https://user-images.githubusercontent.com/32640567/93211466-ddb37400-f793-11ea-8f76-6c8d71a02cb8.png)
   ![image](https://user-images.githubusercontent.com/32640567/93211502-e99f3600-f793-11ea-94ae-4be66e899f00.png)
   ![image](https://user-images.githubusercontent.com/32640567/93211543-f6bc2500-f793-11ea-96a2-3f42362530e4.png)


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

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



[GitHub] [incubator-iotdb] LebronAl edited a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-689985530


   # Benchmark Parameter
   CLIENT_NUMBER=1
   GROUP_NUMBER=1
   DEVICE_NUMBER=20
   SENSOR_NUMBER=20
   BATCH_SIZE=50
   LOOP=1000
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   # Cluster config
   default_replica_num=1
   enablePartition=false
   two nodes's cluster(34,35)
   # Cluster_new's latest Version
   ## forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92681984-bdce1b80-f361-11ea-96d6-c683b7a738a6.png)
   ## non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92682072-fcfc6c80-f361-11ea-8905-06ceae1a8548.png)
   # Cluster_new_cache_leader's latest Version
   ## Close cache leader
   ###  forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683902-bfe6a900-f366-11ea-9dc0-99bda47dce0d.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683361-47331d00-f365-11ea-918b-e18cc3cc03f5.png)
   ## Open cache leader
   ### little forward(benchmark writes data to 34, sg belongs to 35,internal forward first write for each deviceId + cache leader)
   ![image](https://user-images.githubusercontent.com/32640567/92683546-bf99de00-f365-11ea-9515-be3466697bff.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683752-48187e80-f366-11ea-8b1c-8e2b714dbba4.png)
   


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

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



[GitHub] [incubator-iotdb] neuyilan commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above, outside of the if statement

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       One more thing:
   if the last element of the _tmpStatus_ not set the RedirectNode, the final allRedirect value is false,
   This situation should be considered. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       the _return result;_ statements should be outside of the _if (!StatusUtils.NO_LEADER.equals(result)) {_ statements? 
   Otherwise, the plan will be executed twice.
   
   ```suggestion
         if (!StatusUtils.NO_LEADER.equals(result)) {
           result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
         }
         return result; 
   ```
   
   




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

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



[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       I think the redirect strategy can be improved a little.
   Mostly the plan is forwarded to multiple groups because its data has crossed multiple time partitions, and generally, this only happens on time partition borders as a plan contains a relatively short time range compared to the time partition length.
   So typically `planGroupMap.size() <= 2`,  and when it is 2, the following plans most probably belong to the second group. Consequently, it would reasonable if we always set the redirect target to that of the sub-plan with the largest timestamp.
   
   For example, if the partition length is a day, and a plan that contains data from Thuesday 11:50 to Wednesday 00:10 will be split into two sub-plans, each contains data from Thuesday 11:50 to Thuesday 11:59 and  Wednesday 00:00 to  Wednesday 00:10 respectively, and the two sub-plans are forwarded to NodeA and NodeB.
   It is clear that the following plans, which contain data of Wednesday, should be forwarded to NodeB.
   
   By the way, I think the use of a map to split plan is unnecessary, and a `List<Pair<>>` should be just enough, and you may sort it according to the largest timestamp easily.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       `Collections.singletonList()` should be enough.

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I think this condition is redundant.

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       Close operation should be put into a finally block.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       Assuming current metaConnection is to NodeA, but the actual leader is NodeB:
   1. setStorageGroup("root.sg1") is executed, and a redirection to NodeB is returned
   2. leader switched to NodeA
   3. setStorageGroup("root.sg2") is executed, and a redirection to NodeA is returned
   4. `metaSessionConnection = new SessionConnection(this, e.getEndPoint())` is executed, but as there is already one connection to NodeA in `endPointToSessionConnection`, the new connection will be lost when the next redirection comes.
   
   So, maybe you should check `endPointToSessionConnection` before newing a connection.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       Collections.singletonList should be enough.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       I think `new ArrayList<>` is unnecessary.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       I think the redirect strategy can be improved a little.
   Mostly the plan is forwarded to multiple groups because its data has crossed multiple time partitions, and generally, this only happens on time partition borders as a plan contains a relatively short time range compared to the time partition length.
   So typically `planGroupMap.size() <= 2`,  and when it is 2, the following plans most probably belong to the second group. Consequently, it would reasonable if we always set the redirect target to that of the sub-plan with the largest timestamp.
   
   For example, if the partition length is a day, and a plan that contains data from Thuesday 11:50 to Wednesday 00:10 will be split into two sub-plans, each contains data from Thuesday 11:50 to Thuesday 11:59 and  Wednesday 00:00 to  Wednesday 00:10 respectively, and the two sub-plans are forwarded to NodeA and NodeB.
   It is clear that the following plans, which contain data of Wednesday, should be forwarded to NodeB.
   
   By the way, I think the use of a map to split plan is unnecessary, and a `List<Pair<>>` should be just enough, and you may sort it according to the largest timestamp easily.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       `Collections.singletonList()` should be enough.

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I think this condition is redundant.

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       Close operation should be put into a finally block.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       Assuming current metaConnection is to NodeA, but the actual leader is NodeB:
   1. setStorageGroup("root.sg1") is executed, and a redirection to NodeB is returned
   2. leader switched to NodeA
   3. setStorageGroup("root.sg2") is executed, and a redirection to NodeA is returned
   4. `metaSessionConnection = new SessionConnection(this, e.getEndPoint())` is executed, but as there is already one connection to NodeA in `endPointToSessionConnection`, the new connection will be lost when the next redirection comes.
   
   So, maybe you should check `endPointToSessionConnection` before newing a connection.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       Collections.singletonList should be enough.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       I think `new ArrayList<>` is unnecessary.




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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       OK~




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

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



[GitHub] [incubator-iotdb] LebronAl edited a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-692695905


   # Benchmark Parameter
   IS_CLIENT_BIND=True
   CLIENT_NUMBER=20
   GROUP_NUMBER=20
   DEVICE_NUMBER=200
   SENSOR_NUMBER=500
   BATCH_SIZE=50
   LOOP=200
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   
   # Cluster config
   default_replica_num=2
   enablePartition=false
   is_enable_raft_log_persistence=false
   is_use_async_applier=true
   three nodes's cluster(33,34,35),benchmark(32)
   
   # Cluster_new
   ## Version
   ef2c8a2a4  fix method name
   ## Result
   ![image](https://user-images.githubusercontent.com/32640567/93212925-d3927500-f795-11ea-9fb3-8f1704873997.png)
   ![image](https://user-images.githubusercontent.com/32640567/93212874-bfe70e80-f795-11ea-9459-e07c54a5c4f4.png)
   ![image](https://user-images.githubusercontent.com/32640567/93212885-c6758600-f795-11ea-96f5-1bed48c3e958.png)
   ![image](https://user-images.githubusercontent.com/32640567/93212910-cd9c9400-f795-11ea-9410-630b27f8fc6f.png)
   # Cluster_new_cache_leader
   ## Version
   b2e92ed68 Merge branch 'cluster_new' of github.com:Apache/incubator-iotdb into cluster_new_cache_leader
   ## Result
   ![image](https://user-images.githubusercontent.com/32640567/93211566-0176ba00-f794-11ea-9ad2-485e12fbeb0f.png)
   ![image](https://user-images.githubusercontent.com/32640567/93211466-ddb37400-f793-11ea-8f76-6c8d71a02cb8.png)
   ![image](https://user-images.githubusercontent.com/32640567/93211502-e99f3600-f793-11ea-94ae-4be66e899f00.png)
   ![image](https://user-images.githubusercontent.com/32640567/93211543-f6bc2500-f793-11ea-96a2-3f42362530e4.png)


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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       OK~




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

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



[GitHub] [incubator-iotdb] LebronAl commented on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-692446092


   # Benchmark Parameter
   CLIENT_NUMBER=1
   GROUP_NUMBER=1
   DEVICE_NUMBER=20
   SENSOR_NUMBER=20
   BATCH_SIZE=50
   LOOP=10000
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   # Cluster config
   default_replica_num=1
   enablePartition=false
   is_enable_raft_log_persistence=true
   is_use_async_applier=true
   two nodes's cluster(34,35),benchmark(33)
   # Cluster_new 
   ## Version
   b42b0f194 massive refactor: RaftMembers now care about less
   ## forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/93157757-65ba5f00-f73d-11ea-88d4-4911de478109.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157730-59360680-f73d-11ea-99db-6f6195d6a28a.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157742-5e935100-f73d-11ea-9241-e6e2a9d78e41.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157716-53402580-f73d-11ea-8dbd-90a1088bed8e.png)
   ## non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/93157886-afa34500-f73d-11ea-9e07-b9bb1b32446b.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157853-9bf7de80-f73d-11ea-9537-d863abf5b788.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157861-a1552900-f73d-11ea-9205-60a642546de7.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157871-a7e3a080-f73d-11ea-9180-87a6981e3ba0.png)
   # Cluster_new_cache_leader 
   ## Version
   229779f adjust close cache leader logic
   ## Close cache leader
   ###  forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/93161978-4ffd6780-f746-11ea-80d0-8476f5144510.png)
   ![image](https://user-images.githubusercontent.com/32640567/93161944-3e1bc480-f746-11ea-862e-3210515f4de7.png)
   ![image](https://user-images.githubusercontent.com/32640567/93161951-42e07880-f746-11ea-8102-300ee589b338.png)
   ![image](https://user-images.githubusercontent.com/32640567/93161966-4aa01d00-f746-11ea-905a-c2d5940a4ad8.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/93162617-b5058d00-f747-11ea-87ff-5603e2e3874b.png)
   ![image](https://user-images.githubusercontent.com/32640567/93162562-97d0be80-f747-11ea-95a6-b68da54e2a2e.png)
   ![image](https://user-images.githubusercontent.com/32640567/93162571-9bfcdc00-f747-11ea-91a0-c3b1f4e667fe.png)
   ![image](https://user-images.githubusercontent.com/32640567/93162588-a5864400-f747-11ea-9084-dc87c4a3e163.png)
   ## Open cache leader
   ### little forward(benchmark writes data to 34, sg belongs to 35,internal forward first write for each deviceId + cache leader)
   ![image](https://user-images.githubusercontent.com/32640567/93163061-ba170c00-f748-11ea-8d59-a20792ea72a1.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163015-a4a1e200-f748-11ea-9459-c6fdae223e37.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163020-a8cdff80-f748-11ea-89bc-874a46f47442.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163028-ae2b4a00-f748-11ea-82c6-92dcb64ff2f0.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/93163202-167a2b80-f749-11ea-86ff-672882f2a2cc.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163173-04988880-f749-11ea-949d-637c7d4a5066.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163178-0a8e6980-f749-11ea-8cfe-2bd6ed59a6a5.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163187-0febb400-f749-11ea-8ae7-f94bb843e138.png)


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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~ Anything can be discussed here~




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

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



[GitHub] [incubator-iotdb] LebronAl edited a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-689985530


   # Benchmark Parameter
   CLIENT_NUMBER=1
   GROUP_NUMBER=1
   DEVICE_NUMBER=20
   SENSOR_NUMBER=20
   BATCH_SIZE=50
   LOOP=1000
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   # Cluster config
   default_replica_num=1
   enablePartition=false
   is_enable_raft_log_persistence=true
   is_use_async_applier=true
   two nodes's cluster(34,35)
   # Cluster_new's latest Version(df8e0891cb75e0c6ff61ce10aedbaa8b99b66a31 add documents)
   ## forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92681984-bdce1b80-f361-11ea-96d6-c683b7a738a6.png)
   ## non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92682072-fcfc6c80-f361-11ea-8905-06ceae1a8548.png)
   # Cluster_new_cache_leader's latest Version
   ## Close cache leader
   ###  forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683902-bfe6a900-f366-11ea-9dc0-99bda47dce0d.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683361-47331d00-f365-11ea-918b-e18cc3cc03f5.png)
   ## Open cache leader
   ### little forward(benchmark writes data to 34, sg belongs to 35,internal forward first write for each deviceId + cache leader)
   ![image](https://user-images.githubusercontent.com/32640567/92683546-bf99de00-f365-11ea-9515-be3466697bff.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/92683752-48187e80-f366-11ea-8b1c-8e2b714dbba4.png)
   


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

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



[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       I think the redirect strategy can be improved a little.
   Mostly the plan is forwarded to multiple groups because its data has crossed multiple time partitions, and generally, this only happens on time partition borders as a plan contains a relatively short time range compared to the time partition length.
   So typically `planGroupMap.size() <= 2`,  and when it is 2, the following plans most probably belong to the second group. Consequently, it would reasonable if we always set the redirect target to that of the sub-plan with the largest timestamp.
   
   For example, if the partition length is a day, and a plan that contains data from Thuesday 11:50 to Wednesday 00:10 will be split into two sub-plans, each contains data from Thuesday 11:50 to Thuesday 11:59 and  Wednesday 00:00 to  Wednesday 00:10 respectively, and the two sub-plans are forwarded to NodeA and NodeB.
   It is clear that the following plans, which contain data of Wednesday, should be forwarded to NodeB.
   
   By the way, I think the use of a map to split plan is unnecessary, and a `List<Pair<>>` should be just enough, and you may sort it according to the largest timestamp easily.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       `Collections.singletonList()` should be enough.

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I think this condition is redundant.

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       Close operation should be put into a finally block.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       Assuming current metaConnection is to NodeA, but the actual leader is NodeB:
   1. setStorageGroup("root.sg1") is executed, and a redirection to NodeB is returned
   2. leader switched to NodeA
   3. setStorageGroup("root.sg2") is executed, and a redirection to NodeA is returned
   4. `metaSessionConnection = new SessionConnection(this, e.getEndPoint())` is executed, but as there is already one connection to NodeA in `endPointToSessionConnection`, the new connection will be lost when the next redirection comes.
   
   So, maybe you should check `endPointToSessionConnection` before newing a connection.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       Collections.singletonList should be enough.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       I think `new ArrayList<>` is unnecessary.




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

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



[GitHub] [incubator-iotdb] LebronAl commented on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-694602676


   # Benchmark Parameter
   IS_CLIENT_BIND=False
   CLIENT_NUMBER=20
   GROUP_NUMBER=20
   DEVICE_NUMBER=200
   SENSOR_NUMBER=500
   BATCH_SIZE=50
   LOOP=200
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   
   # Cluster config
   default_replica_num=2
   enablePartition=false
   is_enable_raft_log_persistence=false
   is_use_async_applier=true
   three nodes's cluster(33,34,35),benchmark(32) write to 33
   
   # Cluster_new
   ## Version
   4c3395add refactor timer
   ## Result
   ![image](https://user-images.githubusercontent.com/32640567/93545747-ccd15100-f993-11ea-9719-9c2edabb8b5a.png)
   ![image](https://user-images.githubusercontent.com/32640567/93545757-d35fc880-f993-11ea-84aa-c3940e603ff2.png)
   ![image](https://user-images.githubusercontent.com/32640567/93545768-d8bd1300-f993-11ea-8a24-45f889906e28.png)
   ![image](https://user-images.githubusercontent.com/32640567/93545788-e4a8d500-f993-11ea-950e-7ec86474ca64.png)
   ![image](https://user-images.githubusercontent.com/32640567/93545853-0dc96580-f994-11ea-8e51-3f25c7e5b301.png)
   ![image](https://user-images.githubusercontent.com/32640567/93545859-11f58300-f994-11ea-9153-bd1faba7ec1b.png)
   # Cluster_new_cache_leader
   ## Version
   7d12d5ba2 Merge branch 'cluster_new' of github.com:Apache/incubator-iotdb into cluster_new_cache_leader
   ## Result
   ![image](https://user-images.githubusercontent.com/32640567/93546308-1d957980-f995-11ea-90ee-22bb2277b22d.png)
   ![image](https://user-images.githubusercontent.com/32640567/93546246-09517c80-f995-11ea-98c6-7fc9461f26f1.png)
   ![image](https://user-images.githubusercontent.com/32640567/93546264-0f475d80-f995-11ea-9d99-984233b1e229.png)
   ![image](https://user-images.githubusercontent.com/32640567/93546282-14a4a800-f995-11ea-9e3a-71c2a6bae7ec.png)
   ![image](https://user-images.githubusercontent.com/32640567/93546342-2be39580-f995-11ea-92e3-1d06fc83a51e.png)
   ![image](https://user-images.githubusercontent.com/32640567/93546373-37cf5780-f995-11ea-9d35-aaae632dbefc.png)
   


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

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



[GitHub] [incubator-iotdb] LebronAl edited a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-692446092


   # Benchmark Parameter
   CLIENT_NUMBER=1
   GROUP_NUMBER=1
   DEVICE_NUMBER=20
   SENSOR_NUMBER=20
   BATCH_SIZE=50
   LOOP=10000
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   # Cluster config
   default_replica_num=1
   enablePartition=false
   is_enable_raft_log_persistence=false
   is_use_async_applier=true
   two nodes's cluster(34,35),benchmark(33)
   # Cluster_new 
   ## Version
   b42b0f194 massive refactor: RaftMembers now care about less
   ## forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/93157757-65ba5f00-f73d-11ea-88d4-4911de478109.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157730-59360680-f73d-11ea-99db-6f6195d6a28a.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157742-5e935100-f73d-11ea-9241-e6e2a9d78e41.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157716-53402580-f73d-11ea-8dbd-90a1088bed8e.png)
   ## non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/93157886-afa34500-f73d-11ea-9e07-b9bb1b32446b.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157853-9bf7de80-f73d-11ea-9537-d863abf5b788.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157861-a1552900-f73d-11ea-9205-60a642546de7.png)
   ![image](https://user-images.githubusercontent.com/32640567/93157871-a7e3a080-f73d-11ea-9180-87a6981e3ba0.png)
   # Cluster_new_cache_leader 
   ## Version
   229779f adjust close cache leader logic
   ## Close cache leader
   ###  forward(benchmark writes data to 34, sg belongs to 35,internal forward)
   ![image](https://user-images.githubusercontent.com/32640567/93161978-4ffd6780-f746-11ea-80d0-8476f5144510.png)
   ![image](https://user-images.githubusercontent.com/32640567/93161944-3e1bc480-f746-11ea-862e-3210515f4de7.png)
   ![image](https://user-images.githubusercontent.com/32640567/93161951-42e07880-f746-11ea-8102-300ee589b338.png)
   ![image](https://user-images.githubusercontent.com/32640567/93161966-4aa01d00-f746-11ea-905a-c2d5940a4ad8.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/93162617-b5058d00-f747-11ea-87ff-5603e2e3874b.png)
   ![image](https://user-images.githubusercontent.com/32640567/93162562-97d0be80-f747-11ea-95a6-b68da54e2a2e.png)
   ![image](https://user-images.githubusercontent.com/32640567/93162571-9bfcdc00-f747-11ea-91a0-c3b1f4e667fe.png)
   ![image](https://user-images.githubusercontent.com/32640567/93162588-a5864400-f747-11ea-9084-dc87c4a3e163.png)
   ## Open cache leader
   ### little forward(benchmark writes data to 34, sg belongs to 35,internal forward first write for each deviceId + cache leader)
   ![image](https://user-images.githubusercontent.com/32640567/93163061-ba170c00-f748-11ea-8d59-a20792ea72a1.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163015-a4a1e200-f748-11ea-9459-c6fdae223e37.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163020-a8cdff80-f748-11ea-89bc-874a46f47442.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163028-ae2b4a00-f748-11ea-82c6-92dcb64ff2f0.png)
   ### non-forward(benchmark writes data to 35, sg belongs to 35,no need to forward)
   ![image](https://user-images.githubusercontent.com/32640567/93163202-167a2b80-f749-11ea-86ff-672882f2a2cc.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163173-04988880-f749-11ea-949d-637c7d4a5066.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163178-0a8e6980-f749-11ea-8cfe-2bd6ed59a6a5.png)
   ![image](https://user-images.githubusercontent.com/32640567/93163187-0febb400-f749-11ea-8ae7-f94bb843e138.png)


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

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



[GitHub] [incubator-iotdb] neuyilan commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above, outside of the if statement

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       One more thing:
   if the last element of the _tmpStatus_ not set the RedirectNode, the final allRedirect value is false,
   This situation should be considered. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       the _return result;_ statements should be outside of the _if (!StatusUtils.NO_LEADER.equals(result)) {_ statements? 
   Otherwise, the plan will be executed twice.
   
   ```suggestion
         if (!StatusUtils.NO_LEADER.equals(result)) {
           result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
         }
         return result; 
   ```
   
   

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       I got it, better add more comments here~




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

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



[GitHub] [incubator-iotdb] LebronAl removed a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl removed a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-692695905


   # Benchmark Parameter
   IS_CLIENT_BIND=True
   CLIENT_NUMBER=20
   GROUP_NUMBER=20
   DEVICE_NUMBER=200
   SENSOR_NUMBER=500
   BATCH_SIZE=50
   LOOP=200
   DATA_TYPE=DOUBLE
   INSERT_MODE=session
   
   # Cluster config
   default_replica_num=2
   enablePartition=false
   is_enable_raft_log_persistence=false
   is_use_async_applier=true
   three nodes's cluster(33,34,35),benchmark(32)
   
   # Cluster_new
   ## Version
   ef2c8a2a4  fix method name
   ## Result
   ![image](https://user-images.githubusercontent.com/32640567/93212925-d3927500-f795-11ea-9fb3-8f1704873997.png)
   ![image](https://user-images.githubusercontent.com/32640567/93212874-bfe70e80-f795-11ea-9459-e07c54a5c4f4.png)
   ![image](https://user-images.githubusercontent.com/32640567/93212885-c6758600-f795-11ea-96f5-1bed48c3e958.png)
   ![image](https://user-images.githubusercontent.com/32640567/93212910-cd9c9400-f795-11ea-9410-630b27f8fc6f.png)
   # Cluster_new_cache_leader
   ## Version
   b2e92ed68 Merge branch 'cluster_new' of github.com:Apache/incubator-iotdb into cluster_new_cache_leader
   ## Result
   ![image](https://user-images.githubusercontent.com/32640567/93211566-0176ba00-f794-11ea-9ad2-485e12fbeb0f.png)
   ![image](https://user-images.githubusercontent.com/32640567/93211466-ddb37400-f793-11ea-8f76-6c8d71a02cb8.png)
   ![image](https://user-images.githubusercontent.com/32640567/93211502-e99f3600-f793-11ea-94ae-4be66e899f00.png)
   ![image](https://user-images.githubusercontent.com/32640567/93211543-f6bc2500-f793-11ea-96a2-3f42362530e4.png)


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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I copy this from old session's implemention.I will delete it anyway.




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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       ok~




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

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



[GitHub] [incubator-iotdb] LebronAl commented on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl commented on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-690013673


   @jt2594838 @neuyilan @kr11 @qiaojialin PTAL


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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I copy this from old session's implemention.I will delete it anyway.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       OK, I intend to update `endPointToSessionConnection ` first and then update `metaSessionConnection` from `endPointToSessionConnection `

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       Actually I designed this on purpose. If the first call to the `forwardPlan` function returns StatusUtils.NO_LEADER. We should block this request until the new leader is elected. So the following code should be called. This is why I put this return statement in the scope of `if`. Of course,  If the second call to the `forwardPlan` function returns StatusUtils.NO_LEADER too, then maybe just return will be better~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~ Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       Actually I designed this on purpose.If the last element of the tmpStatus do not set the RedirectNode, which means the coordinator can process at lease one plan locally, so the final allRedirect value is false, the server will not recomment client a new Node to redirect.
   BTW, maybe in our user's most scenarios, forwarding the plan to second Node is better, I will fix this.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       copy that!

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I copy this from old session's implemention.I will delete it anyway.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       OK, I intend to update `endPointToSessionConnection ` first and then update `metaSessionConnection` from `endPointToSessionConnection `

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       Actually I designed this on purpose. If the first call to the `forwardPlan` function returns StatusUtils.NO_LEADER. We should block this request until the new leader is elected. So the following code should be called. This is why I put this return statement in the scope of `if`. Of course,  If the second call to the `forwardPlan` function returns StatusUtils.NO_LEADER too, then maybe just return will be better~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~ Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       Actually I designed this on purpose.If the last element of the tmpStatus do not set the RedirectNode, which means the coordinator can process at lease one plan locally, so the final allRedirect value is false, the server will not recomment client a new Node to redirect.
   BTW, maybe in our user's most scenarios, forwarding the plan to second Node is better, I will fix this.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       copy that!

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I copy this from old session's implemention.I will delete it anyway.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       OK, I intend to update `endPointToSessionConnection ` first and then update `metaSessionConnection` from `endPointToSessionConnection `

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       Actually I designed this on purpose. If the first call to the `forwardPlan` function returns StatusUtils.NO_LEADER. We should block this request until the new leader is elected. So the following code should be called. This is why I put this return statement in the scope of `if`. Of course,  If the second call to the `forwardPlan` function returns StatusUtils.NO_LEADER too, then maybe just return will be better~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~ Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       Actually I designed this on purpose.If the last element of the tmpStatus do not set the RedirectNode, which means the coordinator can process at lease one plan locally, so the final allRedirect value is false, the server will not recomment client a new Node to redirect.
   BTW, maybe in our user's most scenarios, forwarding the plan to second Node is better, I will fix this.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       copy that!

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I copy this from old session's implemention.I will delete it anyway.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       OK, I intend to update `endPointToSessionConnection ` first and then update `metaSessionConnection` from `endPointToSessionConnection `

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       Actually I designed this on purpose. If the first call to the `forwardPlan` function returns StatusUtils.NO_LEADER. We should block this request until the new leader is elected. So the following code should be called. This is why I put this return statement in the scope of `if`. Of course,  If the second call to the `forwardPlan` function returns StatusUtils.NO_LEADER too, then maybe just return will be better~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~ Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       Actually I designed this on purpose.If the last element of the tmpStatus do not set the RedirectNode, which means the coordinator can process at lease one plan locally, so the final allRedirect value is false, the server will not recomment client a new Node to redirect.
   BTW, maybe in our user's most scenarios, forwarding the plan to second Node is better, I will fix this.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       copy that!

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I copy this from old session's implemention.I will delete it anyway.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       OK~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       ok~

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       OK, I intend to update `endPointToSessionConnection ` first and then update `metaSessionConnection` from `endPointToSessionConnection `

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       Actually I designed this on purpose. If the first call to the `forwardPlan` function returns StatusUtils.NO_LEADER. We should block this request until the new leader is elected. So the following code should be called. This is why I put this return statement in the scope of `if`. Of course,  If the second call to the `forwardPlan` function returns StatusUtils.NO_LEADER too, then maybe just return will be better~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above~ Anything can be discussed here~

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       Actually I designed this on purpose.If the last element of the tmpStatus do not set the RedirectNode, which means the coordinator can process at lease one plan locally, so the final allRedirect value is false, the server will not recomment client a new Node to redirect.
   BTW, maybe in our user's most scenarios, forwarding the plan to second Node is better, I will fix this.

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       copy that!




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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       OK, I intend to update `endPointToSessionConnection ` first and then update `metaSessionConnection` from `endPointToSessionConnection `




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

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



[GitHub] [incubator-iotdb] jt2594838 merged pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
jt2594838 merged pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691


   


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

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



[GitHub] [incubator-iotdb] LebronAl edited a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-689985530






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

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



[GitHub] [incubator-iotdb] LebronAl edited a comment on pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

Posted by GitBox <gi...@apache.org>.
LebronAl edited a comment on pull request #1691:
URL: https://github.com/apache/incubator-iotdb/pull/1691#issuecomment-690013673


   @jt2594838 @neuyilan @Ring-k  @qiaojialin PTAL


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

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



[GitHub] [incubator-iotdb] neuyilan commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1486,7 +1485,11 @@ private TSStatus processNonPartitionedMetaPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, null);
+      TSStatus result = forwardPlan(plan, leader, null);
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;

Review comment:
       same as above, outside of the if statement

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       One more thing:
   if the last element of the _tmpStatus_ not set the RedirectNode, the final allRedirect value is false,
   This situation should be considered. 

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       the _return result;_ statements should be outside of the _if (!StatusUtils.NO_LEADER.equals(result)) {_ statements? 
   Otherwise, the plan will be executed twice.
   
   ```suggestion
         if (!StatusUtils.NO_LEADER.equals(result)) {
           result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
         }
         return result; 
   ```
   
   

##########
File path: service-rpc/src/main/java/org/apache/iotdb/rpc/RpcUtils.java
##########
@@ -53,6 +55,14 @@ public static void verifySuccess(TSStatus status) throws StatementExecutionExcep
     }
   }
 
+  public static void verifySuccessWithRedirection(TSStatus status)
+      throws StatementExecutionException, RedirectException {
+    verifySuccess(status);
+    if (status.isSetRedirectNode()) {
+      throw new RedirectException(status.getRedirectNode());

Review comment:
       What this method is used for? and why need to throw an exception?

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       I got it, better add more comments here~




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

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



[GitHub] [incubator-iotdb] LebronAl commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
##########
@@ -1115,7 +1118,11 @@ TSStatus executeNonQueryPlan(PhysicalPlan plan) {
         return status;
       }
     } else if (leader != null) {
-      return forwardPlan(plan, leader, getHeader());
+      TSStatus result = forwardPlan(plan, leader, getHeader());
+      if (!StatusUtils.NO_LEADER.equals(result)) {
+        result.setRedirectNode(new EndPoint(leader.getIp(), leader.getClientPort()));
+        return result;
+      }

Review comment:
       copy that!




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

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



[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #1691: [IOTDB-814] Cache and redirect distributed IoTDB's leader for write requests in IoTDB's client

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



##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
##########
@@ -1688,13 +1693,20 @@ private TSStatus forwardInsertTabletPlan(Map<PhysicalPlan, PartitionGroup> planG
     TSStatus[] subStatus = null;
     boolean noFailure = true;
     boolean isBatchFailure = false;
+    boolean allRedirect = true;
+    EndPoint endPoint = null;
     for (Map.Entry<PhysicalPlan, PartitionGroup> entry : planGroupMap.entrySet()) {
       tmpStatus = forwardToSingleGroup(entry);
       logger.debug("{}: from {},{},{}", name, entry.getKey(), entry.getValue(), tmpStatus);
       noFailure =
           (tmpStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) && noFailure;
       isBatchFailure = (tmpStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode())
           || isBatchFailure;
+      if (tmpStatus.isSetRedirectNode()) {
+        endPoint = tmpStatus.getRedirectNode();
+      } else {
+        allRedirect = false;
+      }

Review comment:
       I think the redirect strategy can be improved a little.
   Mostly the plan is forwarded to multiple groups because its data has crossed multiple time partitions, and generally, this only happens on time partition borders as a plan contains a relatively short time range compared to the time partition length.
   So typically `planGroupMap.size() <= 2`,  and when it is 2, the following plans most probably belong to the second group. Consequently, it would reasonable if we always set the redirect target to that of the sub-plan with the largest timestamp.
   
   For example, if the partition length is a day, and a plan that contains data from Thuesday 11:50 to Wednesday 00:10 will be split into two sub-plans, each contains data from Thuesday 11:50 to Thuesday 11:59 and  Wednesday 00:00 to  Wednesday 00:10 respectively, and the two sub-plans are forwarded to NodeA and NodeB.
   It is clear that the following plans, which contain data of Wednesday, should be forwarded to NodeB.
   
   By the way, I think the use of a map to split plan is unnecessary, and a `List<Pair<>>` should be just enough, and you may sort it according to the largest timestamp easily.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);

Review comment:
       `Collections.singletonList()` should be enough.

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {

Review comment:
       I think this condition is redundant.

##########
File path: session/src/main/java/org/apache/iotdb/session/SessionConnection.java
##########
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.session;
+
+import java.time.ZoneId;
+import java.util.List;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.RedirectException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq;
+import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSGetTimeZoneResp;
+import org.apache.iotdb.service.rpc.thrift.TSIService;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertStringRecordsReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
+import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
+import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TFastFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SessionConnection {
+
+  private static final Logger logger = LoggerFactory.getLogger(SessionConnection.class);
+  private Session session;
+  private TTransport transport;
+  private TSIService.Iface client;
+  private long sessionId;
+  private long statementId;
+  private ZoneId zoneId;
+
+  public SessionConnection(Session session, EndPoint endPoint) throws IoTDBConnectionException {
+    this.session = session;
+    init(endPoint);
+  }
+
+  private void init(EndPoint endPoint) throws IoTDBConnectionException {
+    transport = new TFastFramedTransport(
+        new TSocket(endPoint.getIp(), endPoint.getPort(), session.connectionTimeoutInMs));
+
+    if (!transport.isOpen()) {
+      try {
+        transport.open();
+      } catch (TTransportException e) {
+        throw new IoTDBConnectionException(e);
+      }
+    }
+
+    if (session.enableRPCCompression) {
+      client = new TSIService.Client(new TCompactProtocol(transport));
+    } else {
+      client = new TSIService.Client(new TBinaryProtocol(transport));
+    }
+
+    TSOpenSessionReq openReq = new TSOpenSessionReq();
+    openReq.setUsername(session.username);
+    openReq.setPassword(session.password);
+
+    try {
+      TSOpenSessionResp openResp = client.openSession(openReq);
+
+      RpcUtils.verifySuccess(openResp.getStatus());
+
+      if (session.protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
+        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
+            session.protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
+        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
+          throw new TException(String
+              .format("Protocol not supported, Client version is %s, but Server version is %s",
+                  session.protocolVersion.getValue(),
+                  openResp.getServerProtocolVersion().getValue()));
+        }
+      }
+
+      sessionId = openResp.getSessionId();
+      statementId = client.requestStatementId(sessionId);
+
+      if (zoneId != null) {
+        setTimeZone(zoneId.toString());
+      } else {
+        zoneId = ZoneId.of(getTimeZone());
+      }
+
+    } catch (Exception e) {
+      transport.close();
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+
+  public void close() throws IoTDBConnectionException {
+    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
+    try {
+      client.closeSession(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(
+          "Error occurs when closing session at server. Maybe server is down.", e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+
+  protected void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSSetTimeZoneReq req = new TSSetTimeZoneReq(sessionId, zoneId);
+    TSStatus resp;
+    try {
+      resp = client.setTimeZone(req);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp);
+    this.zoneId = ZoneId.of(zoneId);
+  }
+
+  protected String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    if (zoneId != null) {
+      return zoneId.toString();
+    }
+    TSGetTimeZoneResp resp;
+    try {
+      resp = client.getTimeZone(sessionId);
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+    RpcUtils.verifySuccess(resp.getStatus());
+    return resp.getTimeZone();
+  }
+
+  protected void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.setStorageGroup(sessionId, storageGroup));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException, RedirectException {
+    try {
+      RpcUtils.verifySuccessWithRedirection(client.deleteStorageGroups(sessionId, storageGroups));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createTimeseries(TSCreateTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
+      throws IoTDBConnectionException, StatementExecutionException {
+    request.setSessionId(sessionId);
+    try {
+      RpcUtils.verifySuccess(client.createMultiTimeseries(request));
+    } catch (TException e) {
+      throw new IoTDBConnectionException(e);
+    }
+  }
+
+  protected boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    SessionDataSet dataSet = executeQueryStatement(String.format("SHOW TIMESERIES %s", path));
+    boolean result = dataSet.hasNext();
+    dataSet.closeOperationHandle();

Review comment:
       Close operation should be put into a finally block.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);

Review comment:
       Assuming current metaConnection is to NodeA, but the actual leader is NodeB:
   1. setStorageGroup("root.sg1") is executed, and a redirection to NodeB is returned
   2. leader switched to NodeA
   3. setStorageGroup("root.sg2") is executed, and a redirection to NodeA is returned
   4. `metaSessionConnection = new SessionConnection(this, e.getEndPoint())` is executed, but as there is already one connection to NodeA in `endPointToSessionConnection`, the new connection will be lost when the next redirection comes.
   
   So, maybe you should check `endPointToSessionConnection` before newing a connection.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -691,42 +857,38 @@ public void testInsertRecord(String deviceId, long time, List<String> measuremen
    */
   public void deleteTimeseries(String path)
       throws IoTDBConnectionException, StatementExecutionException {
-    List<String> paths = new ArrayList<>();
-    paths.add(path);
-    deleteTimeseries(paths);
+    defaultSessionConnection.deleteTimeseries(new ArrayList<String>() {{
+      add(path);

Review comment:
       Collections.singletonList should be enough.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -121,209 +105,326 @@ private synchronized void open(boolean enableRPCCompression, int connectionTimeo
     if (!isClosed) {
       return;
     }
+    this.enableRPCCompression = enableRPCCompression;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
 
-    transport = new TFastFramedTransport(new TSocket(host, port, connectionTimeoutInMs));
+    defaultSessionConnection = new SessionConnection(this, defaultEndPoint);
+    metaSessionConnection = defaultSessionConnection;
+    isClosed = false;
+    if (Config.DEFAULT_CACHE_LEADER_MODE) {
+      deviceIdToEndpoint = new HashMap<>();
+      endPointToSessionConnection = new HashMap<EndPoint, SessionConnection>() {{
+        put(defaultEndPoint, defaultSessionConnection);
+      }};
+    }
+  }
 
-    if (!transport.isOpen()) {
-      try {
-        transport.open();
-      } catch (TTransportException e) {
-        throw new IoTDBConnectionException(e);
+  public synchronized void close() throws IoTDBConnectionException {
+    if (isClosed) {
+      return;
+    }
+    try {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        for (SessionConnection sessionConnection : endPointToSessionConnection.values()) {
+          sessionConnection.close();
+        }
+      } else {
+        defaultSessionConnection.close();
       }
+    } finally {
+      isClosed = true;
     }
+  }
 
-    if (enableRPCCompression) {
-      client = new TSIService.Client(new TCompactProtocol(transport));
-    } else {
-      client = new TSIService.Client(new TBinaryProtocol(transport));
-    }
+  public synchronized String getTimeZone()
+      throws StatementExecutionException, IoTDBConnectionException {
+    return defaultSessionConnection.getTimeZone();
+  }
 
-    TSOpenSessionReq openReq = new TSOpenSessionReq();
-    openReq.setUsername(username);
-    openReq.setPassword(password);
+  public synchronized void setTimeZone(String zoneId)
+      throws StatementExecutionException, IoTDBConnectionException {
+    defaultSessionConnection.setTimeZone(zoneId);
+  }
 
+  public void setStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
     try {
-      TSOpenSessionResp openResp = client.openSession(openReq);
+      metaSessionConnection.setStorageGroup(storageGroup);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      RpcUtils.verifySuccess(openResp.getStatus());
+  public void deleteStorageGroup(String storageGroup)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(new ArrayList<String>() {{
+        add(storageGroup);
+      }});
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug("storageGroup[{}]:{}", storageGroup, e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
+      }
+    }
+  }
 
-      if (protocolVersion.getValue() != openResp.getServerProtocolVersion().getValue()) {
-        logger.warn("Protocol differ, Client version is {}}, but Server version is {}",
-            protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue());
-        if (openResp.getServerProtocolVersion().getValue() == 0) {// less than 0.10
-          throw new TException(String
-              .format("Protocol not supported, Client version is %s, but Server version is %s",
-                  protocolVersion.getValue(), openResp.getServerProtocolVersion().getValue()));
-        }
+  public void deleteStorageGroups(List<String> storageGroups)
+      throws IoTDBConnectionException, StatementExecutionException {
+    try {
+      metaSessionConnection.deleteStorageGroups(storageGroups);
+    } catch (RedirectException e) {
+      if (Config.DEFAULT_CACHE_LEADER_MODE) {
+        logger.debug(e.getMessage());
+        metaSessionConnection = new SessionConnection(this, e.getEndPoint());
+        endPointToSessionConnection.putIfAbsent(e.getEndPoint(), metaSessionConnection);
       }
+    }
+  }
 
-      sessionId = openResp.getSessionId();
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        null, null, null, null);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      statementId = client.requestStatementId(sessionId);
+  public void createTimeseries(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateTimeseriesReq request = genTSCreateTimeseriesReq(path, dataType, encoding, compressor,
+        props, tags, attributes, measurementAlias);
+    defaultSessionConnection.createTimeseries(request);
+  }
 
-      if (zoneId != null) {
-        setTimeZone(zoneId.toString());
-      } else {
-        zoneId = ZoneId.of(getTimeZone());
-      }
+  private TSCreateTimeseriesReq genTSCreateTimeseriesReq(String path, TSDataType dataType,
+      TSEncoding encoding, CompressionType compressor, Map<String, String> props,
+      Map<String, String> tags, Map<String, String> attributes, String measurementAlias) {
+    TSCreateTimeseriesReq request = new TSCreateTimeseriesReq();
+    request.setPath(path);
+    request.setDataType(dataType.ordinal());
+    request.setEncoding(encoding.ordinal());
+    request.setCompressor(compressor.ordinal());
+    request.setProps(props);
+    request.setTags(tags);
+    request.setAttributes(attributes);
+    request.setMeasurementAlias(measurementAlias);
+    return request;
+  }
 
-    } catch (Exception e) {
-      transport.close();
-      throw new IoTDBConnectionException(e);
-    }
-    isClosed = false;
+  public void createMultiTimeseries(List<String> paths, List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList)
+      throws IoTDBConnectionException, StatementExecutionException {
+    TSCreateMultiTimeseriesReq request = genTSCreateMultiTimeseriesReq(paths, dataTypes, encodings,
+        compressors, propsList, tagsList, attributesList, measurementAliasList);
+    defaultSessionConnection.createMultiTimeseries(request);
+  }
 
-    client = RpcUtils.newSynchronizedClient(client);
+  private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq(List<String> paths,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings, List<CompressionType> compressors,
+      List<Map<String, String>> propsList, List<Map<String, String>> tagsList,
+      List<Map<String, String>> attributesList, List<String> measurementAliasList) {
+    TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq();
 
-  }
+    request.setPaths(paths);
 
-  public synchronized void close() throws IoTDBConnectionException {
-    if (isClosed) {
-      return;
+    List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+    for (TSDataType dataType : dataTypes) {
+      dataTypeOrdinals.add(dataType.ordinal());
     }
-    TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
-    try {
-      client.closeSession(req);
-    } catch (TException e) {
-      throw new IoTDBConnectionException(
-          "Error occurs when closing session at server. Maybe server is down.", e);
-    } finally {
-      isClosed = true;
-      if (transport != null) {
-        transport.close();
-      }
+    request.setDataTypes(dataTypeOrdinals);
+
+    List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+    for (TSEncoding encoding : encodings) {
+      encodingOrdinals.add(encoding.ordinal());
     }
-  }
+    request.setEncodings(encodingOrdinals);
 
-  /**
-   * insert data in one row, if you want to improve your performance, please use insertRecords
-   * method or insertTablet method
-   *
-   * @see Session#insertRecords(List, List, List, List, List)
-   * @see Session#insertTablet(Tablet)
-   */
-  public void insertRecord(String deviceId, long time, List<String> measurements,
-      List<TSDataType> types,
-      Object... values) throws IoTDBConnectionException, StatementExecutionException {
-    List<Object> valuesList = new ArrayList<>(Arrays.asList(values));
+    List<Integer> compressionOrdinals = new ArrayList<>(paths.size());
+    for (CompressionType compression : compressors) {
+      compressionOrdinals.add(compression.ordinal());
+    }
+    request.setCompressors(compressionOrdinals);
 
-    insertRecord(deviceId, time, measurements, types, valuesList);
+    request.setPropsList(propsList);
+    request.setTagsList(tagsList);
+    request.setAttributesList(attributesList);
+    request.setMeasurementAliasList(measurementAliasList);
+
+    return request;
   }
 
+  public boolean checkTimeseriesExists(String path)
+      throws IoTDBConnectionException, StatementExecutionException {
+    return defaultSessionConnection.checkTimeseriesExists(path);
+  }
 
   /**
-   * insert the data of a device. For each timestamp, the number of measurements is the same.
-   * <p>
-   * a Tablet example:
-   * <p>
-   * device1 time s1, s2, s3 1,   1,  1,  1 2,   2,  2,  2 3,   3,  3,  3
-   * <p>
-   * times in Tablet may be not in ascending order
+   * execure query sql
    *
-   * @param tablet data batch
+   * @param sql query statement
+   * @return result set
    */
-  public void insertTablet(Tablet tablet)
+  public SessionDataSet executeQueryStatement(String sql)
       throws StatementExecutionException, IoTDBConnectionException {
-    insertTablet(tablet, false);
+    return defaultSessionConnection.executeQueryStatement(sql);
   }
 
   /**
-   * insert a Tablet
+   * execute non query statement
    *
-   * @param tablet data batch
-   * @param sorted whether times in Tablet are in ascending order
+   * @param sql non query statement
    */
-  public void insertTablet(Tablet tablet, boolean sorted)
+  public void executeNonQueryStatement(String sql)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertTabletReq request = genTSInsertTabletReq(tablet, sorted);
-    try {
-      RpcUtils.verifySuccess(client.insertTablet(request));
-    } catch (TException e) {
-      throw new IoTDBConnectionException(e);
-    }
+    defaultSessionConnection.executeNonQueryStatement(sql);
   }
 
-  private TSInsertTabletReq genTSInsertTabletReq(Tablet tablet, boolean sorted)
-      throws BatchExecutionException {
-    if (sorted) {
-      if (!checkSorted(tablet)) {
-        throw new BatchExecutionException("Times in Tablet are not in ascending order");
-      }
-    } else {
-      sortTablet(tablet);
-    }
+  /**
+   * query eg. select * from paths where time >= startTime and time < endTime time interval include
+   * startTime and exclude endTime
+   *
+   * @param paths
+   * @param startTime included
+   * @param endTime   excluded
+   * @return
+   * @throws StatementExecutionException
+   * @throws IoTDBConnectionException
+   */
 
-    TSInsertTabletReq request = new TSInsertTabletReq();
-    request.setSessionId(sessionId);
-    request.deviceId = tablet.deviceId;
-    for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
-      request.addToMeasurements(measurementSchema.getMeasurementId());
-      request.addToTypes(measurementSchema.getType().ordinal());
-    }
-    request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
-    request.setValues(SessionUtils.getValueBuffer(tablet));
-    request.setSize(tablet.rowSize);
+  public SessionDataSet executeRawDataQuery(List<String> paths, long startTime, long endTime)
+      throws StatementExecutionException, IoTDBConnectionException {
+    TSRawDataQueryReq request = genTSRawDataQueryReq(paths, startTime, endTime);
+    return defaultSessionConnection.executeRawDataQuery(request);
+  }
+
+  private TSRawDataQueryReq genTSRawDataQueryReq(List<String> paths, long startTime, long endTime) {
+    TSRawDataQueryReq request = new TSRawDataQueryReq();
+    request.setPaths(paths);
+    request.setStartTime(startTime);
+    request.setEndTime(endTime);
     return request;
   }
 
   /**
-   * insert the data of several deivces. Given a deivce, for each timestamp, the number of
-   * measurements is the same.
-   * <p>
-   * Times in each Tablet may not be in ascending order
+   * insert data in one row, if you want to improve your performance, please use insertRecords
+   * method or insertTablet method
    *
-   * @param tablets data batch in multiple device
+   * @see Session#insertRecords(List, List, List, List, List)
+   * @see Session#insertTablet(Tablet)
    */
-  public void insertTablets(Map<String, Tablet> tablets)
-      throws IoTDBConnectionException, StatementExecutionException {
-    insertTablets(tablets, false);
+  public void insertRecord(String deviceId, long time, List<String> measurements,
+      List<TSDataType> types,
+      Object... values) throws IoTDBConnectionException, StatementExecutionException {
+    TSInsertRecordReq request = genTSInsertRecordReq(deviceId, time, measurements, types,
+        new ArrayList<>(Arrays.asList(values)));

Review comment:
       I think `new ArrayList<>` is unnecessary.




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

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