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/11/13 10:07:50 UTC

[GitHub] [iotdb] yanhongwangg opened a new pull request #2046: client-go

yanhongwangg opened a new pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046


   


----------------------------------------------------------------
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] [iotdb] neuyilan commented on a change in pull request #2046: client-go

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



##########
File path: client-go/go.mod
##########
@@ -0,0 +1,9 @@
+module client-go
+
+go 1.14
+
+require (
+	github.com/apache/thrift v0.13.0
+	github.com/typa01/go-utils v0.0.0-20181126045345-a86b05b01c1e

Review comment:
       what is this used for? I see this project only has 9 stars, it's reliable? 

##########
File path: client-go/src/main/iotdbSession/GetSession.go
##########
@@ -0,0 +1,97 @@
+/**
+ * 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 iotdbSession

Review comment:
       pacakge name: session is just enough

##########
File path: client-go/src/main/iotdb_session/get_session.go
##########
@@ -0,0 +1,97 @@
+/**
+ * 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 iotdbSession
+
+const DefaultUser = "root"
+const DefaultPasswd = "root"
+const DefaultZoneId = "Asia/Shanghai"
+const DefaultFetchSize int32 = 10000
+
+type Session struct {
+	Host      string
+	Port      string
+	User      string
+	Passwd    string
+	FetchSize int32
+	ZoneId    string
+}
+
+type DialOption interface {
+	apply(*Session)
+}
+
+type FuncOption struct {
+	f func(*Session)
+}
+
+func (O *FuncOption) apply(s *Session) {
+	O.f(s)
+}
+
+func newFuncOption(f func(*Session)) *FuncOption {
+	return &FuncOption{
+		f: f,
+	}
+}
+
+func withUser(user string) DialOption {
+	return newFuncOption(func(session *Session) {
+		session.User = user
+	})
+}
+
+func withPasswd(passwd string) DialOption {
+	return newFuncOption(func(session *Session) {
+		session.Passwd = passwd
+	})
+}
+
+func withFetchSize(fetchSize int32) DialOption {
+	return newFuncOption(func(session *Session) {
+		session.FetchSize = fetchSize
+	})
+}
+
+//默认参数
+func defaultOptions() Session {
+	return Session{
+		User:      DefaultUser,
+		Passwd:    DefaultPasswd,
+		FetchSize: DefaultFetchSize,
+		ZoneId:    DefaultZoneId,
+	}
+}
+
+type SessionConn struct {
+	session Session
+}
+
+func NewSession(host string, port string, opts ...DialOption) Session {
+	sessionConn := &SessionConn{
+		session: defaultOptions(),
+	}
+	//循环调用opts

Review comment:
       same as above

##########
File path: client-go/src/test/Session_test.go
##########
@@ -0,0 +1,185 @@
+/**

Review comment:
       the file name should all use lower case

##########
File path: client-go/src/main/iotdb_session/get_session.go
##########
@@ -0,0 +1,97 @@
+/**
+ * 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 iotdbSession
+
+const DefaultUser = "root"

Review comment:
       ```suggestion
   const {
   DefaultUser = "root"
   DefaultPasswd = "root"
   }
   
   ```

##########
File path: client-go/src/main/iotdb_session/session.go
##########
@@ -0,0 +1,290 @@
+/**
+ * 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 iotdbSession
+
+import (
+	"context"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"log"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var client *rpc.TSIServiceClient

Review comment:
       Group Similar Declarations, This also applies to constants, variables, and type declarations.
   
   https://github.com/uber-go/guide/blob/master/style.md#group-similar-declarations
   

##########
File path: client-go/src/main/iotdb_session/get_session.go
##########
@@ -0,0 +1,97 @@
+/**
+ * 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 iotdbSession

Review comment:
       ```suggestion
   package session
   ```

##########
File path: client-go/src/main/iotdb_session/get_session.go
##########
@@ -0,0 +1,97 @@
+/**
+ * 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 iotdbSession
+
+const DefaultUser = "root"
+const DefaultPasswd = "root"
+const DefaultZoneId = "Asia/Shanghai"
+const DefaultFetchSize int32 = 10000
+
+type Session struct {
+	Host      string
+	Port      string
+	User      string
+	Passwd    string
+	FetchSize int32
+	ZoneId    string
+}
+
+type DialOption interface {
+	apply(*Session)
+}
+
+type FuncOption struct {
+	f func(*Session)
+}
+
+func (O *FuncOption) apply(s *Session) {
+	O.f(s)
+}
+
+func newFuncOption(f func(*Session)) *FuncOption {
+	return &FuncOption{
+		f: f,
+	}
+}
+
+func withUser(user string) DialOption {
+	return newFuncOption(func(session *Session) {
+		session.User = user
+	})
+}
+
+func withPasswd(passwd string) DialOption {
+	return newFuncOption(func(session *Session) {
+		session.Passwd = passwd
+	})
+}
+
+func withFetchSize(fetchSize int32) DialOption {
+	return newFuncOption(func(session *Session) {
+		session.FetchSize = fetchSize
+	})
+}
+
+//默认参数

Review comment:
       change it to English comment

##########
File path: client-go/src/main/iotdb_session/session.go
##########
@@ -0,0 +1,290 @@
+/**
+ * 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 iotdbSession
+
+import (
+	"context"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"log"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var client *rpc.TSIServiceClient
+var sessionId int64
+var isClose bool = true
+var trans thrift.TTransport
+var err error
+var requestStatementId int64
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) error {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)

Review comment:
       I think the logs should be redesign,
   I suggest using the log : https://github.com/sirupsen/logrus 

##########
File path: client-go/src/main/iotdb_session/utils/field.go
##########
@@ -0,0 +1,122 @@
+/**
+ * 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 utils
+
+import (
+	"encoding/json"
+)
+
+type Field struct {
+	DataType int32
+}
+
+var boolV bool

Review comment:
       the same as above




----------------------------------------------------------------
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] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r528515840



##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,708 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var (
+	client             *rpc.TSIServiceClient
+	sessionId          int64
+	trans              thrift.TTransport
+	err                error
+	requestStatementId int64
+	ts                 string
+	sg                 string
+	dv                 string
+	Log                = logrus.New()
+)

Review comment:
       Thanks for your guidance




----------------------------------------------------------------
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] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r528632236



##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,698 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var Log = logrus.New()
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)
+	logFile, _ := os.OpenFile(dir+"\\logs\\all.log", os.O_CREATE|os.O_RDWR|os.O_APPEND, 0666)
+	Log.SetOutput(logFile)
+	var protocolFactory thrift.TProtocolFactory
+	s.trans, s.err = thrift.NewTSocketTimeout(net.JoinHostPort(s.Host, s.Port), time.Duration(connectionTimeoutInMs))
+	if s.err != nil {
+		Log.WithError(s.err).Error("connect failed")
+	}
+	s.trans = thrift.NewTFramedTransport(s.trans)
+	if !s.trans.IsOpen() {
+		s.err = s.trans.Open()
+		if s.err != nil {
+			Log.WithError(s.err).Error("open the conn failed")
+		}
+	}
+	if enableRPCCompression {
+		protocolFactory = thrift.NewTCompactProtocolFactory()
+	} else {
+		protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
+	}
+	iProtocol := protocolFactory.GetProtocol(s.trans)
+	oProtocol := protocolFactory.GetProtocol(s.trans)
+	s.client = rpc.NewTSIServiceClient(thrift.NewTStandardClient(iProtocol, oProtocol))
+	s.ZoneId = DefaultZoneId
+	tSOpenSessionReq := rpc.TSOpenSessionReq{ClientProtocol: protocolVersion, ZoneId: s.ZoneId, Username: &s.User,
+		Password: &s.Passwd}
+	tSOpenSessionResp, err := s.client.OpenSession(context.Background(), &tSOpenSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("open session failed")
+	} else {
+		Log.WithField("code", tSOpenSessionResp.GetStatus().Code).Info("open session success")
+	}
+	s.sessionId = tSOpenSessionResp.GetSessionId()
+	s.requestStatementId, err = s.client.RequestStatementId(context.Background(), s.sessionId)
+	s.FetchSize = DefaultFetchSize
+	if err != nil {
+		Log.WithError(err).Error("request StatementId failed")
+	}
+	if s.ZoneId != "" {
+		s.SetTimeZone(s.ZoneId)
+	} else {
+		s.ZoneId = s.GetTimeZone()
+	}
+}
+
+func (s *Session) CheckTimeseriesExists(path string) bool {
+	dataSet := s.ExecuteQueryStatement("SHOW TIMESERIES " + path)
+	result := dataSet.HasNext()
+	dataSet.CloseOperationHandle()
+	return result
+}
+
+func (s *Session) Close() error {
+	tSCloseSessionReq := rpc.NewTSCloseSessionReq()
+	tSCloseSessionReq.SessionId = s.sessionId
+	status, err := s.client.CloseSession(context.Background(), tSCloseSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("Error occurs when closing session at server. Maybe server is down. Error message")
+		return err
+	} else {
+		Log.WithField("code", status.Code).Info("close session success")
+	}
+	s.trans.Close()
+	return nil
+}
+
+/*
+ *set one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) SetStorageGroup(storageGroupId string) error {
+	status, err := s.client.SetStorageGroup(context.Background(), s.sessionId, storageGroupId)
+	if err != nil {
+		Log.WithError(err).Error("setting storage group failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Info("setting storage group success")
+	}
+	return err
+}
+
+/*
+ *delete one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteStorageGroup(storageGroupId string) {

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] [iotdb] neuyilan commented on a change in pull request #2046: client-go

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



##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,667 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	log "github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)
+	logFile, _ := os.OpenFile(dir+"\\logs\\all.log", os.O_CREATE|os.O_RDWR|os.O_APPEND, 0666)
+	log.SetOutput(logFile)
+	log.SetLevel(log.InfoLevel)
+	var protocolFactory thrift.TProtocolFactory
+	s.trans, s.err = thrift.NewTSocketTimeout(net.JoinHostPort(s.Host, s.Port), time.Duration(connectionTimeoutInMs))
+	if s.err != nil {
+		log.WithError(s.err).Error("connect failed")
+	}
+	s.trans = thrift.NewTFramedTransport(s.trans)
+	if !s.trans.IsOpen() {
+		s.err = s.trans.Open()
+		if s.err != nil {
+			log.WithError(s.err).Error("open the conn failed")
+		}
+	}
+	if enableRPCCompression {
+		protocolFactory = thrift.NewTCompactProtocolFactory()
+	} else {
+		protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
+	}
+	iProtocol := protocolFactory.GetProtocol(s.trans)
+	oProtocol := protocolFactory.GetProtocol(s.trans)
+	s.client = rpc.NewTSIServiceClient(thrift.NewTStandardClient(iProtocol, oProtocol))
+	s.ZoneId = DefaultZoneId
+	tSOpenSessionReq := rpc.TSOpenSessionReq{ClientProtocol: protocolVersion, ZoneId: s.ZoneId, Username: &s.User,
+		Password: &s.Passwd}
+	tSOpenSessionResp, err := s.client.OpenSession(context.Background(), &tSOpenSessionReq)
+	if err != nil {
+		log.WithError(err).Error("open session failed")
+	} else {
+		log.WithField("code", tSOpenSessionResp.GetStatus().Code).Debug("open session success")
+	}
+	s.sessionId = tSOpenSessionResp.GetSessionId()
+	s.requestStatementId, err = s.client.RequestStatementId(context.Background(), s.sessionId)
+	s.FetchSize = DefaultFetchSize
+	if err != nil {
+		log.WithError(err).Error("request StatementId failed")
+	}
+	if s.ZoneId != "" {
+		s.SetTimeZone(s.ZoneId)
+	} else {
+		s.ZoneId = s.GetTimeZone()
+	}
+}
+
+func (s *Session) CheckTimeseriesExists(path string) bool {
+	dataSet := s.ExecuteQueryStatement("SHOW TIMESERIES " + path)
+	result := dataSet.HasNext()
+	dataSet.CloseOperationHandle()
+	return result
+}
+
+func (s *Session) Close() {
+	tSCloseSessionReq := rpc.NewTSCloseSessionReq()
+	tSCloseSessionReq.SessionId = s.sessionId
+	status, err := s.client.CloseSession(context.Background(), tSCloseSessionReq)
+	if err != nil {
+		log.WithError(err).Error("Error occurs when closing session at server. Maybe server is down.")
+	} else {
+		log.WithField("code", status.Code).Debug("close session success")
+	}
+	s.trans.Close()
+}
+
+/*
+ *set one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ */
+func (s *Session) SetStorageGroup(storageGroupId string) {
+	status, err := s.client.SetStorageGroup(context.Background(), s.sessionId, storageGroupId)
+	if err != nil {
+		log.WithError(err).Error("setting storage group failed")
+	} else {
+		log.WithFields(log.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Debug("setting storage group success")
+	}
+}
+
+/*
+ *delete one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ */
+func (s *Session) DeleteStorageGroup(storageGroupId string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, []string{storageGroupId})
+	if err != nil {
+		log.WithError(err).Error("delete storage group failed")
+	} else {
+		log.WithFields(log.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Debug("delete storage group success")
+	}
+}
+
+/*
+ *delete multiple storage group
+ *
+ *param
+ *storageGroupIds: []string, paths of the target storage groups
+ *
+ */
+func (s *Session) DeleteStorageGroups(storageGroupIds []string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, storageGroupIds)
+	s.sg = strings.Replace(strings.Trim(fmt.Sprint(storageGroupIds), "[]"), " ", ",", -1)

Review comment:
       Do not do the code here as you just want to print the devices in debug log, for every operation, the code will be called, it'll cost cpu
   `s.sg = strings.Replace(strings.Trim(fmt.Sprint(storageGroupIds), "[]"), " ", ",", -1)`

##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,667 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	log "github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)
+	logFile, _ := os.OpenFile(dir+"\\logs\\all.log", os.O_CREATE|os.O_RDWR|os.O_APPEND, 0666)
+	log.SetOutput(logFile)
+	log.SetLevel(log.InfoLevel)
+	var protocolFactory thrift.TProtocolFactory
+	s.trans, s.err = thrift.NewTSocketTimeout(net.JoinHostPort(s.Host, s.Port), time.Duration(connectionTimeoutInMs))
+	if s.err != nil {
+		log.WithError(s.err).Error("connect failed")
+	}
+	s.trans = thrift.NewTFramedTransport(s.trans)
+	if !s.trans.IsOpen() {
+		s.err = s.trans.Open()
+		if s.err != nil {
+			log.WithError(s.err).Error("open the conn failed")
+		}
+	}
+	if enableRPCCompression {
+		protocolFactory = thrift.NewTCompactProtocolFactory()
+	} else {
+		protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
+	}
+	iProtocol := protocolFactory.GetProtocol(s.trans)
+	oProtocol := protocolFactory.GetProtocol(s.trans)
+	s.client = rpc.NewTSIServiceClient(thrift.NewTStandardClient(iProtocol, oProtocol))
+	s.ZoneId = DefaultZoneId
+	tSOpenSessionReq := rpc.TSOpenSessionReq{ClientProtocol: protocolVersion, ZoneId: s.ZoneId, Username: &s.User,
+		Password: &s.Passwd}
+	tSOpenSessionResp, err := s.client.OpenSession(context.Background(), &tSOpenSessionReq)
+	if err != nil {
+		log.WithError(err).Error("open session failed")
+	} else {
+		log.WithField("code", tSOpenSessionResp.GetStatus().Code).Debug("open session success")
+	}
+	s.sessionId = tSOpenSessionResp.GetSessionId()
+	s.requestStatementId, err = s.client.RequestStatementId(context.Background(), s.sessionId)
+	s.FetchSize = DefaultFetchSize
+	if err != nil {
+		log.WithError(err).Error("request StatementId failed")
+	}
+	if s.ZoneId != "" {
+		s.SetTimeZone(s.ZoneId)
+	} else {
+		s.ZoneId = s.GetTimeZone()
+	}
+}
+
+func (s *Session) CheckTimeseriesExists(path string) bool {
+	dataSet := s.ExecuteQueryStatement("SHOW TIMESERIES " + path)
+	result := dataSet.HasNext()
+	dataSet.CloseOperationHandle()
+	return result
+}
+
+func (s *Session) Close() {
+	tSCloseSessionReq := rpc.NewTSCloseSessionReq()
+	tSCloseSessionReq.SessionId = s.sessionId
+	status, err := s.client.CloseSession(context.Background(), tSCloseSessionReq)
+	if err != nil {
+		log.WithError(err).Error("Error occurs when closing session at server. Maybe server is down.")
+	} else {
+		log.WithField("code", status.Code).Debug("close session success")
+	}
+	s.trans.Close()
+}
+
+/*
+ *set one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ */
+func (s *Session) SetStorageGroup(storageGroupId string) {
+	status, err := s.client.SetStorageGroup(context.Background(), s.sessionId, storageGroupId)
+	if err != nil {
+		log.WithError(err).Error("setting storage group failed")
+	} else {
+		log.WithFields(log.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Debug("setting storage group success")
+	}
+}
+
+/*
+ *delete one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ */
+func (s *Session) DeleteStorageGroup(storageGroupId string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, []string{storageGroupId})
+	if err != nil {
+		log.WithError(err).Error("delete storage group failed")
+	} else {
+		log.WithFields(log.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Debug("delete storage group success")
+	}
+}
+
+/*
+ *delete multiple storage group
+ *
+ *param
+ *storageGroupIds: []string, paths of the target storage groups
+ *
+ */
+func (s *Session) DeleteStorageGroups(storageGroupIds []string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, storageGroupIds)
+	s.sg = strings.Replace(strings.Trim(fmt.Sprint(storageGroupIds), "[]"), " ", ",", -1)
+	if err != nil {
+		log.WithError(err).Error("delete storage groups failed")
+	} else {
+		log.WithFields(log.Fields{
+			"sg":   s.sg,
+			"code": status.Code,
+		}).Debug("delete storage groups success")
+	}
+}
+
+/*
+ *create single time series
+ *
+ *params
+ *path: string, complete time series path (starts from root)
+ *dataType: int32, data type for this time series
+ *encoding: int32, data type for this time series
+ *compressor: int32, compressing type for this time series
+ *
+ */
+func (s *Session) CreateTimeseries(path string, dataType int32, encoding int32, compressor int32) {
+	request := rpc.TSCreateTimeseriesReq{SessionId: s.sessionId, Path: path, DataType: dataType, Encoding: encoding,
+		Compressor: compressor}
+	status, err := s.client.CreateTimeseries(context.Background(), &request)
+	if err != nil {
+		log.WithError(err).Error("creating time series failed")
+	} else {
+		log.WithFields(log.Fields{
+			"ts":   path,
+			"code": status.Code,
+		}).Debug("creating time series success")
+	}
+}
+
+/*
+ *create multiple time series
+ *
+ *params
+ *paths: []string, complete time series paths (starts from root)
+ *dataTypes: []int32, data types for time series
+ *encodings: []int32, encodings for time series
+ *compressors: []int32, compressing types for time series
+ *
+ */
+func (s *Session) CreateMultiTimeseries(paths []string, dataTypes []int32, encodings []int32, compressors []int32) {
+	request := rpc.TSCreateMultiTimeseriesReq{SessionId: s.sessionId, Paths: paths, DataTypes: dataTypes,
+		Encodings: encodings, Compressors: compressors}
+	status, err := s.client.CreateMultiTimeseries(context.Background(), &request)
+	s.ts = strings.Replace(strings.Trim(fmt.Sprint(paths), "[]"), " ", ",", -1)

Review comment:
       same as above!

##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,698 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var Log = logrus.New()
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)
+	logFile, _ := os.OpenFile(dir+"\\logs\\all.log", os.O_CREATE|os.O_RDWR|os.O_APPEND, 0666)
+	Log.SetOutput(logFile)
+	var protocolFactory thrift.TProtocolFactory
+	s.trans, s.err = thrift.NewTSocketTimeout(net.JoinHostPort(s.Host, s.Port), time.Duration(connectionTimeoutInMs))
+	if s.err != nil {
+		Log.WithError(s.err).Error("connect failed")
+	}
+	s.trans = thrift.NewTFramedTransport(s.trans)
+	if !s.trans.IsOpen() {
+		s.err = s.trans.Open()
+		if s.err != nil {
+			Log.WithError(s.err).Error("open the conn failed")
+		}
+	}
+	if enableRPCCompression {
+		protocolFactory = thrift.NewTCompactProtocolFactory()
+	} else {
+		protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
+	}
+	iProtocol := protocolFactory.GetProtocol(s.trans)
+	oProtocol := protocolFactory.GetProtocol(s.trans)
+	s.client = rpc.NewTSIServiceClient(thrift.NewTStandardClient(iProtocol, oProtocol))
+	s.ZoneId = DefaultZoneId
+	tSOpenSessionReq := rpc.TSOpenSessionReq{ClientProtocol: protocolVersion, ZoneId: s.ZoneId, Username: &s.User,
+		Password: &s.Passwd}
+	tSOpenSessionResp, err := s.client.OpenSession(context.Background(), &tSOpenSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("open session failed")
+	} else {
+		Log.WithField("code", tSOpenSessionResp.GetStatus().Code).Info("open session success")
+	}
+	s.sessionId = tSOpenSessionResp.GetSessionId()
+	s.requestStatementId, err = s.client.RequestStatementId(context.Background(), s.sessionId)
+	s.FetchSize = DefaultFetchSize
+	if err != nil {
+		Log.WithError(err).Error("request StatementId failed")
+	}
+	if s.ZoneId != "" {
+		s.SetTimeZone(s.ZoneId)
+	} else {
+		s.ZoneId = s.GetTimeZone()
+	}
+}
+
+func (s *Session) CheckTimeseriesExists(path string) bool {
+	dataSet := s.ExecuteQueryStatement("SHOW TIMESERIES " + path)
+	result := dataSet.HasNext()
+	dataSet.CloseOperationHandle()
+	return result
+}
+
+func (s *Session) Close() error {
+	tSCloseSessionReq := rpc.NewTSCloseSessionReq()
+	tSCloseSessionReq.SessionId = s.sessionId
+	status, err := s.client.CloseSession(context.Background(), tSCloseSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("Error occurs when closing session at server. Maybe server is down. Error message")
+		return err
+	} else {
+		Log.WithField("code", status.Code).Info("close session success")
+	}
+	s.trans.Close()
+	return nil
+}
+
+/*
+ *set one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) SetStorageGroup(storageGroupId string) error {
+	status, err := s.client.SetStorageGroup(context.Background(), s.sessionId, storageGroupId)
+	if err != nil {
+		Log.WithError(err).Error("setting storage group failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Info("setting storage group success")
+	}
+	return err
+}
+
+/*
+ *delete one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteStorageGroup(storageGroupId string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, []string{storageGroupId})
+	if err != nil {
+		Log.WithError(err).Error("delete storage group failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Info("delete storage group success")
+	}
+}
+
+/*
+ *delete multiple storage group
+ *
+ *param
+ *storageGroupIds: []string, paths of the target storage groups
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteStorageGroups(storageGroupIds []string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, storageGroupIds)
+	s.sg = strings.Replace(strings.Trim(fmt.Sprint(storageGroupIds), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("delete storage groups failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   s.sg,
+			"code": status.Code,
+		}).Info("delete storage groups success")
+	}
+}
+
+/*
+ *create single time series
+ *
+ *params
+ *path: string, complete time series path (starts from root)
+ *dataType: int32, data type for this time series
+ *encoding: int32, data type for this time series
+ *compressor: int32, compressing type for this time series
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) CreateTimeseries(path string, dataType int32, encoding int32, compressor int32) {
+	request := rpc.TSCreateTimeseriesReq{SessionId: s.sessionId, Path: path, DataType: dataType, Encoding: encoding,
+		Compressor: compressor}
+	status, err := s.client.CreateTimeseries(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("creating time series failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   path,
+			"code": status.Code,
+		}).Info("creating time series success")
+	}
+}
+
+/*
+ *create multiple time series
+ *
+ *params
+ *paths: []string, complete time series paths (starts from root)
+ *dataTypes: []int32, data types for time series
+ *encodings: []int32, encodings for time series
+ *compressors: []int32, compressing types for time series
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) CreateMultiTimeseries(paths []string, dataTypes []int32, encodings []int32, compressors []int32) {
+	request := rpc.TSCreateMultiTimeseriesReq{SessionId: s.sessionId, Paths: paths, DataTypes: dataTypes,
+		Encodings: encodings, Compressors: compressors}
+	status, err := s.client.CreateMultiTimeseries(context.Background(), &request)
+	s.ts = strings.Replace(strings.Trim(fmt.Sprint(paths), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("creating multi time series failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   s.ts,
+			"code": status.Code,
+		}).Info("creating multi time series success")
+	}
+}
+
+/*
+ *delete multiple time series, including data and schema
+ *
+ *params
+ *paths: []string, time series paths, which should be complete (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteTimeseries(paths []string) {
+	status, err := s.client.DeleteTimeseries(context.Background(), s.sessionId, paths)
+	var ts = strings.Replace(strings.Trim(fmt.Sprint(paths), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("delete time series failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   ts,
+			"code": status.Code,
+		}).Info("delete time series success")
+	}
+}
+
+/*
+ *delete all startTime <= data <= endTime in multiple time series
+ *
+ *params
+ *paths: []string, time series array that the data in
+ *startTime: int64, start time of deletion range
+ *endTime: int64, end time of deletion range
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteData(paths []string, startTime int64, endTime int64) {
+	request := rpc.TSDeleteDataReq{SessionId: s.sessionId, Paths: paths, StartTime: startTime, EndTime: endTime}
+	status, err := s.client.DeleteData(context.Background(), &request)
+	s.ts = strings.Replace(strings.Trim(fmt.Sprint(paths), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("delete data failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   s.ts,
+			"code": status.Code,
+		}).Info("delete data success")
+	}
+}
+
+/*
+ *special case for inserting one row of String (TEXT) value
+ *
+ *params
+ *deviceId: string, time series path for device
+ *measurements: []string, sensor names
+ *values: []string, values to be inserted, for each sensor
+ *timestamp: int64, indicate the timestamp of the row of data
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertStringRecord(deviceId string, measurements []string, values []string, timestamp int64) {
+	request := rpc.TSInsertStringRecordReq{SessionId: s.sessionId, DeviceId: deviceId, Measurements: measurements,
+		Values: values, Timestamp: timestamp}
+	status, err := s.client.InsertStringRecord(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert one string record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one string record success")
+	}
+}
+
+func (s *Session) TestInsertStringRecord(deviceId string, measurements []string, values []string, timestamp int64) {
+	request := rpc.TSInsertStringRecordReq{SessionId: s.sessionId, DeviceId: deviceId, Measurements: measurements,
+		Values: values, Timestamp: timestamp}
+	status, err := s.client.TestInsertStringRecord(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert one string record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one string record success")
+	}
+}
+
+/*
+ *special case for inserting multiple rows of String (TEXT) value
+ *
+ *params
+ *deviceIds: []string, time series paths for device
+ *measurements: [][]string, each element of outer list indicates measurements of a device
+ *values: [][]interface{}, values to be inserted, for each device
+ *timestamps: []int64, timestamps for records
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertStringRecords(deviceIds []string, measurements [][]string, values [][]string,
+	timestamps []int64) {
+	request := rpc.TSInsertStringRecordsReq{SessionId: s.sessionId, DeviceIds: deviceIds, MeasurementsList: measurements,
+		ValuesList: values, Timestamps: timestamps}
+	s.dv = strings.Replace(strings.Trim(fmt.Sprint(deviceIds), "[]"), " ", ",", -1)
+	status, err := s.client.InsertStringRecords(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert multi string records failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   s.dv,
+			"code": status.Code,
+		}).Info("insert multi string records success")
+	}
+}
+
+func (s *Session) TestInsertStringRecords(deviceIds []string, measurements [][]string, values [][]string,
+	timestamps []int64) {
+	request := rpc.TSInsertStringRecordsReq{SessionId: s.sessionId, DeviceIds: deviceIds, MeasurementsList: measurements,
+		ValuesList: values, Timestamps: timestamps}
+	s.dv = strings.Replace(strings.Trim(fmt.Sprint(deviceIds), "[]"), " ", ",", -1)
+	status, err := s.client.TestInsertStringRecords(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert multi string records failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   s.dv,
+			"code": status.Code,
+		}).Info("insert multi string records success")
+	}
+}
+
+/*
+ *insert one row of record into database, if you want improve your performance, please use insertTablet method
+ *
+ *params
+ *deviceId: string, time series path for device
+ *measurements: []string, sensor names
+ *dataTypes: []int32, list of dataType, indicate the data type for each sensor
+ *values: []interface{}, values to be inserted, for each sensor
+ *timestamp: int64, indicate the timestamp of the row of data
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertRecord(deviceId string, measurements []string, dataTypes []int16, values []interface{},
+	timestamp int64) {
+	request := s.genInsertRecordReq(deviceId, measurements, dataTypes, values, timestamp)
+	status, err := s.client.InsertRecord(context.Background(), request)
+	if err != nil {
+		Log.WithError(err).Error("insert one record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one record success")
+	}
+}
+
+func (s *Session) TestInsertRecord(deviceId string, measurements []string, dataTypes []int16, values []interface{},
+	timestamp int64) {
+	request := s.genInsertRecordReq(deviceId, measurements, dataTypes, values, timestamp)
+	status, err := s.client.TestInsertRecord(context.Background(), request)
+	if err != nil {
+		Log.WithError(err).Error("insert one record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one record success")
+	}
+}
+
+func (s *Session) genInsertRecordReq(deviceId string, measurements []string, dataTypes []int16, values []interface{},
+	timestamp int64) *rpc.TSInsertRecordReq {
+	request := rpc.TSInsertRecordReq{SessionId: s.sessionId, DeviceId: deviceId, Measurements: measurements,
+		Timestamp: timestamp}
+	request.Values = valuesToBytes(dataTypes, values)
+	return &request
+}
+
+/*
+ *insert multiple rows of data, records are independent to each other, in other words, there's no relationship
+ *between those records
+ *
+ *params
+ *deviceIds: []string, time series paths for device
+ *measurements: [][]string, each element of outer list indicates measurements of a device
+ *dataTypes: [][]int32, each element of outer list indicates sensor data types of a device
+ *values: [][]interface{}, values to be inserted, for each device
+ *timestamps: []int64, timestamps for records
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertRecords(deviceIds []string, measurements [][]string, dataTypes [][]int16, values [][]interface{},
+	timestamps []int64) {
+	s.dv = strings.Replace(strings.Trim(fmt.Sprint(deviceIds), "[]"), " ", ",", -1)

Review comment:
       then you can do it if the log debug level is enabled, if the log level is info,  for every operation the code will be called once, it's 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] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r524232724



##########
File path: client-go/src/main/iotdb_session/session.go
##########
@@ -0,0 +1,290 @@
+/**
+ * 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 iotdbSession
+
+import (
+	"context"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"log"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var client *rpc.TSIServiceClient

Review comment:
       I think what you said is very good




----------------------------------------------------------------
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] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r524231278



##########
File path: client-go/src/main/iotdb_session/utils/field.go
##########
@@ -0,0 +1,122 @@
+/**
+ * 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 utils
+
+import (
+	"encoding/json"
+)
+
+type Field struct {
+	DataType int32
+}
+
+var boolV bool

Review comment:
       Thank you for your guidance




----------------------------------------------------------------
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] [iotdb] neuyilan commented on a change in pull request #2046: client-go

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



##########
File path: client-go/README.md
##########
@@ -0,0 +1,304 @@
+<!--
+
+    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.
+
+-->
+
+To use go client, add "import github.com/yanhongwangg/incubator-iotdb/client-go" 
+
+## Requirement
+* GoLang 1.14+
+
+## Getting started
+*Create project eg:session
+
+*Enter project directory eg:cd session
+
+*Download dependency : go get github.com/yanhongwangg/incubator-iotdb@client-go
+
+*eg:session_example.go
+
+```
+package main
+
+import (

Review comment:
       if those codes same as the session_example.go, I think we can just add the session_example.go link here.

##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,698 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var Log = logrus.New()
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)
+	logFile, _ := os.OpenFile(dir+"\\logs\\all.log", os.O_CREATE|os.O_RDWR|os.O_APPEND, 0666)
+	Log.SetOutput(logFile)
+	var protocolFactory thrift.TProtocolFactory
+	s.trans, s.err = thrift.NewTSocketTimeout(net.JoinHostPort(s.Host, s.Port), time.Duration(connectionTimeoutInMs))
+	if s.err != nil {
+		Log.WithError(s.err).Error("connect failed")
+	}
+	s.trans = thrift.NewTFramedTransport(s.trans)
+	if !s.trans.IsOpen() {
+		s.err = s.trans.Open()
+		if s.err != nil {
+			Log.WithError(s.err).Error("open the conn failed")
+		}
+	}
+	if enableRPCCompression {
+		protocolFactory = thrift.NewTCompactProtocolFactory()
+	} else {
+		protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
+	}
+	iProtocol := protocolFactory.GetProtocol(s.trans)
+	oProtocol := protocolFactory.GetProtocol(s.trans)
+	s.client = rpc.NewTSIServiceClient(thrift.NewTStandardClient(iProtocol, oProtocol))
+	s.ZoneId = DefaultZoneId
+	tSOpenSessionReq := rpc.TSOpenSessionReq{ClientProtocol: protocolVersion, ZoneId: s.ZoneId, Username: &s.User,
+		Password: &s.Passwd}
+	tSOpenSessionResp, err := s.client.OpenSession(context.Background(), &tSOpenSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("open session failed")
+	} else {
+		Log.WithField("code", tSOpenSessionResp.GetStatus().Code).Info("open session success")
+	}
+	s.sessionId = tSOpenSessionResp.GetSessionId()
+	s.requestStatementId, err = s.client.RequestStatementId(context.Background(), s.sessionId)
+	s.FetchSize = DefaultFetchSize
+	if err != nil {
+		Log.WithError(err).Error("request StatementId failed")
+	}
+	if s.ZoneId != "" {
+		s.SetTimeZone(s.ZoneId)
+	} else {
+		s.ZoneId = s.GetTimeZone()
+	}
+}
+
+func (s *Session) CheckTimeseriesExists(path string) bool {
+	dataSet := s.ExecuteQueryStatement("SHOW TIMESERIES " + path)
+	result := dataSet.HasNext()
+	dataSet.CloseOperationHandle()
+	return result
+}
+
+func (s *Session) Close() error {
+	tSCloseSessionReq := rpc.NewTSCloseSessionReq()
+	tSCloseSessionReq.SessionId = s.sessionId
+	status, err := s.client.CloseSession(context.Background(), tSCloseSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("Error occurs when closing session at server. Maybe server is down. Error message")

Review comment:
       please make the error message more clear.

##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,698 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var Log = logrus.New()
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)
+	logFile, _ := os.OpenFile(dir+"\\logs\\all.log", os.O_CREATE|os.O_RDWR|os.O_APPEND, 0666)
+	Log.SetOutput(logFile)
+	var protocolFactory thrift.TProtocolFactory
+	s.trans, s.err = thrift.NewTSocketTimeout(net.JoinHostPort(s.Host, s.Port), time.Duration(connectionTimeoutInMs))
+	if s.err != nil {
+		Log.WithError(s.err).Error("connect failed")
+	}
+	s.trans = thrift.NewTFramedTransport(s.trans)
+	if !s.trans.IsOpen() {
+		s.err = s.trans.Open()
+		if s.err != nil {
+			Log.WithError(s.err).Error("open the conn failed")
+		}
+	}
+	if enableRPCCompression {
+		protocolFactory = thrift.NewTCompactProtocolFactory()
+	} else {
+		protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
+	}
+	iProtocol := protocolFactory.GetProtocol(s.trans)
+	oProtocol := protocolFactory.GetProtocol(s.trans)
+	s.client = rpc.NewTSIServiceClient(thrift.NewTStandardClient(iProtocol, oProtocol))
+	s.ZoneId = DefaultZoneId
+	tSOpenSessionReq := rpc.TSOpenSessionReq{ClientProtocol: protocolVersion, ZoneId: s.ZoneId, Username: &s.User,
+		Password: &s.Passwd}
+	tSOpenSessionResp, err := s.client.OpenSession(context.Background(), &tSOpenSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("open session failed")
+	} else {
+		Log.WithField("code", tSOpenSessionResp.GetStatus().Code).Info("open session success")
+	}
+	s.sessionId = tSOpenSessionResp.GetSessionId()
+	s.requestStatementId, err = s.client.RequestStatementId(context.Background(), s.sessionId)
+	s.FetchSize = DefaultFetchSize
+	if err != nil {
+		Log.WithError(err).Error("request StatementId failed")
+	}
+	if s.ZoneId != "" {
+		s.SetTimeZone(s.ZoneId)
+	} else {
+		s.ZoneId = s.GetTimeZone()
+	}
+}
+
+func (s *Session) CheckTimeseriesExists(path string) bool {
+	dataSet := s.ExecuteQueryStatement("SHOW TIMESERIES " + path)
+	result := dataSet.HasNext()
+	dataSet.CloseOperationHandle()
+	return result
+}
+
+func (s *Session) Close() error {
+	tSCloseSessionReq := rpc.NewTSCloseSessionReq()
+	tSCloseSessionReq.SessionId = s.sessionId
+	status, err := s.client.CloseSession(context.Background(), tSCloseSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("Error occurs when closing session at server. Maybe server is down. Error message")
+		return err
+	} else {
+		Log.WithField("code", status.Code).Info("close session success")
+	}
+	s.trans.Close()
+	return nil
+}
+
+/*
+ *set one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) SetStorageGroup(storageGroupId string) error {
+	status, err := s.client.SetStorageGroup(context.Background(), s.sessionId, storageGroupId)
+	if err != nil {
+		Log.WithError(err).Error("setting storage group failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Info("setting storage group success")
+	}
+	return err
+}
+
+/*
+ *delete one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteStorageGroup(storageGroupId string) {

Review comment:
       the method no return value, the annotation should be consistent with the codes,  the same for other methods

##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,698 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var Log = logrus.New()
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)
+	logFile, _ := os.OpenFile(dir+"\\logs\\all.log", os.O_CREATE|os.O_RDWR|os.O_APPEND, 0666)
+	Log.SetOutput(logFile)
+	var protocolFactory thrift.TProtocolFactory
+	s.trans, s.err = thrift.NewTSocketTimeout(net.JoinHostPort(s.Host, s.Port), time.Duration(connectionTimeoutInMs))
+	if s.err != nil {
+		Log.WithError(s.err).Error("connect failed")
+	}
+	s.trans = thrift.NewTFramedTransport(s.trans)
+	if !s.trans.IsOpen() {
+		s.err = s.trans.Open()
+		if s.err != nil {
+			Log.WithError(s.err).Error("open the conn failed")
+		}
+	}
+	if enableRPCCompression {
+		protocolFactory = thrift.NewTCompactProtocolFactory()
+	} else {
+		protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
+	}
+	iProtocol := protocolFactory.GetProtocol(s.trans)
+	oProtocol := protocolFactory.GetProtocol(s.trans)
+	s.client = rpc.NewTSIServiceClient(thrift.NewTStandardClient(iProtocol, oProtocol))
+	s.ZoneId = DefaultZoneId
+	tSOpenSessionReq := rpc.TSOpenSessionReq{ClientProtocol: protocolVersion, ZoneId: s.ZoneId, Username: &s.User,
+		Password: &s.Passwd}
+	tSOpenSessionResp, err := s.client.OpenSession(context.Background(), &tSOpenSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("open session failed")
+	} else {
+		Log.WithField("code", tSOpenSessionResp.GetStatus().Code).Info("open session success")
+	}
+	s.sessionId = tSOpenSessionResp.GetSessionId()
+	s.requestStatementId, err = s.client.RequestStatementId(context.Background(), s.sessionId)
+	s.FetchSize = DefaultFetchSize
+	if err != nil {
+		Log.WithError(err).Error("request StatementId failed")
+	}
+	if s.ZoneId != "" {
+		s.SetTimeZone(s.ZoneId)
+	} else {
+		s.ZoneId = s.GetTimeZone()
+	}
+}
+
+func (s *Session) CheckTimeseriesExists(path string) bool {
+	dataSet := s.ExecuteQueryStatement("SHOW TIMESERIES " + path)
+	result := dataSet.HasNext()
+	dataSet.CloseOperationHandle()
+	return result
+}
+
+func (s *Session) Close() error {
+	tSCloseSessionReq := rpc.NewTSCloseSessionReq()
+	tSCloseSessionReq.SessionId = s.sessionId
+	status, err := s.client.CloseSession(context.Background(), tSCloseSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("Error occurs when closing session at server. Maybe server is down. Error message")
+		return err
+	} else {
+		Log.WithField("code", status.Code).Info("close session success")
+	}
+	s.trans.Close()
+	return nil
+}
+
+/*
+ *set one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) SetStorageGroup(storageGroupId string) error {
+	status, err := s.client.SetStorageGroup(context.Background(), s.sessionId, storageGroupId)
+	if err != nil {
+		Log.WithError(err).Error("setting storage group failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Info("setting storage group success")

Review comment:
       I think if the operation succeeds, we do not need to print the info message, as this will print so many logs for users who use the go-client. I think the debug message is ok. and the default log level is info.

##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,698 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var Log = logrus.New()
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)
+	logFile, _ := os.OpenFile(dir+"\\logs\\all.log", os.O_CREATE|os.O_RDWR|os.O_APPEND, 0666)
+	Log.SetOutput(logFile)
+	var protocolFactory thrift.TProtocolFactory
+	s.trans, s.err = thrift.NewTSocketTimeout(net.JoinHostPort(s.Host, s.Port), time.Duration(connectionTimeoutInMs))
+	if s.err != nil {
+		Log.WithError(s.err).Error("connect failed")
+	}
+	s.trans = thrift.NewTFramedTransport(s.trans)
+	if !s.trans.IsOpen() {
+		s.err = s.trans.Open()
+		if s.err != nil {
+			Log.WithError(s.err).Error("open the conn failed")
+		}
+	}
+	if enableRPCCompression {
+		protocolFactory = thrift.NewTCompactProtocolFactory()
+	} else {
+		protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
+	}
+	iProtocol := protocolFactory.GetProtocol(s.trans)
+	oProtocol := protocolFactory.GetProtocol(s.trans)
+	s.client = rpc.NewTSIServiceClient(thrift.NewTStandardClient(iProtocol, oProtocol))
+	s.ZoneId = DefaultZoneId
+	tSOpenSessionReq := rpc.TSOpenSessionReq{ClientProtocol: protocolVersion, ZoneId: s.ZoneId, Username: &s.User,
+		Password: &s.Passwd}
+	tSOpenSessionResp, err := s.client.OpenSession(context.Background(), &tSOpenSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("open session failed")
+	} else {
+		Log.WithField("code", tSOpenSessionResp.GetStatus().Code).Info("open session success")
+	}
+	s.sessionId = tSOpenSessionResp.GetSessionId()
+	s.requestStatementId, err = s.client.RequestStatementId(context.Background(), s.sessionId)
+	s.FetchSize = DefaultFetchSize
+	if err != nil {
+		Log.WithError(err).Error("request StatementId failed")
+	}
+	if s.ZoneId != "" {
+		s.SetTimeZone(s.ZoneId)
+	} else {
+		s.ZoneId = s.GetTimeZone()
+	}
+}
+
+func (s *Session) CheckTimeseriesExists(path string) bool {
+	dataSet := s.ExecuteQueryStatement("SHOW TIMESERIES " + path)
+	result := dataSet.HasNext()
+	dataSet.CloseOperationHandle()
+	return result
+}
+
+func (s *Session) Close() error {
+	tSCloseSessionReq := rpc.NewTSCloseSessionReq()
+	tSCloseSessionReq.SessionId = s.sessionId
+	status, err := s.client.CloseSession(context.Background(), tSCloseSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("Error occurs when closing session at server. Maybe server is down. Error message")
+		return err
+	} else {
+		Log.WithField("code", status.Code).Info("close session success")
+	}
+	s.trans.Close()
+	return nil
+}
+
+/*
+ *set one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) SetStorageGroup(storageGroupId string) error {
+	status, err := s.client.SetStorageGroup(context.Background(), s.sessionId, storageGroupId)
+	if err != nil {
+		Log.WithError(err).Error("setting storage group failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Info("setting storage group success")
+	}
+	return err
+}
+
+/*
+ *delete one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteStorageGroup(storageGroupId string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, []string{storageGroupId})
+	if err != nil {
+		Log.WithError(err).Error("delete storage group failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Info("delete storage group success")
+	}
+}
+
+/*
+ *delete multiple storage group
+ *
+ *param
+ *storageGroupIds: []string, paths of the target storage groups
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteStorageGroups(storageGroupIds []string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, storageGroupIds)
+	s.sg = strings.Replace(strings.Trim(fmt.Sprint(storageGroupIds), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("delete storage groups failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   s.sg,
+			"code": status.Code,
+		}).Info("delete storage groups success")
+	}
+}
+
+/*
+ *create single time series
+ *
+ *params
+ *path: string, complete time series path (starts from root)
+ *dataType: int32, data type for this time series
+ *encoding: int32, data type for this time series
+ *compressor: int32, compressing type for this time series
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) CreateTimeseries(path string, dataType int32, encoding int32, compressor int32) {
+	request := rpc.TSCreateTimeseriesReq{SessionId: s.sessionId, Path: path, DataType: dataType, Encoding: encoding,
+		Compressor: compressor}
+	status, err := s.client.CreateTimeseries(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("creating time series failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   path,
+			"code": status.Code,
+		}).Info("creating time series success")
+	}
+}
+
+/*
+ *create multiple time series
+ *
+ *params
+ *paths: []string, complete time series paths (starts from root)
+ *dataTypes: []int32, data types for time series
+ *encodings: []int32, encodings for time series
+ *compressors: []int32, compressing types for time series
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) CreateMultiTimeseries(paths []string, dataTypes []int32, encodings []int32, compressors []int32) {
+	request := rpc.TSCreateMultiTimeseriesReq{SessionId: s.sessionId, Paths: paths, DataTypes: dataTypes,
+		Encodings: encodings, Compressors: compressors}
+	status, err := s.client.CreateMultiTimeseries(context.Background(), &request)
+	s.ts = strings.Replace(strings.Trim(fmt.Sprint(paths), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("creating multi time series failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   s.ts,
+			"code": status.Code,
+		}).Info("creating multi time series success")
+	}
+}
+
+/*
+ *delete multiple time series, including data and schema
+ *
+ *params
+ *paths: []string, time series paths, which should be complete (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteTimeseries(paths []string) {
+	status, err := s.client.DeleteTimeseries(context.Background(), s.sessionId, paths)
+	var ts = strings.Replace(strings.Trim(fmt.Sprint(paths), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("delete time series failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   ts,
+			"code": status.Code,
+		}).Info("delete time series success")
+	}
+}
+
+/*
+ *delete all startTime <= data <= endTime in multiple time series
+ *
+ *params
+ *paths: []string, time series array that the data in
+ *startTime: int64, start time of deletion range
+ *endTime: int64, end time of deletion range
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteData(paths []string, startTime int64, endTime int64) {
+	request := rpc.TSDeleteDataReq{SessionId: s.sessionId, Paths: paths, StartTime: startTime, EndTime: endTime}
+	status, err := s.client.DeleteData(context.Background(), &request)
+	s.ts = strings.Replace(strings.Trim(fmt.Sprint(paths), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("delete data failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   s.ts,
+			"code": status.Code,
+		}).Info("delete data success")
+	}
+}
+
+/*
+ *special case for inserting one row of String (TEXT) value
+ *
+ *params
+ *deviceId: string, time series path for device
+ *measurements: []string, sensor names
+ *values: []string, values to be inserted, for each sensor
+ *timestamp: int64, indicate the timestamp of the row of data
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertStringRecord(deviceId string, measurements []string, values []string, timestamp int64) {
+	request := rpc.TSInsertStringRecordReq{SessionId: s.sessionId, DeviceId: deviceId, Measurements: measurements,
+		Values: values, Timestamp: timestamp}
+	status, err := s.client.InsertStringRecord(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert one string record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one string record success")
+	}
+}
+
+func (s *Session) TestInsertStringRecord(deviceId string, measurements []string, values []string, timestamp int64) {
+	request := rpc.TSInsertStringRecordReq{SessionId: s.sessionId, DeviceId: deviceId, Measurements: measurements,
+		Values: values, Timestamp: timestamp}
+	status, err := s.client.TestInsertStringRecord(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert one string record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one string record success")
+	}
+}
+
+/*
+ *special case for inserting multiple rows of String (TEXT) value
+ *
+ *params
+ *deviceIds: []string, time series paths for device
+ *measurements: [][]string, each element of outer list indicates measurements of a device
+ *values: [][]interface{}, values to be inserted, for each device
+ *timestamps: []int64, timestamps for records
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertStringRecords(deviceIds []string, measurements [][]string, values [][]string,
+	timestamps []int64) {
+	request := rpc.TSInsertStringRecordsReq{SessionId: s.sessionId, DeviceIds: deviceIds, MeasurementsList: measurements,
+		ValuesList: values, Timestamps: timestamps}
+	s.dv = strings.Replace(strings.Trim(fmt.Sprint(deviceIds), "[]"), " ", ",", -1)
+	status, err := s.client.InsertStringRecords(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert multi string records failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   s.dv,
+			"code": status.Code,
+		}).Info("insert multi string records success")
+	}
+}
+
+func (s *Session) TestInsertStringRecords(deviceIds []string, measurements [][]string, values [][]string,
+	timestamps []int64) {
+	request := rpc.TSInsertStringRecordsReq{SessionId: s.sessionId, DeviceIds: deviceIds, MeasurementsList: measurements,
+		ValuesList: values, Timestamps: timestamps}
+	s.dv = strings.Replace(strings.Trim(fmt.Sprint(deviceIds), "[]"), " ", ",", -1)
+	status, err := s.client.TestInsertStringRecords(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert multi string records failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   s.dv,
+			"code": status.Code,
+		}).Info("insert multi string records success")
+	}
+}
+
+/*
+ *insert one row of record into database, if you want improve your performance, please use insertTablet method
+ *
+ *params
+ *deviceId: string, time series path for device
+ *measurements: []string, sensor names
+ *dataTypes: []int32, list of dataType, indicate the data type for each sensor
+ *values: []interface{}, values to be inserted, for each sensor
+ *timestamp: int64, indicate the timestamp of the row of data
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertRecord(deviceId string, measurements []string, dataTypes []int16, values []interface{},
+	timestamp int64) {
+	request := s.genInsertRecordReq(deviceId, measurements, dataTypes, values, timestamp)
+	status, err := s.client.InsertRecord(context.Background(), request)
+	if err != nil {
+		Log.WithError(err).Error("insert one record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one record success")
+	}
+}
+
+func (s *Session) TestInsertRecord(deviceId string, measurements []string, dataTypes []int16, values []interface{},
+	timestamp int64) {
+	request := s.genInsertRecordReq(deviceId, measurements, dataTypes, values, timestamp)
+	status, err := s.client.TestInsertRecord(context.Background(), request)
+	if err != nil {
+		Log.WithError(err).Error("insert one record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one record success")
+	}
+}
+
+func (s *Session) genInsertRecordReq(deviceId string, measurements []string, dataTypes []int16, values []interface{},
+	timestamp int64) *rpc.TSInsertRecordReq {
+	request := rpc.TSInsertRecordReq{SessionId: s.sessionId, DeviceId: deviceId, Measurements: measurements,
+		Timestamp: timestamp}
+	request.Values = valuesToBytes(dataTypes, values)
+	return &request
+}
+
+/*
+ *insert multiple rows of data, records are independent to each other, in other words, there's no relationship
+ *between those records
+ *
+ *params
+ *deviceIds: []string, time series paths for device
+ *measurements: [][]string, each element of outer list indicates measurements of a device
+ *dataTypes: [][]int32, each element of outer list indicates sensor data types of a device
+ *values: [][]interface{}, values to be inserted, for each device
+ *timestamps: []int64, timestamps for records
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertRecords(deviceIds []string, measurements [][]string, dataTypes [][]int16, values [][]interface{},
+	timestamps []int64) {
+	s.dv = strings.Replace(strings.Trim(fmt.Sprint(deviceIds), "[]"), " ", ",", -1)

Review comment:
       for every inset operation, this may time-consume, If you want to print the devices,  I think the `deviceIds[]` is just enough.




----------------------------------------------------------------
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] [iotdb] JackieTien97 closed pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
JackieTien97 closed pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046


   


----------------------------------------------------------------
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] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r528516042



##########
File path: client-go/pom.xml
##########
@@ -0,0 +1,33 @@
+<?xml version="1.0" encoding="UTF-8"?>

Review comment:
       I agree with you




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

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



[GitHub] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r528629868



##########
File path: client-go/README.md
##########
@@ -0,0 +1,304 @@
+<!--
+
+    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.
+
+-->
+
+To use go client, add "import github.com/yanhongwangg/incubator-iotdb/client-go" 
+
+## Requirement
+* GoLang 1.14+
+
+## Getting started
+*Create project eg:session
+
+*Enter project directory eg:cd session
+
+*Download dependency : go get github.com/yanhongwangg/incubator-iotdb@client-go
+
+*eg:session_example.go
+
+```
+package main
+
+import (

Review comment:
       It's exactly the same. I'll post the link




----------------------------------------------------------------
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] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r528515635



##########
File path: client-go/src/main/client/utils/iotdb_constants.go
##########
@@ -0,0 +1,52 @@
+/**
+ * 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 utils
+
+const (
+	BOOLEAN int32 = 0
+	INT32   int32 = 1
+	INT64   int32 = 2
+	FLOAT   int32 = 3
+	DOUBLE  int32 = 4
+	TEXT    int32 = 5
+)
+
+const (
+	PLAIN            int32 = 0
+	PLAIN_DICTIONARY int32 = 1
+	RLE              int32 = 2
+	DIFF             int32 = 3
+	TS_2DIFF         int32 = 4
+	BITMAP           int32 = 5
+	GORILLA_V1       int32 = 6
+	REGULAR          int32 = 7
+	GORILLA          int32 = 8
+)
+
+const (
+	UNCOMPRESSED int32 = 0
+	SNAPPY       int32 = 1
+	GZIP         int32 = 2
+	LZO          int32 = 3
+	SDT          int32 = 4
+	PAA          int32 = 5
+	PLA          int32 = 6
+	LZ4          int32 = 7

Review comment:
       Well, you're right




----------------------------------------------------------------
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] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r528515437



##########
File path: client-go/src/main/client/utils/row_record.go
##########
@@ -0,0 +1,68 @@
+/**
+ * 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 utils
+
+import (
+	"strconv"
+)
+
+type RowRecord struct {
+	Timestamp int64
+	Fields    []Field
+}
+
+func (r *RowRecord) AddField(field Field) {
+	r.Fields[len(r.Fields)] = field
+}
+
+/*func (r *RowRecord) String() string {
+	res := strconv.Itoa(int(r.Timestamp))
+	for i := 0; i < len(r.Fields); i++ {
+		res += "\t"
+		if r.Fields[i].DataType == "" {
+			res += "null"
+			continue
+		}
+		objValue := r.Fields[i].GetObjectValue(r.Fields[i].DataType)
+		switch r.Fields[i].DataType {
+		case "BOOLEAN":
+			res += strconv.FormatBool(objValue.(bool))
+			break
+		case "INT32":
+			res += strconv.FormatInt(int64(objValue.(int32)), 10)
+			break
+		case "INT64":
+			res += strconv.FormatInt(objValue.(int64), 10)
+			break
+		case "FLOAT":
+
+			res += strconv.FormatFloat(float64(objValue.(float32)), 'f', 1, 32)
+			break
+		case "DOUBLE":
+			res += strconv.FormatFloat(objValue.(float64), 'f', -1, 64)
+			break
+		case "TEXT":

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] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r524231841



##########
File path: client-go/src/main/iotdb_session/session.go
##########
@@ -0,0 +1,290 @@
+/**
+ * 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 iotdbSession
+
+import (
+	"context"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"log"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var client *rpc.TSIServiceClient
+var sessionId int64
+var isClose bool = true
+var trans thrift.TTransport
+var err error
+var requestStatementId int64
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) error {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)

Review comment:
       good idea!!!




----------------------------------------------------------------
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] [iotdb] JackieTien97 commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r528490927



##########
File path: client-go/pom.xml
##########
@@ -0,0 +1,33 @@
+<?xml version="1.0" encoding="UTF-8"?>

Review comment:
       Maybe we don't need this pom file, because it's actually not a java project.
   
   And I think we will open another repository to store the go-client, as so it will be a pure go project.

##########
File path: client-go/README.md
##########
@@ -0,0 +1,298 @@
+<!--
+
+    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.
+
+-->
+
+To use go client, add "import github.com/yanhongwangg/incubator-iotdb/client-go" 
+
+## Requirement
+* GoLang 1.13+
+
+## Getting started
+*Create project eg:session
+*Enter project directory eg:cd session
+*Download dependency : go get github.com/yanhongwangg/incubator-iotdb@client-go
+*eg:session_example.go
+package main

Review comment:
       it's better to use a code block quote to bracket the ession_example.go

##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,708 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var (
+	client             *rpc.TSIServiceClient
+	sessionId          int64
+	trans              thrift.TTransport
+	err                error
+	requestStatementId int64
+	ts                 string
+	sg                 string
+	dv                 string
+	Log                = logrus.New()
+)

Review comment:
       I think these attributes should be the properties of Session struct.

##########
File path: client-go/src/main/client/utils/iotdb_constants.go
##########
@@ -0,0 +1,52 @@
+/**
+ * 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 utils
+
+const (
+	BOOLEAN int32 = 0
+	INT32   int32 = 1
+	INT64   int32 = 2
+	FLOAT   int32 = 3
+	DOUBLE  int32 = 4
+	TEXT    int32 = 5
+)
+
+const (
+	PLAIN            int32 = 0
+	PLAIN_DICTIONARY int32 = 1
+	RLE              int32 = 2
+	DIFF             int32 = 3
+	TS_2DIFF         int32 = 4
+	BITMAP           int32 = 5
+	GORILLA_V1       int32 = 6
+	REGULAR          int32 = 7
+	GORILLA          int32 = 8
+)
+
+const (
+	UNCOMPRESSED int32 = 0
+	SNAPPY       int32 = 1
+	GZIP         int32 = 2
+	LZO          int32 = 3
+	SDT          int32 = 4
+	PAA          int32 = 5
+	PLA          int32 = 6
+	LZ4          int32 = 7

Review comment:
       I think byte is enough.

##########
File path: client-go/src/main/client/utils/field.go
##########
@@ -0,0 +1,220 @@
+/**
+ * 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 utils
+
+import (
+	"bytes"
+	"encoding/binary"
+	log "github.com/sirupsen/logrus"
+)
+
+type Field struct {
+	DataType string
+}
+
+var (
+	boolV   bool
+	intV    int32
+	longV   int64
+	floatV  float32
+	doubleV float64
+	binaryV []byte
+)

Review comment:
       Same as above, I think these properties should be defined in the Field struct

##########
File path: client-go/README.md
##########
@@ -0,0 +1,298 @@
+<!--
+
+    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.
+
+-->
+
+To use go client, add "import github.com/yanhongwangg/incubator-iotdb/client-go" 
+
+## Requirement
+* GoLang 1.13+

Review comment:
       Maybe 1.14+, because you write `go 1.14` in your go.mod file.

##########
File path: client-go/src/main/client/utils/row_record.go
##########
@@ -0,0 +1,68 @@
+/**
+ * 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 utils
+
+import (
+	"strconv"

Review comment:
       It seems that this package is not used, don't forget to delete it.

##########
File path: client-go/src/main/client/utils/row_record.go
##########
@@ -0,0 +1,68 @@
+/**
+ * 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 utils
+
+import (
+	"strconv"
+)
+
+type RowRecord struct {
+	Timestamp int64
+	Fields    []Field
+}
+
+func (r *RowRecord) AddField(field Field) {
+	r.Fields[len(r.Fields)] = field
+}
+
+/*func (r *RowRecord) String() string {
+	res := strconv.Itoa(int(r.Timestamp))
+	for i := 0; i < len(r.Fields); i++ {
+		res += "\t"
+		if r.Fields[i].DataType == "" {
+			res += "null"
+			continue
+		}
+		objValue := r.Fields[i].GetObjectValue(r.Fields[i].DataType)
+		switch r.Fields[i].DataType {
+		case "BOOLEAN":
+			res += strconv.FormatBool(objValue.(bool))
+			break
+		case "INT32":
+			res += strconv.FormatInt(int64(objValue.(int32)), 10)
+			break
+		case "INT64":
+			res += strconv.FormatInt(objValue.(int64), 10)
+			break
+		case "FLOAT":
+
+			res += strconv.FormatFloat(float64(objValue.(float32)), 'f', 1, 32)
+			break
+		case "DOUBLE":
+			res += strconv.FormatFloat(objValue.(float64), 'f', -1, 64)
+			break
+		case "TEXT":

Review comment:
       delete it




----------------------------------------------------------------
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] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r530095804



##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,667 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	log "github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)
+	logFile, _ := os.OpenFile(dir+"\\logs\\all.log", os.O_CREATE|os.O_RDWR|os.O_APPEND, 0666)
+	log.SetOutput(logFile)
+	log.SetLevel(log.InfoLevel)
+	var protocolFactory thrift.TProtocolFactory
+	s.trans, s.err = thrift.NewTSocketTimeout(net.JoinHostPort(s.Host, s.Port), time.Duration(connectionTimeoutInMs))
+	if s.err != nil {
+		log.WithError(s.err).Error("connect failed")
+	}
+	s.trans = thrift.NewTFramedTransport(s.trans)
+	if !s.trans.IsOpen() {
+		s.err = s.trans.Open()
+		if s.err != nil {
+			log.WithError(s.err).Error("open the conn failed")
+		}
+	}
+	if enableRPCCompression {
+		protocolFactory = thrift.NewTCompactProtocolFactory()
+	} else {
+		protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
+	}
+	iProtocol := protocolFactory.GetProtocol(s.trans)
+	oProtocol := protocolFactory.GetProtocol(s.trans)
+	s.client = rpc.NewTSIServiceClient(thrift.NewTStandardClient(iProtocol, oProtocol))
+	s.ZoneId = DefaultZoneId
+	tSOpenSessionReq := rpc.TSOpenSessionReq{ClientProtocol: protocolVersion, ZoneId: s.ZoneId, Username: &s.User,
+		Password: &s.Passwd}
+	tSOpenSessionResp, err := s.client.OpenSession(context.Background(), &tSOpenSessionReq)
+	if err != nil {
+		log.WithError(err).Error("open session failed")
+	} else {
+		log.WithField("code", tSOpenSessionResp.GetStatus().Code).Debug("open session success")
+	}
+	s.sessionId = tSOpenSessionResp.GetSessionId()
+	s.requestStatementId, err = s.client.RequestStatementId(context.Background(), s.sessionId)
+	s.FetchSize = DefaultFetchSize
+	if err != nil {
+		log.WithError(err).Error("request StatementId failed")
+	}
+	if s.ZoneId != "" {
+		s.SetTimeZone(s.ZoneId)
+	} else {
+		s.ZoneId = s.GetTimeZone()
+	}
+}
+
+func (s *Session) CheckTimeseriesExists(path string) bool {
+	dataSet := s.ExecuteQueryStatement("SHOW TIMESERIES " + path)
+	result := dataSet.HasNext()
+	dataSet.CloseOperationHandle()
+	return result
+}
+
+func (s *Session) Close() {
+	tSCloseSessionReq := rpc.NewTSCloseSessionReq()
+	tSCloseSessionReq.SessionId = s.sessionId
+	status, err := s.client.CloseSession(context.Background(), tSCloseSessionReq)
+	if err != nil {
+		log.WithError(err).Error("Error occurs when closing session at server. Maybe server is down.")
+	} else {
+		log.WithField("code", status.Code).Debug("close session success")
+	}
+	s.trans.Close()
+}
+
+/*
+ *set one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ */
+func (s *Session) SetStorageGroup(storageGroupId string) {
+	status, err := s.client.SetStorageGroup(context.Background(), s.sessionId, storageGroupId)
+	if err != nil {
+		log.WithError(err).Error("setting storage group failed")
+	} else {
+		log.WithFields(log.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Debug("setting storage group success")
+	}
+}
+
+/*
+ *delete one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ */
+func (s *Session) DeleteStorageGroup(storageGroupId string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, []string{storageGroupId})
+	if err != nil {
+		log.WithError(err).Error("delete storage group failed")
+	} else {
+		log.WithFields(log.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Debug("delete storage group success")
+	}
+}
+
+/*
+ *delete multiple storage group
+ *
+ *param
+ *storageGroupIds: []string, paths of the target storage groups
+ *
+ */
+func (s *Session) DeleteStorageGroups(storageGroupIds []string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, storageGroupIds)
+	s.sg = strings.Replace(strings.Trim(fmt.Sprint(storageGroupIds), "[]"), " ", ",", -1)

Review comment:
       thanks for your guidance




----------------------------------------------------------------
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] [iotdb] yanhongwangg commented on a change in pull request #2046: client-go

Posted by GitBox <gi...@apache.org>.
yanhongwangg commented on a change in pull request #2046:
URL: https://github.com/apache/iotdb/pull/2046#discussion_r528630522



##########
File path: client-go/src/main/client/session.go
##########
@@ -0,0 +1,698 @@
+/**
+ * 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"fmt"
+	"github.com/apache/thrift/lib/go/thrift"
+	"github.com/sirupsen/logrus"
+	"github.com/yanhongwangg/go-thrift/rpc"
+	"github.com/yanhongwangg/incubator-iotdb/client-go/src/main/client/utils"
+	"net"
+	"os"
+	"strings"
+	"time"
+)
+
+const protocolVersion = rpc.TSProtocolVersion_IOTDB_SERVICE_PROTOCOL_V3
+
+var Log = logrus.New()
+
+func (s *Session) Open(enableRPCCompression bool, connectionTimeoutInMs int) {
+	dir, _ := os.Getwd()
+	os.Mkdir(dir+"\\logs", os.ModePerm)
+	logFile, _ := os.OpenFile(dir+"\\logs\\all.log", os.O_CREATE|os.O_RDWR|os.O_APPEND, 0666)
+	Log.SetOutput(logFile)
+	var protocolFactory thrift.TProtocolFactory
+	s.trans, s.err = thrift.NewTSocketTimeout(net.JoinHostPort(s.Host, s.Port), time.Duration(connectionTimeoutInMs))
+	if s.err != nil {
+		Log.WithError(s.err).Error("connect failed")
+	}
+	s.trans = thrift.NewTFramedTransport(s.trans)
+	if !s.trans.IsOpen() {
+		s.err = s.trans.Open()
+		if s.err != nil {
+			Log.WithError(s.err).Error("open the conn failed")
+		}
+	}
+	if enableRPCCompression {
+		protocolFactory = thrift.NewTCompactProtocolFactory()
+	} else {
+		protocolFactory = thrift.NewTBinaryProtocolFactoryDefault()
+	}
+	iProtocol := protocolFactory.GetProtocol(s.trans)
+	oProtocol := protocolFactory.GetProtocol(s.trans)
+	s.client = rpc.NewTSIServiceClient(thrift.NewTStandardClient(iProtocol, oProtocol))
+	s.ZoneId = DefaultZoneId
+	tSOpenSessionReq := rpc.TSOpenSessionReq{ClientProtocol: protocolVersion, ZoneId: s.ZoneId, Username: &s.User,
+		Password: &s.Passwd}
+	tSOpenSessionResp, err := s.client.OpenSession(context.Background(), &tSOpenSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("open session failed")
+	} else {
+		Log.WithField("code", tSOpenSessionResp.GetStatus().Code).Info("open session success")
+	}
+	s.sessionId = tSOpenSessionResp.GetSessionId()
+	s.requestStatementId, err = s.client.RequestStatementId(context.Background(), s.sessionId)
+	s.FetchSize = DefaultFetchSize
+	if err != nil {
+		Log.WithError(err).Error("request StatementId failed")
+	}
+	if s.ZoneId != "" {
+		s.SetTimeZone(s.ZoneId)
+	} else {
+		s.ZoneId = s.GetTimeZone()
+	}
+}
+
+func (s *Session) CheckTimeseriesExists(path string) bool {
+	dataSet := s.ExecuteQueryStatement("SHOW TIMESERIES " + path)
+	result := dataSet.HasNext()
+	dataSet.CloseOperationHandle()
+	return result
+}
+
+func (s *Session) Close() error {
+	tSCloseSessionReq := rpc.NewTSCloseSessionReq()
+	tSCloseSessionReq.SessionId = s.sessionId
+	status, err := s.client.CloseSession(context.Background(), tSCloseSessionReq)
+	if err != nil {
+		Log.WithError(err).Error("Error occurs when closing session at server. Maybe server is down. Error message")
+		return err
+	} else {
+		Log.WithField("code", status.Code).Info("close session success")
+	}
+	s.trans.Close()
+	return nil
+}
+
+/*
+ *set one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) SetStorageGroup(storageGroupId string) error {
+	status, err := s.client.SetStorageGroup(context.Background(), s.sessionId, storageGroupId)
+	if err != nil {
+		Log.WithError(err).Error("setting storage group failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Info("setting storage group success")
+	}
+	return err
+}
+
+/*
+ *delete one storage group
+ *
+ *param
+ *storageGroupId: string, storage group name (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteStorageGroup(storageGroupId string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, []string{storageGroupId})
+	if err != nil {
+		Log.WithError(err).Error("delete storage group failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   storageGroupId,
+			"code": status.Code,
+		}).Info("delete storage group success")
+	}
+}
+
+/*
+ *delete multiple storage group
+ *
+ *param
+ *storageGroupIds: []string, paths of the target storage groups
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteStorageGroups(storageGroupIds []string) {
+	status, err := s.client.DeleteStorageGroups(context.Background(), s.sessionId, storageGroupIds)
+	s.sg = strings.Replace(strings.Trim(fmt.Sprint(storageGroupIds), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("delete storage groups failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"sg":   s.sg,
+			"code": status.Code,
+		}).Info("delete storage groups success")
+	}
+}
+
+/*
+ *create single time series
+ *
+ *params
+ *path: string, complete time series path (starts from root)
+ *dataType: int32, data type for this time series
+ *encoding: int32, data type for this time series
+ *compressor: int32, compressing type for this time series
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) CreateTimeseries(path string, dataType int32, encoding int32, compressor int32) {
+	request := rpc.TSCreateTimeseriesReq{SessionId: s.sessionId, Path: path, DataType: dataType, Encoding: encoding,
+		Compressor: compressor}
+	status, err := s.client.CreateTimeseries(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("creating time series failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   path,
+			"code": status.Code,
+		}).Info("creating time series success")
+	}
+}
+
+/*
+ *create multiple time series
+ *
+ *params
+ *paths: []string, complete time series paths (starts from root)
+ *dataTypes: []int32, data types for time series
+ *encodings: []int32, encodings for time series
+ *compressors: []int32, compressing types for time series
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) CreateMultiTimeseries(paths []string, dataTypes []int32, encodings []int32, compressors []int32) {
+	request := rpc.TSCreateMultiTimeseriesReq{SessionId: s.sessionId, Paths: paths, DataTypes: dataTypes,
+		Encodings: encodings, Compressors: compressors}
+	status, err := s.client.CreateMultiTimeseries(context.Background(), &request)
+	s.ts = strings.Replace(strings.Trim(fmt.Sprint(paths), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("creating multi time series failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   s.ts,
+			"code": status.Code,
+		}).Info("creating multi time series success")
+	}
+}
+
+/*
+ *delete multiple time series, including data and schema
+ *
+ *params
+ *paths: []string, time series paths, which should be complete (starts from root)
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteTimeseries(paths []string) {
+	status, err := s.client.DeleteTimeseries(context.Background(), s.sessionId, paths)
+	var ts = strings.Replace(strings.Trim(fmt.Sprint(paths), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("delete time series failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   ts,
+			"code": status.Code,
+		}).Info("delete time series success")
+	}
+}
+
+/*
+ *delete all startTime <= data <= endTime in multiple time series
+ *
+ *params
+ *paths: []string, time series array that the data in
+ *startTime: int64, start time of deletion range
+ *endTime: int64, end time of deletion range
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) DeleteData(paths []string, startTime int64, endTime int64) {
+	request := rpc.TSDeleteDataReq{SessionId: s.sessionId, Paths: paths, StartTime: startTime, EndTime: endTime}
+	status, err := s.client.DeleteData(context.Background(), &request)
+	s.ts = strings.Replace(strings.Trim(fmt.Sprint(paths), "[]"), " ", ",", -1)
+	if err != nil {
+		Log.WithError(err).Error("delete data failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"ts":   s.ts,
+			"code": status.Code,
+		}).Info("delete data success")
+	}
+}
+
+/*
+ *special case for inserting one row of String (TEXT) value
+ *
+ *params
+ *deviceId: string, time series path for device
+ *measurements: []string, sensor names
+ *values: []string, values to be inserted, for each sensor
+ *timestamp: int64, indicate the timestamp of the row of data
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertStringRecord(deviceId string, measurements []string, values []string, timestamp int64) {
+	request := rpc.TSInsertStringRecordReq{SessionId: s.sessionId, DeviceId: deviceId, Measurements: measurements,
+		Values: values, Timestamp: timestamp}
+	status, err := s.client.InsertStringRecord(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert one string record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one string record success")
+	}
+}
+
+func (s *Session) TestInsertStringRecord(deviceId string, measurements []string, values []string, timestamp int64) {
+	request := rpc.TSInsertStringRecordReq{SessionId: s.sessionId, DeviceId: deviceId, Measurements: measurements,
+		Values: values, Timestamp: timestamp}
+	status, err := s.client.TestInsertStringRecord(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert one string record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one string record success")
+	}
+}
+
+/*
+ *special case for inserting multiple rows of String (TEXT) value
+ *
+ *params
+ *deviceIds: []string, time series paths for device
+ *measurements: [][]string, each element of outer list indicates measurements of a device
+ *values: [][]interface{}, values to be inserted, for each device
+ *timestamps: []int64, timestamps for records
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertStringRecords(deviceIds []string, measurements [][]string, values [][]string,
+	timestamps []int64) {
+	request := rpc.TSInsertStringRecordsReq{SessionId: s.sessionId, DeviceIds: deviceIds, MeasurementsList: measurements,
+		ValuesList: values, Timestamps: timestamps}
+	s.dv = strings.Replace(strings.Trim(fmt.Sprint(deviceIds), "[]"), " ", ",", -1)
+	status, err := s.client.InsertStringRecords(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert multi string records failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   s.dv,
+			"code": status.Code,
+		}).Info("insert multi string records success")
+	}
+}
+
+func (s *Session) TestInsertStringRecords(deviceIds []string, measurements [][]string, values [][]string,
+	timestamps []int64) {
+	request := rpc.TSInsertStringRecordsReq{SessionId: s.sessionId, DeviceIds: deviceIds, MeasurementsList: measurements,
+		ValuesList: values, Timestamps: timestamps}
+	s.dv = strings.Replace(strings.Trim(fmt.Sprint(deviceIds), "[]"), " ", ",", -1)
+	status, err := s.client.TestInsertStringRecords(context.Background(), &request)
+	if err != nil {
+		Log.WithError(err).Error("insert multi string records failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   s.dv,
+			"code": status.Code,
+		}).Info("insert multi string records success")
+	}
+}
+
+/*
+ *insert one row of record into database, if you want improve your performance, please use insertTablet method
+ *
+ *params
+ *deviceId: string, time series path for device
+ *measurements: []string, sensor names
+ *dataTypes: []int32, list of dataType, indicate the data type for each sensor
+ *values: []interface{}, values to be inserted, for each sensor
+ *timestamp: int64, indicate the timestamp of the row of data
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertRecord(deviceId string, measurements []string, dataTypes []int16, values []interface{},
+	timestamp int64) {
+	request := s.genInsertRecordReq(deviceId, measurements, dataTypes, values, timestamp)
+	status, err := s.client.InsertRecord(context.Background(), request)
+	if err != nil {
+		Log.WithError(err).Error("insert one record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one record success")
+	}
+}
+
+func (s *Session) TestInsertRecord(deviceId string, measurements []string, dataTypes []int16, values []interface{},
+	timestamp int64) {
+	request := s.genInsertRecordReq(deviceId, measurements, dataTypes, values, timestamp)
+	status, err := s.client.TestInsertRecord(context.Background(), request)
+	if err != nil {
+		Log.WithError(err).Error("insert one record failed")
+	} else {
+		Log.WithFields(logrus.Fields{
+			"dv":   deviceId,
+			"code": status.Code,
+		}).Info("insert one record success")
+	}
+}
+
+func (s *Session) genInsertRecordReq(deviceId string, measurements []string, dataTypes []int16, values []interface{},
+	timestamp int64) *rpc.TSInsertRecordReq {
+	request := rpc.TSInsertRecordReq{SessionId: s.sessionId, DeviceId: deviceId, Measurements: measurements,
+		Timestamp: timestamp}
+	request.Values = valuesToBytes(dataTypes, values)
+	return &request
+}
+
+/*
+ *insert multiple rows of data, records are independent to each other, in other words, there's no relationship
+ *between those records
+ *
+ *params
+ *deviceIds: []string, time series paths for device
+ *measurements: [][]string, each element of outer list indicates measurements of a device
+ *dataTypes: [][]int32, each element of outer list indicates sensor data types of a device
+ *values: [][]interface{}, values to be inserted, for each device
+ *timestamps: []int64, timestamps for records
+ *
+ *return
+ *error: correctness of operation
+ */
+func (s *Session) InsertRecords(deviceIds []string, measurements [][]string, dataTypes [][]int16, values [][]interface{},
+	timestamps []int64) {
+	s.dv = strings.Replace(strings.Trim(fmt.Sprint(deviceIds), "[]"), " ", ",", -1)

Review comment:
       But the direct output deviceids [] is the address




----------------------------------------------------------------
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