You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tinkerpop.apache.org by GitBox <gi...@apache.org> on 2022/01/31 20:54:04 UTC

[GitHub] [tinkerpop] lyndonb-bq opened a new pull request #1556: Milestone 1 of the gremlin-go driver

lyndonb-bq opened a new pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556


   This has the changes for Milestone 1 of the gremlin-go driver. We ended up losing our commit history, but I would like to thank @xiazcy, @simonz-bq, and @L0Lmaker for the work they put into this. Going forward we will make sure commits are properly preserved.
   
   With this milestone, a user can send string queries and receive string results. A 'Getting Started' section is being worked on and will add more detail on how this can be used at this stage.
   
   Pull request comments will be added as backlog items for the next milestone, instead of being addressed within this pull request.
   
   This pull request will remain open for a week or so, so the community has a chance to look at it and critique 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.

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] spmallette commented on pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
spmallette commented on pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#issuecomment-1040196209


   > Without this design document, it would be very hard to code review, since we would have to resort to reverse engineer the contract from the code itself.
   
   I think a design document would be helpful - in fact perhaps the Go driver could set a model by which we document all of the drivers. However, I don't think we should place too high a bar on the early stages of this work. We can place a high bar on calling it "production ready" but for purposes of reaching a point where this PR can be merged I think the bar is fairly low. If it can send requests and get back results, that's at least as much as the unofficial drivers do and I think this PR is getting closer to doing far more than that. Personally, I'd advocate for merging early with experimental release candidates for folks to try the functionality and to provide feedback. In the mean time, folks working on this PR can address issues related to getting to a fully production ready driver in additional PRs.
   
   As for the driver documentation, i think we should consider expanding (or re-writing) the provider documentation which has server protocol related content in it:
   
   https://github.com/apache/tinkerpop/blob/master/docs/src/dev/provider/index.asciidoc#graph-driver-provider-requirements


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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r806384003



##########
File path: gremlin-go/driver/client.go
##########
@@ -0,0 +1,87 @@
+/*
+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 gremlingo
+
+import (
+	"golang.org/x/text/language"
+)
+
+// ClientSettings is used to modify a Client's settings on initialization.
+type ClientSettings struct {
+	TransporterType TransporterType
+	LogVerbosity    LogVerbosity
+	Logger          Logger
+	Language        language.Tag
+}
+
+// Client is used to connect and interact with a Gremlin-supported server.
+type Client struct {
+	host            string
+	port            int
+	logHandler      *logHandler
+	transporterType TransporterType
+	connection      *connection
+}
+
+// NewClient creates a Client and configures it with the given parameters.
+func NewClient(host string, port int, configurations ...func(settings *ClientSettings)) *Client {
+	settings := &ClientSettings{
+		TransporterType: Gorilla,
+		LogVerbosity:    Info,
+		Logger:          &defaultLogger{},
+		Language:        language.English,
+	}
+	for _, configuration := range configurations {
+		configuration(settings)
+	}
+
+	logHandler := newLogHandler(settings.Logger, settings.LogVerbosity, settings.Language)
+	client := &Client{
+		host:            host,
+		port:            port,
+		logHandler:      logHandler,
+		transporterType: settings.TransporterType,
+		connection:      nil,
+	}
+	return client
+}
+
+// Close closes the client via connection
+func (client *Client) Close() error {
+	return client.connection.close()
+}
+
+// Submit submits a Gremlin script to the server and returns a ResultSet

Review comment:
       As of MS2, it is no longer a synchronous call!

##########
File path: gremlin-go/driver/request.go
##########
@@ -0,0 +1,42 @@
+/*
+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 gremlingo
+
+import "github.com/google/uuid"
+
+const op = "eval"
+const processor = ""
+
+// request represents a request to the server
+type request struct {
+	requestID uuid.UUID              `json:"requestId"`
+	op        string                 `json:"op"`
+	processor string                 `json:"processor"`
+	args      map[string]interface{} `json:"args"`
+}
+
+func makeStringRequest(requestString string) (req request) {
+	return request{uuid.New(), op, processor, map[string]interface{}{
+		"gremlin": requestString,
+		"aliases": map[string]interface{}{
+			"g": "g",

Review comment:
       TODO added and tracked.




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r806384793



##########
File path: gremlin-go/driver/gorillaTransporter.go
##########
@@ -0,0 +1,91 @@
+/*
+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 gremlingo
+
+import (
+	"net/url"
+	"strconv"
+
+	"github.com/gorilla/websocket"
+)
+
+type gorillaTransporter struct {
+	host       string
+	port       int
+	connection websocketConn
+	isClosed   bool
+}
+
+func (transporter *gorillaTransporter) Connect() (err error) {
+	if transporter.connection != nil {
+		return

Review comment:
       This part of the logic has been refactored as of MS2 to have a more intuitive workflow.




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] spmallette commented on pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
spmallette commented on pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#issuecomment-1040196209


   > Without this design document, it would be very hard to code review, since we would have to resort to reverse engineer the contract from the code itself.
   
   I think a design document would be helpful - in fact perhaps the Go driver could set a model by which we document all of the drivers. However, I don't think we should place too high a bar on the early stages of this work. We can place a high bar on calling it "production ready" but for purposes of reaching a point where this PR can be merged I think the bar is fairly low. If it can send requests and get back results, that's at least as much as the unofficial drivers do and I think this PR is getting closer to doing far more than that. Personally, I'd advocate for merging early with experimental release candidates for folks to try the functionality and to provide feedback. In the mean time, folks working on this PR can address issues related to getting to a fully production ready driver in additional PRs.
   
   As for the driver documentation, i think we should consider expanding (or re-writing) the provider documentation which has server protocol related content in it:
   
   https://github.com/apache/tinkerpop/blob/master/docs/src/dev/provider/index.asciidoc#graph-driver-provider-requirements


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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] lyndonb-bq commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r801084685



##########
File path: gremlin-go/driver/graphTraversalSource.go
##########
@@ -0,0 +1,23 @@
+/*
+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 gremlingo
+
+type GraphTraversalSource interface {

Review comment:
       @divijvaidya 
   
   > Thank you for taking up this body of work.
   > 
   > Please add a style guide for GoLang that other future develops contributing to this code would follow.
   > 
   > I added very initial comments based on first look. I will delve deeper into this PR in the coming days but as a general rule of thumb we would try to apply the learnings from structure of other clients and improve this one. Primary focus will be testability and debuggability of the client.
   
   Thanks for reviewing Divij, we will definitely look into that. We are having a bit of a hard time with the initial structure because of the way Golang works - if we move a file to a subfolder, it can now only access public facing structs of other classes within the project. However as we get more components in, I think we can begin to organize this better. We look forward to the feedback you will be giving and will incorporate it into the coming Milestones.




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r806376918



##########
File path: gremlin-go/driver/connection.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 gremlingo
+
+type connection struct {

Review comment:
       Added as a part of MS2

##########
File path: gremlin-go/driver/connection.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 gremlingo
+
+type connection struct {
+	host            string
+	port            int
+	transporterType TransporterType
+	logHandler      *logHandler
+	transporter     transporter
+	protocol        protocol
+	results         map[string]ResultSet
+}
+
+func (connection *connection) close() (err error) {
+	if connection.transporter != nil {
+		err = connection.transporter.Close()
+	}
+	return
+}
+
+func (connection *connection) connect() error {
+	if connection.transporter != nil {
+		closeErr := connection.transporter.Close()
+		connection.logHandler.logf(Warning, transportCloseFailed, closeErr)
+	}
+	connection.protocol = newGremlinServerWSProtocol(connection.logHandler)
+	connection.transporter = getTransportLayer(connection.transporterType, connection.host, connection.port)
+	err := connection.transporter.Connect()
+	if err != nil {
+		return err
+	}
+	connection.protocol.connectionMade(connection.transporter)
+	return nil
+}
+
+func (connection *connection) write(request *request) (ResultSet, error) {
+	if connection.transporter == nil || connection.transporter.IsClosed() {
+		err := connection.connect()

Review comment:
       The proper logic for this method is updated as a part of MS2.

##########
File path: gremlin-go/driver/protocol.go
##########
@@ -0,0 +1,127 @@
+/*
+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 gremlingo
+
+import (
+	"errors"
+	"fmt"
+	"net/http"
+)
+
+type protocol interface {

Review comment:
       Documentation added - more detailed explanation cannot live here however.

##########
File path: gremlin-go/driver/protocol.go
##########
@@ -0,0 +1,127 @@
+/*
+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 gremlingo
+
+import (
+	"errors"
+	"fmt"
+	"net/http"
+)
+
+type protocol interface {
+	connectionMade(transport transporter)
+	read(resultSets map[string]ResultSet) (string, error)
+	write(request *request, results map[string]ResultSet) (string, error)
+}
+
+type protocolBase struct {
+	protocol
+
+	transporter transporter
+}
+
+type gremlinServerWSProtocol struct {
+	*protocolBase
+
+	serializer       serializer
+	logHandler       *logHandler
+	maxContentLength int
+	username         string
+	password         string
+}
+
+func (protocol *protocolBase) connectionMade(transporter transporter) {
+	protocol.transporter = transporter
+}
+
+func (protocol *gremlinServerWSProtocol) read(resultSets map[string]ResultSet) (string, error) {
+	// Read data from transport layer.
+	msg, err := protocol.transporter.Read()
+	if err != nil || msg == nil {
+		if err != nil {
+			return "", err
+		}
+		protocol.logHandler.log(Error, malformedURL)
+		return "", errors.New("malformed ws or wss URL")
+	}
+	// Deserialize message and unpack.
+	response, err := protocol.serializer.deserializeMessage(msg)
+	if err != nil {
+		return "", err
+	}
+
+	responseID, statusCode, metadata, data := response.responseID, response.responseStatus.code,
+		response.responseResult.meta, response.responseResult.data
+
+	resultSet := resultSets[responseID.String()]
+	if resultSet == nil {
+		resultSet = newChannelResultSet(responseID.String())
+	}
+	resultSets[responseID.String()] = resultSet
+	if aggregateTo, ok := metadata["aggregateTo"]; ok {
+		resultSet.setAggregateTo(aggregateTo.(string))
+	}
+
+	// Handle status codes appropriately. If status code is http.StatusPartialContent, we need to re-read data.
+	if statusCode == http.StatusProxyAuthRequired {

Review comment:
       Done as a part of MS2.

##########
File path: gremlin-go/driver/client.go
##########
@@ -0,0 +1,87 @@
+/*
+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 gremlingo
+
+import (
+	"golang.org/x/text/language"
+)
+
+// ClientSettings is used to modify a Client's settings on initialization.
+type ClientSettings struct {
+	TransporterType TransporterType
+	LogVerbosity    LogVerbosity
+	Logger          Logger
+	Language        language.Tag
+}
+
+// Client is used to connect and interact with a Gremlin-supported server.
+type Client struct {
+	host            string
+	port            int
+	logHandler      *logHandler
+	transporterType TransporterType
+	connection      *connection
+}
+
+// NewClient creates a Client and configures it with the given parameters.
+func NewClient(host string, port int, configurations ...func(settings *ClientSettings)) *Client {
+	settings := &ClientSettings{
+		TransporterType: Gorilla,
+		LogVerbosity:    Info,
+		Logger:          &defaultLogger{},
+		Language:        language.English,
+	}
+	for _, configuration := range configurations {
+		configuration(settings)
+	}
+
+	logHandler := newLogHandler(settings.Logger, settings.LogVerbosity, settings.Language)
+	client := &Client{
+		host:            host,
+		port:            port,
+		logHandler:      logHandler,
+		transporterType: settings.TransporterType,
+		connection:      nil,
+	}
+	return client
+}
+
+// Close closes the client via connection
+func (client *Client) Close() error {
+	return client.connection.close()
+}
+
+// Submit submits a Gremlin script to the server and returns a ResultSet

Review comment:
       As of MS2, it is no longer a synchronous call!

##########
File path: gremlin-go/driver/request.go
##########
@@ -0,0 +1,42 @@
+/*
+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 gremlingo
+
+import "github.com/google/uuid"
+
+const op = "eval"
+const processor = ""
+
+// request represents a request to the server
+type request struct {
+	requestID uuid.UUID              `json:"requestId"`
+	op        string                 `json:"op"`
+	processor string                 `json:"processor"`
+	args      map[string]interface{} `json:"args"`
+}
+
+func makeStringRequest(requestString string) (req request) {
+	return request{uuid.New(), op, processor, map[string]interface{}{
+		"gremlin": requestString,
+		"aliases": map[string]interface{}{
+			"g": "g",

Review comment:
       TODO added and tracked.

##########
File path: gremlin-go/driver/transporterFactory.go
##########
@@ -0,0 +1,37 @@
+/*
+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 gremlingo
+
+// TransporterType is an alias for valid transport protocols.
+type TransporterType int
+
+const (
+	// Gorilla transport layer: github.com/gorilla/websocket
+	Gorilla TransporterType = iota

Review comment:
       Given the way we consume configuration, it wouldn't be an issue in this case.
   
   Regardless, this is a good suggestion and has been changed for MS2!

##########
File path: gremlin-go/driver/gorillaTransporter.go
##########
@@ -0,0 +1,91 @@
+/*
+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 gremlingo
+
+import (
+	"net/url"
+	"strconv"
+
+	"github.com/gorilla/websocket"
+)
+

Review comment:
       Done.

##########
File path: gremlin-go/driver/gorillaTransporter.go
##########
@@ -0,0 +1,91 @@
+/*
+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 gremlingo
+
+import (
+	"net/url"
+	"strconv"
+
+	"github.com/gorilla/websocket"
+)
+
+type gorillaTransporter struct {
+	host       string
+	port       int
+	connection websocketConn
+	isClosed   bool
+}
+
+func (transporter *gorillaTransporter) Connect() (err error) {
+	if transporter.connection != nil {
+		return

Review comment:
       This part of the logic has been refactored as of MS2 to have a more intuitive workflow.




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] divijvaidya commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
divijvaidya commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r804094327



##########
File path: gremlin-go/driver/client.go
##########
@@ -0,0 +1,87 @@
+/*
+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 gremlingo
+
+import (
+	"golang.org/x/text/language"
+)
+
+// ClientSettings is used to modify a Client's settings on initialization.
+type ClientSettings struct {
+	TransporterType TransporterType
+	LogVerbosity    LogVerbosity
+	Logger          Logger
+	Language        language.Tag
+}
+
+// Client is used to connect and interact with a Gremlin-supported server.
+type Client struct {
+	host            string
+	port            int
+	logHandler      *logHandler
+	transporterType TransporterType
+	connection      *connection
+}
+
+// NewClient creates a Client and configures it with the given parameters.
+func NewClient(host string, port int, configurations ...func(settings *ClientSettings)) *Client {
+	settings := &ClientSettings{
+		TransporterType: Gorilla,
+		LogVerbosity:    Info,
+		Logger:          &defaultLogger{},
+		Language:        language.English,
+	}
+	for _, configuration := range configurations {
+		configuration(settings)
+	}
+
+	logHandler := newLogHandler(settings.Logger, settings.LogVerbosity, settings.Language)
+	client := &Client{
+		host:            host,
+		port:            port,
+		logHandler:      logHandler,
+		transporterType: settings.TransporterType,
+		connection:      nil,
+	}
+	return client
+}
+
+// Close closes the client via connection
+func (client *Client) Close() error {
+	return client.connection.close()
+}
+
+// Submit submits a Gremlin script to the server and returns a ResultSet
+func (client *Client) Submit(traversalString string) (ResultSet, error) {

Review comment:
       please validate input string here and add debug logs to print the start of submit

##########
File path: gremlin-go/driver/client.go
##########
@@ -0,0 +1,87 @@
+/*
+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 gremlingo
+
+import (
+	"golang.org/x/text/language"
+)
+
+// ClientSettings is used to modify a Client's settings on initialization.
+type ClientSettings struct {
+	TransporterType TransporterType
+	LogVerbosity    LogVerbosity
+	Logger          Logger
+	Language        language.Tag
+}
+
+// Client is used to connect and interact with a Gremlin-supported server.
+type Client struct {
+	host            string
+	port            int
+	logHandler      *logHandler
+	transporterType TransporterType
+	connection      *connection
+}
+
+// NewClient creates a Client and configures it with the given parameters.
+func NewClient(host string, port int, configurations ...func(settings *ClientSettings)) *Client {
+	settings := &ClientSettings{
+		TransporterType: Gorilla,
+		LogVerbosity:    Info,
+		Logger:          &defaultLogger{},
+		Language:        language.English,
+	}
+	for _, configuration := range configurations {
+		configuration(settings)
+	}
+
+	logHandler := newLogHandler(settings.Logger, settings.LogVerbosity, settings.Language)
+	client := &Client{
+		host:            host,
+		port:            port,
+		logHandler:      logHandler,
+		transporterType: settings.TransporterType,
+		connection:      nil,
+	}
+	return client
+}
+
+// Close closes the client via connection
+func (client *Client) Close() error {
+	return client.connection.close()
+}
+
+// Submit submits a Gremlin script to the server and returns a ResultSet

Review comment:
       please specify in documentation that this is synchronous/blocking call

##########
File path: gremlin-go/driver/gorillaTransporter.go
##########
@@ -0,0 +1,91 @@
+/*
+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 gremlingo
+
+import (
+	"net/url"
+	"strconv"
+
+	"github.com/gorilla/websocket"
+)
+

Review comment:
       documentation

##########
File path: gremlin-go/driver/transporterFactory.go
##########
@@ -0,0 +1,37 @@
+/*
+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 gremlingo
+
+// TransporterType is an alias for valid transport protocols.
+type TransporterType int
+
+const (
+	// Gorilla transport layer: github.com/gorilla/websocket
+	Gorilla TransporterType = iota

Review comment:
       as a general style guide, let us start enums from 1 because variables have a 0 default value and hence starting an enum from 1 will help distinguish between uninitialized enum vs. intentional value of an enum

##########
File path: gremlin-go/driver/request.go
##########
@@ -0,0 +1,42 @@
+/*
+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 gremlingo
+
+import "github.com/google/uuid"
+
+const op = "eval"
+const processor = ""
+
+// request represents a request to the server
+type request struct {
+	requestID uuid.UUID              `json:"requestId"`
+	op        string                 `json:"op"`
+	processor string                 `json:"processor"`
+	args      map[string]interface{} `json:"args"`
+}
+
+func makeStringRequest(requestString string) (req request) {
+	return request{uuid.New(), op, processor, map[string]interface{}{
+		"gremlin": requestString,
+		"aliases": map[string]interface{}{
+			"g": "g",

Review comment:
       please add a TODO to make this configurable in future (similar to AliasedClient in Java). 

##########
File path: gremlin-go/driver/gorillaTransporter.go
##########
@@ -0,0 +1,91 @@
+/*
+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 gremlingo
+
+import (
+	"net/url"
+	"strconv"
+
+	"github.com/gorilla/websocket"
+)
+
+type gorillaTransporter struct {
+	host       string
+	port       int
+	connection websocketConn
+	isClosed   bool
+}
+
+func (transporter *gorillaTransporter) Connect() (err error) {
+	if transporter.connection != nil {
+		return

Review comment:
       please return an error so that caller can know it is a non-retriable error and also that it is a configuration error.




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r806384527



##########
File path: gremlin-go/driver/gorillaTransporter.go
##########
@@ -0,0 +1,91 @@
+/*
+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 gremlingo
+
+import (
+	"net/url"
+	"strconv"
+
+	"github.com/gorilla/websocket"
+)
+

Review comment:
       Done.




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r806376918



##########
File path: gremlin-go/driver/connection.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 gremlingo
+
+type connection struct {

Review comment:
       Added as a part of MS2




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r806383631



##########
File path: gremlin-go/driver/protocol.go
##########
@@ -0,0 +1,127 @@
+/*
+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 gremlingo
+
+import (
+	"errors"
+	"fmt"
+	"net/http"
+)
+
+type protocol interface {
+	connectionMade(transport transporter)
+	read(resultSets map[string]ResultSet) (string, error)
+	write(request *request, results map[string]ResultSet) (string, error)
+}
+
+type protocolBase struct {
+	protocol
+
+	transporter transporter
+}
+
+type gremlinServerWSProtocol struct {
+	*protocolBase
+
+	serializer       serializer
+	logHandler       *logHandler
+	maxContentLength int
+	username         string
+	password         string
+}
+
+func (protocol *protocolBase) connectionMade(transporter transporter) {
+	protocol.transporter = transporter
+}
+
+func (protocol *gremlinServerWSProtocol) read(resultSets map[string]ResultSet) (string, error) {
+	// Read data from transport layer.
+	msg, err := protocol.transporter.Read()
+	if err != nil || msg == nil {
+		if err != nil {
+			return "", err
+		}
+		protocol.logHandler.log(Error, malformedURL)
+		return "", errors.New("malformed ws or wss URL")
+	}
+	// Deserialize message and unpack.
+	response, err := protocol.serializer.deserializeMessage(msg)
+	if err != nil {
+		return "", err
+	}
+
+	responseID, statusCode, metadata, data := response.responseID, response.responseStatus.code,
+		response.responseResult.meta, response.responseResult.data
+
+	resultSet := resultSets[responseID.String()]
+	if resultSet == nil {
+		resultSet = newChannelResultSet(responseID.String())
+	}
+	resultSets[responseID.String()] = resultSet
+	if aggregateTo, ok := metadata["aggregateTo"]; ok {
+		resultSet.setAggregateTo(aggregateTo.(string))
+	}
+
+	// Handle status codes appropriately. If status code is http.StatusPartialContent, we need to re-read data.
+	if statusCode == http.StatusProxyAuthRequired {

Review comment:
       Done as a part of MS2.




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#issuecomment-1039772844


   @divijvaidya thanks for the feedback!
   
   I responded to a majority of your comments to keep track of what we have already tackled in milestone 2 as well as additional changes that were made to address your points.
   
   Anything that wasn't addressed is tracked and will either require work at a later time, or some investigation on what the best way to approach finding a solution is.


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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] lyndonb-bq merged pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
lyndonb-bq merged pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556


   


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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] divijvaidya commented on pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
divijvaidya commented on pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#issuecomment-1040599340


   @spmallette I agree with most of the things that you mentioned except for the below.
    
   > We can place a high bar on calling it "production ready" but for purposes of reaching a point where this PR can be merged I think the bar is fairly low. If it can send requests and get back results, that's at least as much as the unofficial drivers do and I think this PR is getting closer to doing far more than that. Personally, I'd advocate for merging early with experimental release candidates for folks to try the functionality and to provide feedback.
   
   Once a code is merged to our main TinkerPop repository, we are picking up the ownership of maintaining it in the future. Thus, the experimental version aka minimum viable version (MVP) that gets merged into the repository has to adhere to certain code quality which would help us in code maintenance. The minimal acceptable standard should be documented code, documented features for what it can & cannot do, unit & integration tests for the feature which the client is documented to support and adherence to best practices for build & code style.
   
   IMO, this PR does not meet that standard right now because of missing documentation, missing integration tests to handle scenarios such as server and code styling practices. I am open to discussing a merge in a separate branch but when merging to master again, the whole body of code would need to go through a PR.
   My preferred approach and suggestion would be to incrementally add features to a larger project being merged into master while ensuring that each feature is production ready.
   
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r806377167



##########
File path: gremlin-go/driver/connection.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 gremlingo
+
+type connection struct {
+	host            string
+	port            int
+	transporterType TransporterType
+	logHandler      *logHandler
+	transporter     transporter
+	protocol        protocol
+	results         map[string]ResultSet
+}
+
+func (connection *connection) close() (err error) {
+	if connection.transporter != nil {
+		err = connection.transporter.Close()
+	}
+	return
+}
+
+func (connection *connection) connect() error {
+	if connection.transporter != nil {
+		closeErr := connection.transporter.Close()
+		connection.logHandler.logf(Warning, transportCloseFailed, closeErr)
+	}
+	connection.protocol = newGremlinServerWSProtocol(connection.logHandler)
+	connection.transporter = getTransportLayer(connection.transporterType, connection.host, connection.port)
+	err := connection.transporter.Connect()
+	if err != nil {
+		return err
+	}
+	connection.protocol.connectionMade(connection.transporter)
+	return nil
+}
+
+func (connection *connection) write(request *request) (ResultSet, error) {
+	if connection.transporter == nil || connection.transporter.IsClosed() {
+		err := connection.connect()

Review comment:
       The proper logic for this method is updated as a part of MS2.




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] divijvaidya commented on pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
divijvaidya commented on pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#issuecomment-1040599340


   @spmallette I agree with most of the things that you mentioned except for the below.
    
   > We can place a high bar on calling it "production ready" but for purposes of reaching a point where this PR can be merged I think the bar is fairly low. If it can send requests and get back results, that's at least as much as the unofficial drivers do and I think this PR is getting closer to doing far more than that. Personally, I'd advocate for merging early with experimental release candidates for folks to try the functionality and to provide feedback.
   
   Once a code is merged to our main TinkerPop repository, we are picking up the ownership of maintaining it in the future. Thus, the experimental version aka minimum viable version (MVP) that gets merged into the repository has to adhere to certain code quality which would help us in code maintenance. The minimal acceptable standard should be documented code, documented features for what it can & cannot do, unit & integration tests for the feature which the client is documented to support and adherence to best practices for build & code style.
   
   IMO, this PR does not meet that standard right now because of missing documentation, missing integration tests to handle scenarios such as server and code styling practices. I am open to discussing a merge in a separate branch but when merging to master again, the whole body of code would need to go through a PR.
   My preferred approach and suggestion would be to incrementally add features to a larger project being merged into master while ensuring that each feature is production ready.
   
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#issuecomment-1039772844


   @divijvaidya thanks for the feedback!
   
   I responded to a majority of your comments to keep track of what we have already tackled in milestone 2 as well as additional changes that were made to address your points.
   
   Anything that wasn't addressed is tracked and will either require work at a later time, or some investigation on what the best way to approach finding a solution is.


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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] divijvaidya commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
divijvaidya commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r800879591



##########
File path: gremlin-go/go.mod
##########
@@ -0,0 +1,18 @@
+module github.com/lyndonb-bq/tinkerpop/gremlin-go
+
+go 1.17
+
+require (
+	github.com/google/uuid v1.3.0

Review comment:
       please ensure that the libraries satisfies the Apache requirement for third party libraries. https://www.apache.org/legal/resolved.html#category-a

##########
File path: gremlin-go/driver/connection.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 gremlingo
+
+type connection struct {

Review comment:
       From my experience of maintaining other clients, I have observed that making a concept of `state` in multiple client entities could be very useful for debugging.
   
   My suggestion would be to add stateful transition to the lifecycle of a connection such as:
   Initialised
   Connection With Remote Initiated
   Connection With Remote Established
   Close request received
   Close in progress
   Closed
   
   These state transitions help us model various edge cases such as what happens when a connection receives a message to disconnect when it is in "Connection With Remote Initiated" state.
   
   
   

##########
File path: gremlin-go/driver/connection.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 gremlingo
+
+type connection struct {
+	host            string
+	port            int
+	transporterType TransporterType
+	logHandler      *logHandler
+	transporter     transporter
+	protocol        protocol
+	results         map[string]ResultSet
+}
+
+func (connection *connection) close() (err error) {
+	if connection.transporter != nil {
+		err = connection.transporter.Close()
+	}
+	return
+}
+
+func (connection *connection) connect() error {
+	if connection.transporter != nil {
+		closeErr := connection.transporter.Close()
+		connection.logHandler.logf(Warning, transportCloseFailed, closeErr)
+	}
+	connection.protocol = newGremlinServerWSProtocol(connection.logHandler)
+	connection.transporter = getTransportLayer(connection.transporterType, connection.host, connection.port)
+	err := connection.transporter.Connect()
+	if err != nil {
+		return err
+	}
+	connection.protocol.connectionMade(connection.transporter)
+	return nil
+}
+
+func (connection *connection) write(request *request) (ResultSet, error) {
+	if connection.transporter == nil || connection.transporter.IsClosed() {
+		err := connection.connect()

Review comment:
       Let's keep things simple and not retry to re-connect here. A connection should be in "connected" state if we want to write on it. Else throw an error about illegal state and the upper layers should choose a different connection to work with.

##########
File path: gremlin-go/driver/serializer.go
##########
@@ -0,0 +1,343 @@
+/*
+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 gremlingo
+
+import (
+	"bytes"
+	"encoding/binary"
+	"errors"
+	"math/big"
+	"strings"
+
+	"github.com/google/uuid"
+)
+
+const graphBinaryMimeType = "application/vnd.graphbinary-v1.0"

Review comment:
       note that graphbinary has one more mime type: 
   
   `application/vnd.graphbinary-v1.0-stringd`

##########
File path: gremlin-go/driver/protocol.go
##########
@@ -0,0 +1,127 @@
+/*
+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 gremlingo
+
+import (
+	"errors"
+	"fmt"
+	"net/http"
+)
+
+type protocol interface {

Review comment:
       please document the responsibility of this class. More specifically, please explain somewhere the contract between the transporter, connection and protocol. Please also explain how their lifecycles impact each other.

##########
File path: gremlin-go/driver/protocol.go
##########
@@ -0,0 +1,127 @@
+/*
+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 gremlingo
+
+import (
+	"errors"
+	"fmt"
+	"net/http"
+)
+
+type protocol interface {
+	connectionMade(transport transporter)
+	read(resultSets map[string]ResultSet) (string, error)
+	write(request *request, results map[string]ResultSet) (string, error)
+}
+
+type protocolBase struct {
+	protocol
+
+	transporter transporter
+}
+
+type gremlinServerWSProtocol struct {
+	*protocolBase
+
+	serializer       serializer
+	logHandler       *logHandler
+	maxContentLength int
+	username         string
+	password         string
+}
+
+func (protocol *protocolBase) connectionMade(transporter transporter) {
+	protocol.transporter = transporter
+}
+
+func (protocol *gremlinServerWSProtocol) read(resultSets map[string]ResultSet) (string, error) {
+	// Read data from transport layer.
+	msg, err := protocol.transporter.Read()
+	if err != nil || msg == nil {
+		if err != nil {
+			return "", err
+		}
+		protocol.logHandler.log(Error, malformedURL)
+		return "", errors.New("malformed ws or wss URL")
+	}
+	// Deserialize message and unpack.
+	response, err := protocol.serializer.deserializeMessage(msg)
+	if err != nil {
+		return "", err
+	}
+
+	responseID, statusCode, metadata, data := response.responseID, response.responseStatus.code,
+		response.responseResult.meta, response.responseResult.data
+
+	resultSet := resultSets[responseID.String()]
+	if resultSet == nil {
+		resultSet = newChannelResultSet(responseID.String())
+	}
+	resultSets[responseID.String()] = resultSet
+	if aggregateTo, ok := metadata["aggregateTo"]; ok {
+		resultSet.setAggregateTo(aggregateTo.(string))
+	}
+
+	// Handle status codes appropriately. If status code is http.StatusPartialContent, we need to re-read data.
+	if statusCode == http.StatusProxyAuthRequired {

Review comment:
       please move handling of status code in a separate function. 

##########
File path: gremlin-go/driver/graphTraversalSource.go
##########
@@ -0,0 +1,23 @@
+/*
+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 gremlingo
+
+type GraphTraversalSource interface {

Review comment:
       this and other data model  objects associated with Graph should perhaps be separated from connection related stuff as driver/structure/




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r806384461



##########
File path: gremlin-go/driver/transporterFactory.go
##########
@@ -0,0 +1,37 @@
+/*
+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 gremlingo
+
+// TransporterType is an alias for valid transport protocols.
+type TransporterType int
+
+const (
+	// Gorilla transport layer: github.com/gorilla/websocket
+	Gorilla TransporterType = iota

Review comment:
       Given the way we consume configuration, it wouldn't be an issue in this case.
   
   Regardless, this is a good suggestion and has been changed for MS2!




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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



[GitHub] [tinkerpop] simonz-bq commented on a change in pull request #1556: Milestone 1 of the gremlin-go driver

Posted by GitBox <gi...@apache.org>.
simonz-bq commented on a change in pull request #1556:
URL: https://github.com/apache/tinkerpop/pull/1556#discussion_r806383412



##########
File path: gremlin-go/driver/protocol.go
##########
@@ -0,0 +1,127 @@
+/*
+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 gremlingo
+
+import (
+	"errors"
+	"fmt"
+	"net/http"
+)
+
+type protocol interface {

Review comment:
       Documentation added - more detailed explanation cannot live here however.




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

To unsubscribe, e-mail: commits-unsubscribe@tinkerpop.apache.org

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