You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "zeroshade (via GitHub)" <gi...@apache.org> on 2023/04/13 21:42:18 UTC

[GitHub] [arrow-adbc] zeroshade opened a new pull request, #586: WIP: feat(go/adbc/driver): Adbc Driver for Snowflake

zeroshade opened a new pull request, #586:
URL: https://github.com/apache/arrow-adbc/pull/586

   Initial work to start creating a snowflake ADBC driver which we can eventually package up like we do for the Flight SQL driver. Currently only `GetInfo` and `GetObjects` are implemented, but it's a start!
   
   Will have to add secrets to the repo eventually to allow tests to work.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1174223272


##########
go/adbc/driver/snowflake/statement.go:
##########
@@ -0,0 +1,572 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql/driver"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+)
+
+const (
+	OptionStatementQueueSize = "adbc.flight.sql.rpc.queue_size"
+)
+
+type statement struct {
+	cnxn      *cnxn
+	alloc     memory.Allocator
+	queueSize int
+
+	query       string
+	targetTable string
+	append      bool
+
+	bound      arrow.Record
+	streamBind array.RecordReader
+}
+
+// Close releases any relevant resources associated with this statement
+// and closes it (particularly if it is a prepared statement).
+//
+// A statement instance should not be used after Close is called.
+func (st *statement) Close() error {
+	if st.cnxn == nil {
+		return adbc.Error{
+			Msg:  "statement already closed",
+			Code: adbc.StatusInvalidState}
+	}
+
+	if st.bound != nil {
+		st.bound.Release()
+		st.bound = nil
+	} else if st.streamBind != nil {
+		st.streamBind.Release()
+		st.streamBind = nil
+	}
+	st.cnxn = nil
+	return nil
+}
+
+// SetOption sets a string option on this statement
+func (st *statement) SetOption(key string, val string) error {
+	switch key {
+	case adbc.OptionKeyIngestTargetTable:
+		st.query = ""
+		st.targetTable = val
+	case adbc.OptionKeyIngestMode:
+		switch val {
+		case adbc.OptionValueIngestModeAppend:
+			st.append = true
+		case adbc.OptionValueIngestModeCreate:
+			st.append = false
+		default:
+			return adbc.Error{
+				Msg:  fmt.Sprintf("invalid statement option %s=%s", key, val),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+	default:
+		return adbc.Error{
+			Msg:  fmt.Sprintf("invalid statement option %s=%s", key, val),
+			Code: adbc.StatusInvalidArgument,
+		}
+	}
+	return nil
+}
+
+// SetSqlQuery sets the query string to be executed.
+//
+// The query can then be executed with any of the Execute methods.
+// For queries expected to be executed repeatedly, Prepare should be
+// called before execution.
+func (st *statement) SetSqlQuery(query string) error {
+	st.query = query
+	st.targetTable = ""
+	return nil
+}
+
+func toSnowflakeType(dt arrow.DataType) string {
+	switch dt.ID() {
+	case arrow.EXTENSION:
+		return toSnowflakeType(dt.(arrow.ExtensionType).StorageType())
+	case arrow.DICTIONARY:
+		return toSnowflakeType(dt.(*arrow.DictionaryType).ValueType)
+	case arrow.RUN_END_ENCODED:
+		return toSnowflakeType(dt.(*arrow.RunEndEncodedType).Encoded())
+	case arrow.INT8, arrow.INT16, arrow.INT32, arrow.INT64,
+		arrow.UINT8, arrow.UINT16, arrow.UINT32, arrow.UINT64:
+		return "integer"
+	case arrow.FLOAT32, arrow.FLOAT16, arrow.FLOAT64:
+		return "double"
+	case arrow.DECIMAL, arrow.DECIMAL256:
+		dec := dt.(arrow.DecimalType)
+		return fmt.Sprintf("NUMERIC(%d,%d)", dec.GetPrecision(), dec.GetScale())
+	case arrow.STRING, arrow.LARGE_STRING:
+		return "text"
+	case arrow.BINARY, arrow.LARGE_BINARY:
+		return "binary"
+	case arrow.FIXED_SIZE_BINARY:
+		fsb := dt.(*arrow.FixedSizeBinaryType)
+		return fmt.Sprintf("binary(%d)", fsb.ByteWidth)
+	case arrow.BOOL:
+		return "boolean"
+	case arrow.TIME32, arrow.TIME64:
+		t := dt.(arrow.TemporalWithUnit)
+		prec := int(t.TimeUnit()) * 3
+		return fmt.Sprintf("time(%d)", prec)
+	case arrow.DATE32, arrow.DATE64:
+		return "date"
+	case arrow.TIMESTAMP:
+		ts := dt.(*arrow.TimestampType)
+		prec := int(ts.Unit) * 3
+		if ts.TimeZone == "" {
+			return fmt.Sprintf("timestamp_tz(%d)", prec)
+		}
+		return fmt.Sprintf("timestamp_ltz(%d)", prec)
+	case arrow.DENSE_UNION, arrow.SPARSE_UNION:
+		return "variant"
+	case arrow.LIST, arrow.LARGE_LIST, arrow.FIXED_SIZE_LIST:
+		return "array"
+	case arrow.STRUCT, arrow.MAP:
+		return "object"
+	}
+
+	return ""
+}
+
+func (st *statement) initIngest(ctx context.Context) (string, error) {
+	var (
+		createBldr, insertBldr strings.Builder
+	)
+
+	createBldr.WriteString("CREATE TABLE ")
+	createBldr.WriteString(st.targetTable)

Review Comment:
   Possibly table names can't be escaped. If it becomes an issue, we could always just restrict it to alphanumeric or similar.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175856536


##########
.github/workflows/native-unix.yml:
##########
@@ -189,6 +189,165 @@ jobs:
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_build.sh "$(pwd)" "$(pwd)/build"
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_test.sh "$(pwd)" "$(pwd)/build"
 
+  # ------------------------------------------------------------
+  # C/C++ driver service integration tests

Review Comment:
   There's another PR refactoring this...maybe I'll merge that first so we can rebase here?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1173875047


##########
go/adbc/driver/snowflake/statement.go:
##########
@@ -0,0 +1,572 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql/driver"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+)
+
+const (
+	OptionStatementQueueSize = "adbc.flight.sql.rpc.queue_size"

Review Comment:
   hmm. should we change the option name to be more generic?



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175869535


##########
.github/workflows/native-unix.yml:
##########
@@ -189,6 +189,165 @@ jobs:
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_build.sh "$(pwd)" "$(pwd)/build"
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_test.sh "$(pwd)" "$(pwd)/build"
 
+  # ------------------------------------------------------------
+  # C/C++ driver service integration tests

Review Comment:
   Sounds good to me. I'm still waiting for Snowflake to merge the PR on their end, so let me know when that gets merged and i'll rebase this for 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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#issuecomment-1526848196

   > Actually, I should update the install and package files for this shouldn't I? or should that be a separate change?
   
   Up to you - whatever is easier


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1173874731


##########
go/adbc/driver/snowflake/driver.go:
##########
@@ -0,0 +1,150 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql"
+	"errors"
+	"runtime/debug"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+	"golang.org/x/exp/maps"
+)
+
+const (
+	infoDriverName = "ADBC Snowflake Driver - Go"
+	infoVendorName = "Snowflake"
+)
+
+var (
+	infoDriverVersion      string
+	infoDriverArrowVersion string
+	infoSupportedCodes     []adbc.InfoCode
+)
+
+func init() {
+	if info, ok := debug.ReadBuildInfo(); ok {
+		for _, dep := range info.Deps {
+			switch {
+			case dep.Path == "github.com/apache/arrow-adbc/go/adbc/driver/snowflake":
+				infoDriverVersion = dep.Version
+			case strings.HasPrefix(dep.Path, "github.com/apache/arrow/go/"):
+				infoDriverArrowVersion = dep.Version
+			}
+		}
+	}
+	// XXX: Deps not populated in tests
+	// https://github.com/golang/go/issues/33976
+	if infoDriverVersion == "" {
+		infoDriverVersion = "(unknown or development build)"
+	}
+	if infoDriverArrowVersion == "" {
+		infoDriverArrowVersion = "(unknown or development build)"
+	}
+
+	infoSupportedCodes = []adbc.InfoCode{
+		adbc.InfoDriverName,
+		adbc.InfoDriverVersion,
+		adbc.InfoDriverArrowVersion,
+		adbc.InfoVendorName,
+	}
+}
+
+func errToAdbcErr(code adbc.Status, err error) error {
+	if err == nil {
+		return nil
+	}
+
+	var e adbc.Error
+	if errors.As(err, &e) {
+		e.Code = code
+		return e
+	}
+
+	var sferr *gosnowflake.SnowflakeError
+	if errors.As(err, &sferr) {
+		var sqlstate [5]byte
+		copy(sqlstate[:], sferr.SQLState[:5])
+		return adbc.Error{
+			Code:       code,
+			Msg:        sferr.Error(),
+			VendorCode: int32(sferr.Number),
+			SqlState:   sqlstate,
+		}
+	}
+
+	return adbc.Error{
+		Msg:  err.Error(),
+		Code: code,
+	}
+}
+
+type Driver struct {
+	Alloc memory.Allocator
+}
+
+func (d Driver) NewDatabase(opts map[string]string) (adbc.Database, error) {
+	db := &database{alloc: d.Alloc}
+
+	opts = maps.Clone(opts)
+	uri, ok := opts[adbc.OptionKeyURI]
+	if ok {
+		cfg, err := gosnowflake.ParseDSN(uri)
+		if err != nil {
+			return nil, err
+		}
+
+		db.cfg = cfg
+		delete(opts, adbc.OptionKeyURI)
+	}
+
+	if db.alloc == nil {
+		db.alloc = memory.DefaultAllocator
+	}
+
+	return db, db.SetOptions(opts)
+}
+
+var drv = gosnowflake.SnowflakeDriver{}
+
+type database struct {
+	cfg   *gosnowflake.Config
+	alloc memory.Allocator
+}
+
+func (d *database) SetOptions(cnOptions map[string]string) error {
+	return nil

Review Comment:
   You know what, I completely forgot to implement the various options here, I'm gonna go do 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on pull request #586: WIP: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#issuecomment-1515321441

   @lidavidm This is ready for review now, though the CI/tests aren't going to pass until I get the SNOWFLAKE_URI secret added as per the JIRA ticket I filed. In the meantime, the rest of the code can be reviewed and looked at. Though this still relies on some PRs for the `gosnowflake` lib that haven't gotten merged yet.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade merged pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade merged PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on pull request #586: WIP: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#issuecomment-1507644849

   requires https://github.com/snowflakedb/gosnowflake/pull/769 in order to work properly


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1174223515


##########
go/adbc/driver/snowflake/statement.go:
##########
@@ -0,0 +1,572 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql/driver"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+)
+
+const (
+	OptionStatementQueueSize = "adbc.flight.sql.rpc.queue_size"

Review Comment:
   Possibly something like `adbc.???.fetch_queue_size` (if there's a namespace we're already using that can go in to `???`)



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175902174


##########
go/adbc/go.mod:
##########
@@ -22,51 +22,87 @@ go 1.18
 require (
 	github.com/apache/arrow/go/v12 v12.0.0-20230421000340-388f3a88c647
 	github.com/bluele/gcache v0.0.2
-	github.com/stretchr/testify v1.8.1
-	golang.org/x/exp v0.0.0-20230206171751-46f607a40771
+	github.com/snowflakedb/gosnowflake v1.6.19-0.20230228202026-7c7d86c6b3d0
+	github.com/stretchr/testify v1.8.2
+	golang.org/x/exp v0.0.0-20230420155640-133eef4313cb
 	golang.org/x/sync v0.1.0
-	golang.org/x/tools v0.6.0
-	google.golang.org/grpc v1.53.0
-	google.golang.org/protobuf v1.28.1
+	golang.org/x/tools v0.8.0
+	google.golang.org/grpc v1.54.0
+	google.golang.org/protobuf v1.30.0
 )
 
 require (
-	github.com/andybalholm/brotli v1.0.4 // indirect
+	github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect
+	github.com/99designs/keyring v1.2.2 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/azcore v1.5.0 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 // indirect
+	github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect
+	github.com/andybalholm/brotli v1.0.5 // indirect
+	github.com/apache/arrow/go/v11 v11.0.0 // indirect
 	github.com/apache/thrift v0.17.0 // indirect
+	github.com/aws/aws-sdk-go-v2 v1.17.8 // indirect
+	github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.10 // indirect
+	github.com/aws/aws-sdk-go-v2/credentials v1.13.20 // indirect
+	github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.63 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.32 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.26 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.24 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.11 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.27 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.26 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.1 // indirect
+	github.com/aws/aws-sdk-go-v2/service/s3 v1.32.0 // indirect
+	github.com/aws/smithy-go v1.13.5 // indirect
+	github.com/danieljoos/wincred v1.1.2 // indirect
 	github.com/davecgh/go-spew v1.1.1 // indirect
 	github.com/dustin/go-humanize v1.0.1 // indirect
-	github.com/goccy/go-json v0.10.0 // indirect
-	github.com/golang/protobuf v1.5.2 // indirect
+	github.com/dvsekhvalnov/jose2go v1.5.0 // indirect
+	github.com/form3tech-oss/jwt-go v3.2.5+incompatible // indirect
+	github.com/gabriel-vasile/mimetype v1.4.2 // indirect
+	github.com/goccy/go-json v0.10.2 // indirect
+	github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect
+	github.com/golang/protobuf v1.5.3 // indirect
 	github.com/golang/snappy v0.0.4 // indirect
-	github.com/google/flatbuffers v23.1.21+incompatible // indirect
+	github.com/google/flatbuffers v23.3.3+incompatible // indirect
 	github.com/google/uuid v1.3.0 // indirect
+	github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect
+	github.com/jmespath/go-jmespath v0.4.0 // indirect
 	github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 // indirect
 	github.com/klauspost/asmfmt v1.3.2 // indirect
-	github.com/klauspost/compress v1.15.15 // indirect
-	github.com/klauspost/cpuid/v2 v2.2.3 // indirect
+	github.com/klauspost/compress v1.16.5 // indirect
+	github.com/klauspost/cpuid/v2 v2.2.4 // indirect
 	github.com/kr/text v0.2.0 // indirect
-	github.com/mattn/go-isatty v0.0.17 // indirect
+	github.com/mattn/go-isatty v0.0.18 // indirect
 	github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect
 	github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect
+	github.com/mtibben/percent v0.2.1 // indirect
 	github.com/pierrec/lz4/v4 v4.1.17 // indirect
+	github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 // indirect
 	github.com/pmezard/go-difflib v1.0.0 // indirect
 	github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect
+	github.com/sirupsen/logrus v1.9.0 // indirect
 	github.com/zeebo/xxh3 v1.0.2 // indirect
-	golang.org/x/mod v0.8.0 // indirect
-	golang.org/x/net v0.7.0 // indirect
-	golang.org/x/sys v0.5.0 // indirect
-	golang.org/x/text v0.7.0 // indirect
+	golang.org/x/crypto v0.8.0 // indirect
+	golang.org/x/mod v0.10.0 // indirect
+	golang.org/x/net v0.9.0 // indirect
+	golang.org/x/sys v0.7.0 // indirect
+	golang.org/x/term v0.7.0 // indirect
+	golang.org/x/text v0.9.0 // indirect
 	golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
-	google.golang.org/genproto v0.0.0-20230209215440-0dfe4f8abfcc // indirect
+	gonum.org/v1/gonum v0.12.0 // indirect
+	google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect
 	gopkg.in/yaml.v3 v3.0.1 // indirect
-	lukechampine.com/uint128 v1.2.0 // indirect
+	lukechampine.com/uint128 v1.3.0 // indirect
 	modernc.org/cc/v3 v3.40.0 // indirect
 	modernc.org/ccgo/v3 v3.16.13 // indirect
-	modernc.org/libc v1.22.2 // indirect
+	modernc.org/libc v1.22.4 // indirect
 	modernc.org/mathutil v1.5.0 // indirect
 	modernc.org/memory v1.5.0 // indirect
 	modernc.org/opt v0.1.3 // indirect
-	modernc.org/sqlite v1.20.4 // indirect
+	modernc.org/sqlite v1.21.2 // indirect
 	modernc.org/strutil v1.1.3 // indirect
 	modernc.org/token v1.1.0 // indirect
 )
+
+replace github.com/snowflakedb/gosnowflake => github.com/zeroshade/gosnowflake v0.0.0-20230419185854-4870fd63961f

Review Comment:
   IIRC I manually ran and audited https://github.com/google/go-licenses which was a bit of a pain 



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175878043


##########
.github/workflows/native-unix.yml:
##########
@@ -189,6 +189,165 @@ jobs:
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_build.sh "$(pwd)" "$(pwd)/build"
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_test.sh "$(pwd)" "$(pwd)/build"
 
+  # ------------------------------------------------------------
+  # C/C++ driver service integration tests

Review Comment:
   I just merged it!



##########
go/adbc/go.mod:
##########
@@ -22,51 +22,87 @@ go 1.18
 require (
 	github.com/apache/arrow/go/v12 v12.0.0-20230421000340-388f3a88c647
 	github.com/bluele/gcache v0.0.2
-	github.com/stretchr/testify v1.8.1
-	golang.org/x/exp v0.0.0-20230206171751-46f607a40771
+	github.com/snowflakedb/gosnowflake v1.6.19-0.20230228202026-7c7d86c6b3d0
+	github.com/stretchr/testify v1.8.2
+	golang.org/x/exp v0.0.0-20230420155640-133eef4313cb
 	golang.org/x/sync v0.1.0
-	golang.org/x/tools v0.6.0
-	google.golang.org/grpc v1.53.0
-	google.golang.org/protobuf v1.28.1
+	golang.org/x/tools v0.8.0
+	google.golang.org/grpc v1.54.0
+	google.golang.org/protobuf v1.30.0
 )
 
 require (
-	github.com/andybalholm/brotli v1.0.4 // indirect
+	github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect
+	github.com/99designs/keyring v1.2.2 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/azcore v1.5.0 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 // indirect
+	github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect
+	github.com/andybalholm/brotli v1.0.5 // indirect
+	github.com/apache/arrow/go/v11 v11.0.0 // indirect
 	github.com/apache/thrift v0.17.0 // indirect
+	github.com/aws/aws-sdk-go-v2 v1.17.8 // indirect
+	github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.10 // indirect
+	github.com/aws/aws-sdk-go-v2/credentials v1.13.20 // indirect
+	github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.63 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.32 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.26 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.24 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.11 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.27 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.26 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.1 // indirect
+	github.com/aws/aws-sdk-go-v2/service/s3 v1.32.0 // indirect
+	github.com/aws/smithy-go v1.13.5 // indirect
+	github.com/danieljoos/wincred v1.1.2 // indirect
 	github.com/davecgh/go-spew v1.1.1 // indirect
 	github.com/dustin/go-humanize v1.0.1 // indirect
-	github.com/goccy/go-json v0.10.0 // indirect
-	github.com/golang/protobuf v1.5.2 // indirect
+	github.com/dvsekhvalnov/jose2go v1.5.0 // indirect
+	github.com/form3tech-oss/jwt-go v3.2.5+incompatible // indirect
+	github.com/gabriel-vasile/mimetype v1.4.2 // indirect
+	github.com/goccy/go-json v0.10.2 // indirect
+	github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect
+	github.com/golang/protobuf v1.5.3 // indirect
 	github.com/golang/snappy v0.0.4 // indirect
-	github.com/google/flatbuffers v23.1.21+incompatible // indirect
+	github.com/google/flatbuffers v23.3.3+incompatible // indirect
 	github.com/google/uuid v1.3.0 // indirect
+	github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect
+	github.com/jmespath/go-jmespath v0.4.0 // indirect
 	github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 // indirect
 	github.com/klauspost/asmfmt v1.3.2 // indirect
-	github.com/klauspost/compress v1.15.15 // indirect
-	github.com/klauspost/cpuid/v2 v2.2.3 // indirect
+	github.com/klauspost/compress v1.16.5 // indirect
+	github.com/klauspost/cpuid/v2 v2.2.4 // indirect
 	github.com/kr/text v0.2.0 // indirect
-	github.com/mattn/go-isatty v0.0.17 // indirect
+	github.com/mattn/go-isatty v0.0.18 // indirect
 	github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect
 	github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect
+	github.com/mtibben/percent v0.2.1 // indirect
 	github.com/pierrec/lz4/v4 v4.1.17 // indirect
+	github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 // indirect
 	github.com/pmezard/go-difflib v1.0.0 // indirect
 	github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect
+	github.com/sirupsen/logrus v1.9.0 // indirect
 	github.com/zeebo/xxh3 v1.0.2 // indirect
-	golang.org/x/mod v0.8.0 // indirect
-	golang.org/x/net v0.7.0 // indirect
-	golang.org/x/sys v0.5.0 // indirect
-	golang.org/x/text v0.7.0 // indirect
+	golang.org/x/crypto v0.8.0 // indirect
+	golang.org/x/mod v0.10.0 // indirect
+	golang.org/x/net v0.9.0 // indirect
+	golang.org/x/sys v0.7.0 // indirect
+	golang.org/x/term v0.7.0 // indirect
+	golang.org/x/text v0.9.0 // indirect
 	golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
-	google.golang.org/genproto v0.0.0-20230209215440-0dfe4f8abfcc // indirect
+	gonum.org/v1/gonum v0.12.0 // indirect
+	google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect
 	gopkg.in/yaml.v3 v3.0.1 // indirect
-	lukechampine.com/uint128 v1.2.0 // indirect
+	lukechampine.com/uint128 v1.3.0 // indirect
 	modernc.org/cc/v3 v3.40.0 // indirect
 	modernc.org/ccgo/v3 v3.16.13 // indirect
-	modernc.org/libc v1.22.2 // indirect
+	modernc.org/libc v1.22.4 // indirect
 	modernc.org/mathutil v1.5.0 // indirect
 	modernc.org/memory v1.5.0 // indirect
 	modernc.org/opt v0.1.3 // indirect
-	modernc.org/sqlite v1.20.4 // indirect
+	modernc.org/sqlite v1.21.2 // indirect
 	modernc.org/strutil v1.1.3 // indirect
 	modernc.org/token v1.1.0 // indirect
 )
+
+replace github.com/snowflakedb/gosnowflake => github.com/zeroshade/gosnowflake v0.0.0-20230419185854-4870fd63961f

Review Comment:
   We should also re-run the dependency analyzer and make any necessary updates.



##########
docs/source/driver/go/snowflake.rst:
##########
@@ -0,0 +1,325 @@
+.. 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.
+
+================
+Snowflake Driver
+================
+
+The Snowflake Driver provides access to Snowflake Database Warehouses.
+
+Installation
+============
+
+The Snowflake Driver is shipped as a standalone library
+
+.. tab-set::
+
+  .. tab-item:: Go
+    :sync: go
+
+    .. code-block:: shell
+
+      go get github.com/apache/arrow-adbc/go/adbc/driver/snowflake
+
+Usage
+=====
+
+To connect to a Snowflake database you can supply the "uri" parameter when
+constructing the :cpp::class:`AdbcDatabase`.
+
+.. tab-set::
+
+  .. tab-item:: C++
+    :sync: cpp
+
+    .. code-block:: cpp
+
+      #include "adbc.h"
+
+      // Ignoring error handling
+      struct AdbcDatabase database;
+      AdbcDatabaseNew(&database, nullptr);
+      AdbcDatabaseSetOption(&database, "driver", "adbc_driver_snowflake", nullptr);
+      AdbcDatabaseSetOption(&database, "uri", "<snowflake uri>", nullptr);
+      AdbcDatabaseInit(&database, nullptr);
+
+URI Format
+----------
+
+The Snowflake URI should be of one of the following formats:
+
+- ``user[:password]@account/database/schema[?param1=value1&paramN=valueN]``
+- ``user[:password]@account/database[?param1=value1&paramN=valueN]``
+- ``user[:password]@host:port/database/schema?account=user_account[&param1=value1&paramN=valueN]``
+- ``host:port/database/schema?account=user_account[&param1=value1&paramN=valueN]``
+
+Alternately, instead of providing a full URI, the configuration can
+be entirely supplied using the other available options or some combination
+of the URI and other options. If a URI is provided, it will be parsed first
+and any explicit options provided will override anything parsed from the URI.
+
+Supported Features
+==================
+
+The Snowflake driver generally supports features defined in the ADBC API
+specification 1.0.0, as well as some additional, custom options.
+
+Authentication
+--------------
+
+Snowflake requires some form of authentication to be enabled. By default
+it will attempt to use Username/Password authentication. The username and
+password can be provided in the URI or via the ``username`` and ``password``
+options to the :cpp:class:`AdbcDatabase`.
+
+Alternately, other types of authentication can be specified and customized.
+See "Client Options" below.
+
+Bulk Ingestion
+--------------
+
+Bulk ingestion is supported. The mapping from Arrow types to Snowflake types
+is provided below.
+
+Partitioned Result Sets
+-----------------------
+
+Partitioned result sets are not currently supported.
+
+Performance
+-----------
+
+Formal benchmarking is forthcoming. Snowflake does provide an Arrow native
+format for requesting results, but bulk ingestion is still currently executed
+using the REST API. As described in the `Snowflake Documentation
+<https://pkg.go.dev/github.com/snowflakedb/gosnowflake#hdr-Batch_Inserts_and_Binding_Parameters>`
+the driver will potentially attempt to improve performance by streaming the data
+(without creating files on the local machine) to a temporary stage for ingestion
+if the number of values exceeds some threshold.
+
+In order for the driver to leverage this temporary stage, the user must have
+the ``CREATE STAGE`` privilege on the schema. If the user does not have this
+privilege, the driver will fall back to sending the data with the query
+to the snowflake database.
+
+In addition, the current database and schema for the session must be set. If
+these are not set, the ``CREATE TEMPORARY STAGE`` command executed by the driver
+can fail with the following error:
+
+.. code-block::
+  CREATE TEMPORARY STAGE SYSTEM$BIND file_format=(type=csv field_optionally_enclosed_by='"')
+  CANNOT perform CREATE STAGE. This session does not have a current schema. Call 'USE SCHEMA' or use a qualified name.
+
+In addition, results are potentially fetched in parallel from multiple endpoints.
+A limited number of batches are queued per endpoint, though data is always
+returned to the client in the order of the endpoints.
+
+The queue size can be changed by setting an option on the :cpp:class:`AdbcStatement`:
+
+``adbc.sql.results.rpc.queue_size``
+    The number of batches to queue per endpoint. Defaults to 5.
+
+Transactions
+------------
+
+Transactions are supported. Keep in mind that Snowflake transactions will
+implicitly commit if any DDL statements are run, such as ``CREATE TABLE``.
+
+Client Options
+--------------
+
+The options used for creating a Snowflake Database connection can be customized.
+These options map 1:1 with the Snowflake `Config object <https://pkg.go.dev/github.com/snowflakedb/gosnowflake#Config>`.
+
+``adbc.snowflake.sql.db``
+    The database this session should default to using.
+
+``adbc.snowflake.sql.schema``
+    The schema this session should default to using.
+
+``adbc.snowflake.sql.warehouse``
+    The warehouse this session should default to using.
+
+``adbc.snowflake.sql.role``
+    The role that should be used for authentication.
+
+``adbc.snowflake.sql.region``
+    The Snowflake region to use for constructing the connection URI.
+
+``adbc.snowflake.sql.account``
+    The Snowflake account that should be used for authentication and building the
+    connection URI.
+
+``adbc.snowflake.sql.uri.protocol``
+    This should be either `http` or `https`.
+
+``adbc.snowflake.sql.uri.port``
+    The port to use for constructing the URI for connection.
+
+``adbc.snowflake.sql.uri.host``
+    The explicit host to use for constructing the URL to connect to.
+
+``adbc.snowflake.sql.auth_type``
+    Allows specifying alternate types of authentication, the allowed values are:
+
+    - ``auth_snowflake``: General username/password authentication (this is the default)
+    - ``auth_oauth``: Use OAuth authentication for the snowflake connection.
+    - ``auth_ext_browser``: Use an external browser to access a FED and perform SSO auth.
+    - ``auth_okta``: Use a native Okta URL to perform SSO authentication using Okta
+    - ``auth_jwt``: Use a provided JWT to perform authentication.
+    - ``auth_mfa``: Use a username and password with MFA.
+
+``adbc.snowflake.sql.client_option.auth_token``
+    If using OAuth or another form of authentication, this option is how you can
+    explicitly specify the token to be used for connection.
+
+``adbc.snowflake.sql.client_option.okta_url``
+    If using ``auth_okta``, this option is required in order to specify the
+    Okta URL to connect to for SSO authentication.
+
+``adbc.snowflake.sql.client_option.login_timeout``
+    Specify login retry timeout *excluding* network roundtrip and reading http responses.
+    Value should be formatted as described `here <https://pkg.go.dev/time#ParseDuration>`,
+    such as ``300ms``, ``1.5s`` or ``1m30s``. Even though negative values are accepted,
+    the absolute value of such a duration will be used.
+
+``adbc.snowflake.sql.client_option.request_timeout``
+    Specify request retry timeout *excluding* network roundtrip and reading http responses.
+    Value should be formatted as described `here <https://pkg.go.dev/time#ParseDuration>`,
+    such as ``300ms``, ``1.5s`` or ``1m30s``. Even though negative values are accepted,
+    the absolute value of such a duration will be used.
+
+``adbc.snowflake.sql.client_option.jwt_expire_timeout``
+    JWT expiration will occur after this timeout.
+    Value should be formatted as described `here <https://pkg.go.dev/time#ParseDuration>`,
+    such as ``300ms``, ``1.5s`` or ``1m30s``. Even though negative values are accepted,
+    the absolute value of such a duration will be used.
+
+``adbc.snowflake.sql.client_option.client_timeout``
+    Specify timeout for network roundtrip and reading http responses.
+    Value should be formatted as described `here <https://pkg.go.dev/time#ParseDuration>`,
+    such as ``300ms``, ``1.5s`` or ``1m30s``. Even though negative values are accepted,
+    the absolute value of such a duration will be used.
+
+``adbc.snowflake.sql.client_option.app_name``
+    Allows specifying the Application Name to Snowflake for the connection.
+
+``adbc.snowflake.sql.client_option.tls_skip_verify``
+    Disable verification of the server's TLS certificate. Value should be ``true``
+    or ``false``.
+
+``adbc.snowflake.sql.client_option.ocsp_fail_open_mode``
+    Control the fail open mode for OCSP. Default is ``true``. Value should
+    be either ``true`` or ``false``.
+
+``adbc.snowflake.sql.client_option.keep_session_alive``
+    Enable the session to persist even after the connection is closed. Value
+    should be either ``true`` or ``false``.
+
+``adbc.snowflake.sql.client_option.jwt_private_key``
+    Specify the RSA private key which should be used to sign the JWT for
+    authentication. This should be a path to a file containing a PKCS1
+    private key to be read in and parsed. Commonly encoded in PEM blocks
+    of type "RSA PRIVATE KEY".
+
+``adbc.snowflake.sql.client_option.disable_telemetry``
+    The Snowflake driver allows for telemetry information which can be
+    disabled by setting this to ``true``. Value should be either ``true``
+    or ``false``.
+
+``adbc.snowflake.sql.client_option.tracing``
+    Set the logging level
+
+``adbc.snowflake.sql.client_option.cache_mfa_token``
+    When ``true``, the MFA token is cached in the credential manager. Defaults
+    to ``true`` on Windows/OSX, ``false`` on Linux.
+
+``adbc.snowflake.sql.client_option.store_temp_creds``
+    When ``true``, the ID token is cached in the credential manager. Defaults
+    to ``true`` on Windows/OSX, ``false`` on Linux.
+
+
+Metadata
+--------
+
+When calling :cpp:`AdbcConnectionGetTableSchema`, the returned Arrow Schema
+will contain metadata on each field:

Review Comment:
   Maybe we should eventually standardize something like Flight SQL's scheme for encoding this sort of info...



##########
go/adbc/go.mod:
##########
@@ -22,51 +22,87 @@ go 1.18
 require (
 	github.com/apache/arrow/go/v12 v12.0.0-20230421000340-388f3a88c647
 	github.com/bluele/gcache v0.0.2
-	github.com/stretchr/testify v1.8.1
-	golang.org/x/exp v0.0.0-20230206171751-46f607a40771
+	github.com/snowflakedb/gosnowflake v1.6.19-0.20230228202026-7c7d86c6b3d0
+	github.com/stretchr/testify v1.8.2
+	golang.org/x/exp v0.0.0-20230420155640-133eef4313cb
 	golang.org/x/sync v0.1.0
-	golang.org/x/tools v0.6.0
-	google.golang.org/grpc v1.53.0
-	google.golang.org/protobuf v1.28.1
+	golang.org/x/tools v0.8.0
+	google.golang.org/grpc v1.54.0
+	google.golang.org/protobuf v1.30.0
 )
 
 require (
-	github.com/andybalholm/brotli v1.0.4 // indirect
+	github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect
+	github.com/99designs/keyring v1.2.2 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/azcore v1.5.0 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 // indirect
+	github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect
+	github.com/andybalholm/brotli v1.0.5 // indirect
+	github.com/apache/arrow/go/v11 v11.0.0 // indirect
 	github.com/apache/thrift v0.17.0 // indirect
+	github.com/aws/aws-sdk-go-v2 v1.17.8 // indirect
+	github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.10 // indirect
+	github.com/aws/aws-sdk-go-v2/credentials v1.13.20 // indirect
+	github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.63 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.32 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.26 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.24 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.11 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.27 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.26 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.1 // indirect
+	github.com/aws/aws-sdk-go-v2/service/s3 v1.32.0 // indirect
+	github.com/aws/smithy-go v1.13.5 // indirect
+	github.com/danieljoos/wincred v1.1.2 // indirect
 	github.com/davecgh/go-spew v1.1.1 // indirect
 	github.com/dustin/go-humanize v1.0.1 // indirect
-	github.com/goccy/go-json v0.10.0 // indirect
-	github.com/golang/protobuf v1.5.2 // indirect
+	github.com/dvsekhvalnov/jose2go v1.5.0 // indirect
+	github.com/form3tech-oss/jwt-go v3.2.5+incompatible // indirect
+	github.com/gabriel-vasile/mimetype v1.4.2 // indirect
+	github.com/goccy/go-json v0.10.2 // indirect
+	github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect
+	github.com/golang/protobuf v1.5.3 // indirect
 	github.com/golang/snappy v0.0.4 // indirect
-	github.com/google/flatbuffers v23.1.21+incompatible // indirect
+	github.com/google/flatbuffers v23.3.3+incompatible // indirect
 	github.com/google/uuid v1.3.0 // indirect
+	github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect
+	github.com/jmespath/go-jmespath v0.4.0 // indirect
 	github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 // indirect
 	github.com/klauspost/asmfmt v1.3.2 // indirect
-	github.com/klauspost/compress v1.15.15 // indirect
-	github.com/klauspost/cpuid/v2 v2.2.3 // indirect
+	github.com/klauspost/compress v1.16.5 // indirect
+	github.com/klauspost/cpuid/v2 v2.2.4 // indirect
 	github.com/kr/text v0.2.0 // indirect
-	github.com/mattn/go-isatty v0.0.17 // indirect
+	github.com/mattn/go-isatty v0.0.18 // indirect
 	github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect
 	github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect
+	github.com/mtibben/percent v0.2.1 // indirect
 	github.com/pierrec/lz4/v4 v4.1.17 // indirect
+	github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 // indirect
 	github.com/pmezard/go-difflib v1.0.0 // indirect
 	github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect
+	github.com/sirupsen/logrus v1.9.0 // indirect
 	github.com/zeebo/xxh3 v1.0.2 // indirect
-	golang.org/x/mod v0.8.0 // indirect
-	golang.org/x/net v0.7.0 // indirect
-	golang.org/x/sys v0.5.0 // indirect
-	golang.org/x/text v0.7.0 // indirect
+	golang.org/x/crypto v0.8.0 // indirect
+	golang.org/x/mod v0.10.0 // indirect
+	golang.org/x/net v0.9.0 // indirect
+	golang.org/x/sys v0.7.0 // indirect
+	golang.org/x/term v0.7.0 // indirect
+	golang.org/x/text v0.9.0 // indirect
 	golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
-	google.golang.org/genproto v0.0.0-20230209215440-0dfe4f8abfcc // indirect
+	gonum.org/v1/gonum v0.12.0 // indirect
+	google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect
 	gopkg.in/yaml.v3 v3.0.1 // indirect
-	lukechampine.com/uint128 v1.2.0 // indirect
+	lukechampine.com/uint128 v1.3.0 // indirect
 	modernc.org/cc/v3 v3.40.0 // indirect
 	modernc.org/ccgo/v3 v3.16.13 // indirect
-	modernc.org/libc v1.22.2 // indirect
+	modernc.org/libc v1.22.4 // indirect
 	modernc.org/mathutil v1.5.0 // indirect
 	modernc.org/memory v1.5.0 // indirect
 	modernc.org/opt v0.1.3 // indirect
-	modernc.org/sqlite v1.20.4 // indirect
+	modernc.org/sqlite v1.21.2 // indirect
 	modernc.org/strutil v1.1.3 // indirect
 	modernc.org/token v1.1.0 // indirect
 )
+
+replace github.com/snowflakedb/gosnowflake => github.com/zeroshade/gosnowflake v0.0.0-20230419185854-4870fd63961f

Review Comment:
   https://infra.apache.org/licensing-howto.html#alv2-dep
   
   gosnowflake is Apache-2 without a NOTICE file, but we should still update LICENSE.txt (and its copies...) just for completeness



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on pull request #586: WIP: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#issuecomment-1508071912

   Cool!
   
   Let me know if you need help adding secrets (but the process should just be file INFRA Jira, then email to private@apache.org when you get the go-ahead, in case you haven't done it before)


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on a diff in pull request #586: WIP: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1171911006


##########
go/adbc/driver/snowflake/driver.go:
##########
@@ -0,0 +1,150 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql"
+	"errors"
+	"runtime/debug"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+	"golang.org/x/exp/maps"
+)
+
+const (
+	infoDriverName = "ADBC Snowflake Driver - Go"
+	infoVendorName = "Snowflake"
+)
+
+var (
+	infoDriverVersion      string
+	infoDriverArrowVersion string
+	infoSupportedCodes     []adbc.InfoCode
+)
+
+func init() {
+	if info, ok := debug.ReadBuildInfo(); ok {
+		for _, dep := range info.Deps {
+			switch {
+			case dep.Path == "github.com/apache/arrow-adbc/go/adbc/driver/snowflake":
+				infoDriverVersion = dep.Version
+			case strings.HasPrefix(dep.Path, "github.com/apache/arrow/go/"):
+				infoDriverArrowVersion = dep.Version
+			}
+		}
+	}
+	// XXX: Deps not populated in tests
+	// https://github.com/golang/go/issues/33976
+	if infoDriverVersion == "" {
+		infoDriverVersion = "(unknown or development build)"
+	}
+	if infoDriverArrowVersion == "" {
+		infoDriverArrowVersion = "(unknown or development build)"
+	}
+
+	infoSupportedCodes = []adbc.InfoCode{
+		adbc.InfoDriverName,
+		adbc.InfoDriverVersion,
+		adbc.InfoDriverArrowVersion,
+		adbc.InfoVendorName,
+	}
+}
+
+func errToAdbcErr(code adbc.Status, err error) error {
+	if err == nil {
+		return nil
+	}
+
+	var e adbc.Error
+	if errors.As(err, &e) {
+		e.Code = code
+		return e
+	}
+
+	var sferr *gosnowflake.SnowflakeError
+	if errors.As(err, &sferr) {
+		var sqlstate [5]byte
+		copy(sqlstate[:], sferr.SQLState[:5])
+		return adbc.Error{
+			Code:       code,
+			Msg:        sferr.Error(),
+			VendorCode: int32(sferr.Number),
+			SqlState:   sqlstate,
+		}
+	}
+
+	return adbc.Error{
+		Msg:  err.Error(),
+		Code: code,
+	}
+}
+
+type Driver struct {
+	Alloc memory.Allocator
+}
+
+func (d Driver) NewDatabase(opts map[string]string) (adbc.Database, error) {
+	db := &database{alloc: d.Alloc}
+
+	opts = maps.Clone(opts)
+	uri, ok := opts[adbc.OptionKeyURI]
+	if ok {
+		cfg, err := gosnowflake.ParseDSN(uri)
+		if err != nil {
+			return nil, err
+		}
+
+		db.cfg = cfg
+		delete(opts, adbc.OptionKeyURI)
+	}
+
+	if db.alloc == nil {
+		db.alloc = memory.DefaultAllocator
+	}
+
+	return db, db.SetOptions(opts)
+}
+
+var drv = gosnowflake.SnowflakeDriver{}
+
+type database struct {
+	cfg   *gosnowflake.Config
+	alloc memory.Allocator
+}
+
+func (d *database) SetOptions(cnOptions map[string]string) error {
+	return nil

Review Comment:
   Hmm, we don't need an option for authentication credentials here?



##########
go/adbc/driver/snowflake/statement.go:
##########
@@ -0,0 +1,572 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql/driver"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+)
+
+const (
+	OptionStatementQueueSize = "adbc.flight.sql.rpc.queue_size"

Review Comment:
   Do we want to reuse the exact same option name? (Maybe that's for the better.)



##########
go/adbc/driver/snowflake/statement.go:
##########
@@ -0,0 +1,572 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql/driver"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+)
+
+const (
+	OptionStatementQueueSize = "adbc.flight.sql.rpc.queue_size"
+)
+
+type statement struct {
+	cnxn      *cnxn
+	alloc     memory.Allocator
+	queueSize int
+
+	query       string
+	targetTable string
+	append      bool
+
+	bound      arrow.Record
+	streamBind array.RecordReader
+}
+
+// Close releases any relevant resources associated with this statement
+// and closes it (particularly if it is a prepared statement).
+//
+// A statement instance should not be used after Close is called.
+func (st *statement) Close() error {
+	if st.cnxn == nil {
+		return adbc.Error{
+			Msg:  "statement already closed",
+			Code: adbc.StatusInvalidState}
+	}
+
+	if st.bound != nil {
+		st.bound.Release()
+		st.bound = nil
+	} else if st.streamBind != nil {
+		st.streamBind.Release()
+		st.streamBind = nil
+	}
+	st.cnxn = nil
+	return nil
+}
+
+// SetOption sets a string option on this statement
+func (st *statement) SetOption(key string, val string) error {
+	switch key {
+	case adbc.OptionKeyIngestTargetTable:
+		st.query = ""
+		st.targetTable = val
+	case adbc.OptionKeyIngestMode:
+		switch val {
+		case adbc.OptionValueIngestModeAppend:
+			st.append = true
+		case adbc.OptionValueIngestModeCreate:
+			st.append = false
+		default:
+			return adbc.Error{
+				Msg:  fmt.Sprintf("invalid statement option %s=%s", key, val),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+	default:
+		return adbc.Error{
+			Msg:  fmt.Sprintf("invalid statement option %s=%s", key, val),
+			Code: adbc.StatusInvalidArgument,
+		}
+	}
+	return nil
+}
+
+// SetSqlQuery sets the query string to be executed.
+//
+// The query can then be executed with any of the Execute methods.
+// For queries expected to be executed repeatedly, Prepare should be
+// called before execution.
+func (st *statement) SetSqlQuery(query string) error {
+	st.query = query
+	st.targetTable = ""
+	return nil
+}
+
+func toSnowflakeType(dt arrow.DataType) string {
+	switch dt.ID() {
+	case arrow.EXTENSION:
+		return toSnowflakeType(dt.(arrow.ExtensionType).StorageType())
+	case arrow.DICTIONARY:
+		return toSnowflakeType(dt.(*arrow.DictionaryType).ValueType)
+	case arrow.RUN_END_ENCODED:
+		return toSnowflakeType(dt.(*arrow.RunEndEncodedType).Encoded())
+	case arrow.INT8, arrow.INT16, arrow.INT32, arrow.INT64,
+		arrow.UINT8, arrow.UINT16, arrow.UINT32, arrow.UINT64:
+		return "integer"
+	case arrow.FLOAT32, arrow.FLOAT16, arrow.FLOAT64:
+		return "double"
+	case arrow.DECIMAL, arrow.DECIMAL256:
+		dec := dt.(arrow.DecimalType)
+		return fmt.Sprintf("NUMERIC(%d,%d)", dec.GetPrecision(), dec.GetScale())
+	case arrow.STRING, arrow.LARGE_STRING:
+		return "text"
+	case arrow.BINARY, arrow.LARGE_BINARY:
+		return "binary"
+	case arrow.FIXED_SIZE_BINARY:
+		fsb := dt.(*arrow.FixedSizeBinaryType)
+		return fmt.Sprintf("binary(%d)", fsb.ByteWidth)
+	case arrow.BOOL:
+		return "boolean"
+	case arrow.TIME32, arrow.TIME64:
+		t := dt.(arrow.TemporalWithUnit)
+		prec := int(t.TimeUnit()) * 3
+		return fmt.Sprintf("time(%d)", prec)
+	case arrow.DATE32, arrow.DATE64:
+		return "date"
+	case arrow.TIMESTAMP:
+		ts := dt.(*arrow.TimestampType)
+		prec := int(ts.Unit) * 3
+		if ts.TimeZone == "" {
+			return fmt.Sprintf("timestamp_tz(%d)", prec)
+		}
+		return fmt.Sprintf("timestamp_ltz(%d)", prec)
+	case arrow.DENSE_UNION, arrow.SPARSE_UNION:
+		return "variant"
+	case arrow.LIST, arrow.LARGE_LIST, arrow.FIXED_SIZE_LIST:
+		return "array"
+	case arrow.STRUCT, arrow.MAP:
+		return "object"
+	}
+
+	return ""
+}
+
+func (st *statement) initIngest(ctx context.Context) (string, error) {
+	var (
+		createBldr, insertBldr strings.Builder
+	)
+
+	createBldr.WriteString("CREATE TABLE ")
+	createBldr.WriteString(st.targetTable)

Review Comment:
   Does the Snowflake driver provide a string escaper?



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1178117292


##########
LICENSE.txt:
##########
@@ -356,46 +354,137 @@ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
 copies of the Software, and to permit persons to whom the Software is
 furnished to do so, subject to the following conditions:
 
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
 
 THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
 IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
 FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
 AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
 LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE
 
 --------------------------------------------------------------------------------
 
-3rdparty dependency github.com/davecgh/go-spew/spew
+3rdparty dependency github.com/Azure/azure-sdk-for-go/sdk/internal
 is statically linked in certain binary distributions, like the Python wheels.
-github.com/davecgh/go-spew/spew is under the ISC license.
-ISC License
+github.com/Azure/azure-sdk-for-go/sdk/internal is under the MIT license.
+MIT License
 
-Copyright (c) 2012-2016 Dave Collins <da...@davec.name>
+Copyright (c) Microsoft Corporation.
 
-Permission to use, copy, modify, and/or distribute this software for any
-purpose with or without fee is hereby granted, provided that the above
-copyright notice and this permission notice appear in all copies.
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
 
-THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
-WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
-MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
-ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
-WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
-ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
-OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE
 
 --------------------------------------------------------------------------------
 
-3rdparty dependency github.com/goccy/go-json
+3rdparty dependency github.com/Azure/azure-sdk-for-go/sdk/storage/azblob
 is statically linked in certain binary distributions, like the Python wheels.
-github.com/goccy/go-json is under the MIT license.
+github.com/Azure/azure-sdk-for-go/sdk/storage/azblob is under the MIT license.
+    MIT License
+
+    Copyright (c) Microsoft Corporation. All rights reserved.
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in all
+    copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+    SOFTWARE
+--------------------------------------------------------------------------------
+
+3rdparty dependency github.com/JohnCGriffin/overflow
+is statically linked in certain binary distributions, like the Python wheels.
+github.com/JohnCGriffin/overflow is under the MIT license.
+[![Build Status](https://travis-ci.org/JohnCGriffin/overflow.png)](https://travis-ci.org/JohnCGriffin/overflow)
+# overflow
+Check for int/int8/int16/int64/int32 integer overflow in Golang arithmetic.
+### Install

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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175933282


##########
go/adbc/go.mod:
##########
@@ -22,51 +22,87 @@ go 1.18
 require (
 	github.com/apache/arrow/go/v12 v12.0.0-20230421000340-388f3a88c647
 	github.com/bluele/gcache v0.0.2
-	github.com/stretchr/testify v1.8.1
-	golang.org/x/exp v0.0.0-20230206171751-46f607a40771
+	github.com/snowflakedb/gosnowflake v1.6.19-0.20230228202026-7c7d86c6b3d0
+	github.com/stretchr/testify v1.8.2
+	golang.org/x/exp v0.0.0-20230420155640-133eef4313cb
 	golang.org/x/sync v0.1.0
-	golang.org/x/tools v0.6.0
-	google.golang.org/grpc v1.53.0
-	google.golang.org/protobuf v1.28.1
+	golang.org/x/tools v0.8.0
+	google.golang.org/grpc v1.54.0
+	google.golang.org/protobuf v1.30.0
 )
 
 require (
-	github.com/andybalholm/brotli v1.0.4 // indirect
+	github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect
+	github.com/99designs/keyring v1.2.2 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/azcore v1.5.0 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 // indirect
+	github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect
+	github.com/andybalholm/brotli v1.0.5 // indirect
+	github.com/apache/arrow/go/v11 v11.0.0 // indirect
 	github.com/apache/thrift v0.17.0 // indirect
+	github.com/aws/aws-sdk-go-v2 v1.17.8 // indirect
+	github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.10 // indirect
+	github.com/aws/aws-sdk-go-v2/credentials v1.13.20 // indirect
+	github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.63 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.32 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.26 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.24 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.11 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.27 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.26 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.1 // indirect
+	github.com/aws/aws-sdk-go-v2/service/s3 v1.32.0 // indirect
+	github.com/aws/smithy-go v1.13.5 // indirect
+	github.com/danieljoos/wincred v1.1.2 // indirect
 	github.com/davecgh/go-spew v1.1.1 // indirect
 	github.com/dustin/go-humanize v1.0.1 // indirect
-	github.com/goccy/go-json v0.10.0 // indirect
-	github.com/golang/protobuf v1.5.2 // indirect
+	github.com/dvsekhvalnov/jose2go v1.5.0 // indirect
+	github.com/form3tech-oss/jwt-go v3.2.5+incompatible // indirect
+	github.com/gabriel-vasile/mimetype v1.4.2 // indirect
+	github.com/goccy/go-json v0.10.2 // indirect
+	github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect
+	github.com/golang/protobuf v1.5.3 // indirect
 	github.com/golang/snappy v0.0.4 // indirect
-	github.com/google/flatbuffers v23.1.21+incompatible // indirect
+	github.com/google/flatbuffers v23.3.3+incompatible // indirect
 	github.com/google/uuid v1.3.0 // indirect
+	github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect
+	github.com/jmespath/go-jmespath v0.4.0 // indirect
 	github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 // indirect
 	github.com/klauspost/asmfmt v1.3.2 // indirect
-	github.com/klauspost/compress v1.15.15 // indirect
-	github.com/klauspost/cpuid/v2 v2.2.3 // indirect
+	github.com/klauspost/compress v1.16.5 // indirect
+	github.com/klauspost/cpuid/v2 v2.2.4 // indirect
 	github.com/kr/text v0.2.0 // indirect
-	github.com/mattn/go-isatty v0.0.17 // indirect
+	github.com/mattn/go-isatty v0.0.18 // indirect
 	github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect
 	github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect
+	github.com/mtibben/percent v0.2.1 // indirect
 	github.com/pierrec/lz4/v4 v4.1.17 // indirect
+	github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 // indirect
 	github.com/pmezard/go-difflib v1.0.0 // indirect
 	github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect
+	github.com/sirupsen/logrus v1.9.0 // indirect
 	github.com/zeebo/xxh3 v1.0.2 // indirect
-	golang.org/x/mod v0.8.0 // indirect
-	golang.org/x/net v0.7.0 // indirect
-	golang.org/x/sys v0.5.0 // indirect
-	golang.org/x/text v0.7.0 // indirect
+	golang.org/x/crypto v0.8.0 // indirect
+	golang.org/x/mod v0.10.0 // indirect
+	golang.org/x/net v0.9.0 // indirect
+	golang.org/x/sys v0.7.0 // indirect
+	golang.org/x/term v0.7.0 // indirect
+	golang.org/x/text v0.9.0 // indirect
 	golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
-	google.golang.org/genproto v0.0.0-20230209215440-0dfe4f8abfcc // indirect
+	gonum.org/v1/gonum v0.12.0 // indirect
+	google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect
 	gopkg.in/yaml.v3 v3.0.1 // indirect
-	lukechampine.com/uint128 v1.2.0 // indirect
+	lukechampine.com/uint128 v1.3.0 // indirect
 	modernc.org/cc/v3 v3.40.0 // indirect
 	modernc.org/ccgo/v3 v3.16.13 // indirect
-	modernc.org/libc v1.22.2 // indirect
+	modernc.org/libc v1.22.4 // indirect
 	modernc.org/mathutil v1.5.0 // indirect
 	modernc.org/memory v1.5.0 // indirect
 	modernc.org/opt v0.1.3 // indirect
-	modernc.org/sqlite v1.20.4 // indirect
+	modernc.org/sqlite v1.21.2 // indirect
 	modernc.org/strutil v1.1.3 // indirect
 	modernc.org/token v1.1.0 // indirect
 )
+
+replace github.com/snowflakedb/gosnowflake => github.com/zeroshade/gosnowflake v0.0.0-20230419185854-4870fd63961f

Review Comment:
   Updated + added instructions to CONTRIBUTING.md for how to easily re-generate 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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#issuecomment-1526414854

   Actually, I should update the install and package files for this shouldn't I? or should that be a separate change?


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1177304833


##########
.github/workflows/integration.yml:
##########
@@ -208,3 +208,161 @@ jobs:
           ADBC_POSTGRESQL_TEST_URI: "postgres://localhost:5432/postgres?user=postgres&password=password"
         run: |
           ./ci/scripts/python_test.sh "$(pwd)" "$(pwd)/build"
+
+  # ------------------------------------------------------------
+  # C/C++ driver service integration tests
+  # ------------------------------------------------------------
+  drivers-integration-conda:
+    name: "C/C++ (Conda) Driver Service Integration Tests"

Review Comment:
   ```suggestion
     snowflake:
       name: "Snowflake Integration Tests"
   ```



##########
.github/workflows/integration.yml:
##########
@@ -208,3 +208,161 @@ jobs:
           ADBC_POSTGRESQL_TEST_URI: "postgres://localhost:5432/postgres?user=postgres&password=password"
         run: |
           ./ci/scripts/python_test.sh "$(pwd)" "$(pwd)/build"
+
+  # ------------------------------------------------------------
+  # C/C++ driver service integration tests
+  # ------------------------------------------------------------
+  drivers-integration-conda:
+    name: "C/C++ (Conda) Driver Service Integration Tests"
+    runs-on: ubuntu-latest
+    services:
+      postgres:

Review Comment:
   Remove all the postgres/dremio stuff? They're separate pipelines now



##########
c/validation/adbc_validation.cc:
##########
@@ -53,6 +54,15 @@ namespace {
   } while (false)
 }  // namespace
 
+/// case insensitive string compare
+bool iequals(std::string s1, std::string s2) {

Review Comment:
   move this into the anonymous namespace?



##########
LICENSE.txt:
##########
@@ -356,46 +354,137 @@ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
 copies of the Software, and to permit persons to whom the Software is
 furnished to do so, subject to the following conditions:
 
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
 
 THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
 IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
 FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
 AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
 LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE
 
 --------------------------------------------------------------------------------
 
-3rdparty dependency github.com/davecgh/go-spew/spew
+3rdparty dependency github.com/Azure/azure-sdk-for-go/sdk/internal
 is statically linked in certain binary distributions, like the Python wheels.
-github.com/davecgh/go-spew/spew is under the ISC license.
-ISC License
+github.com/Azure/azure-sdk-for-go/sdk/internal is under the MIT license.
+MIT License
 
-Copyright (c) 2012-2016 Dave Collins <da...@davec.name>
+Copyright (c) Microsoft Corporation.
 
-Permission to use, copy, modify, and/or distribute this software for any
-purpose with or without fee is hereby granted, provided that the above
-copyright notice and this permission notice appear in all copies.
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
 
-THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
-WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
-MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
-ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
-WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
-ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
-OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE
 
 --------------------------------------------------------------------------------
 
-3rdparty dependency github.com/goccy/go-json
+3rdparty dependency github.com/Azure/azure-sdk-for-go/sdk/storage/azblob
 is statically linked in certain binary distributions, like the Python wheels.
-github.com/goccy/go-json is under the MIT license.
+github.com/Azure/azure-sdk-for-go/sdk/storage/azblob is under the MIT license.
+    MIT License
+
+    Copyright (c) Microsoft Corporation. All rights reserved.
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in all
+    copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+    SOFTWARE
+--------------------------------------------------------------------------------
+
+3rdparty dependency github.com/JohnCGriffin/overflow
+is statically linked in certain binary distributions, like the Python wheels.
+github.com/JohnCGriffin/overflow is under the MIT license.
+[![Build Status](https://travis-ci.org/JohnCGriffin/overflow.png)](https://travis-ci.org/JohnCGriffin/overflow)
+# overflow
+Check for int/int8/int16/int64/int32 integer overflow in Golang arithmetic.
+### Install

Review Comment:
   We may want to manually audit this, else it sorta defeats the purpose...for instance, here, we included the entire readme



##########
c/validation/adbc_validation.cc:
##########
@@ -32,6 +32,7 @@
 #include <gtest/gtest-matchers.h>
 #include <gtest/gtest.h>
 #include <nanoarrow/nanoarrow.h>
+#include <algorithm>

Review Comment:
   nit, but sort this with the other stdlib headers



##########
c/validation/adbc_validation.cc:
##########
@@ -53,6 +54,15 @@ namespace {
   } while (false)
 }  // namespace
 
+/// case insensitive string compare
+bool iequals(std::string s1, std::string s2) {

Review Comment:
   also, try to use string_view here or at least const string&



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175761012


##########
go/adbc/driver/snowflake/statement.go:
##########
@@ -0,0 +1,572 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql/driver"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+)
+
+const (
+	OptionStatementQueueSize = "adbc.flight.sql.rpc.queue_size"

Review Comment:
   updated to `adbc.sql.results.rpc.queue_size`



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1179842903


##########
ci/linux-packages/debian/rules:
##########
@@ -40,7 +40,8 @@ override_dh_auto_configure:
           -DADBC_DRIVER_MANAGER=ON                      \
           -DADBC_DRIVER_POSTGRESQL=ON                   \
           -DADBC_DRIVER_SQLITE=ON                       \
-          -DADBC_DRIVER_FLIGHTSQL=ON
+          -DADBC_DRIVER_FLIGHTSQL=ON										\
+					-DADBC_DRIVER_SNOWFLAKE=ON

Review Comment:
   nit: indentation is way off (spaces vs tabs?)



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] github-actions[bot] commented on pull request #586: WIP: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#issuecomment-1507644351

   :warning: Please follow the [Conventional Commits format in CONTRIBUTING.md](https://github.com/apache/arrow-adbc/blob/main/CONTRIBUTING.md) for PR titles.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1173875948


##########
go/adbc/driver/snowflake/statement.go:
##########
@@ -0,0 +1,572 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql/driver"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+)
+
+const (
+	OptionStatementQueueSize = "adbc.flight.sql.rpc.queue_size"
+)
+
+type statement struct {
+	cnxn      *cnxn
+	alloc     memory.Allocator
+	queueSize int
+
+	query       string
+	targetTable string
+	append      bool
+
+	bound      arrow.Record
+	streamBind array.RecordReader
+}
+
+// Close releases any relevant resources associated with this statement
+// and closes it (particularly if it is a prepared statement).
+//
+// A statement instance should not be used after Close is called.
+func (st *statement) Close() error {
+	if st.cnxn == nil {
+		return adbc.Error{
+			Msg:  "statement already closed",
+			Code: adbc.StatusInvalidState}
+	}
+
+	if st.bound != nil {
+		st.bound.Release()
+		st.bound = nil
+	} else if st.streamBind != nil {
+		st.streamBind.Release()
+		st.streamBind = nil
+	}
+	st.cnxn = nil
+	return nil
+}
+
+// SetOption sets a string option on this statement
+func (st *statement) SetOption(key string, val string) error {
+	switch key {
+	case adbc.OptionKeyIngestTargetTable:
+		st.query = ""
+		st.targetTable = val
+	case adbc.OptionKeyIngestMode:
+		switch val {
+		case adbc.OptionValueIngestModeAppend:
+			st.append = true
+		case adbc.OptionValueIngestModeCreate:
+			st.append = false
+		default:
+			return adbc.Error{
+				Msg:  fmt.Sprintf("invalid statement option %s=%s", key, val),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+	default:
+		return adbc.Error{
+			Msg:  fmt.Sprintf("invalid statement option %s=%s", key, val),
+			Code: adbc.StatusInvalidArgument,
+		}
+	}
+	return nil
+}
+
+// SetSqlQuery sets the query string to be executed.
+//
+// The query can then be executed with any of the Execute methods.
+// For queries expected to be executed repeatedly, Prepare should be
+// called before execution.
+func (st *statement) SetSqlQuery(query string) error {
+	st.query = query
+	st.targetTable = ""
+	return nil
+}
+
+func toSnowflakeType(dt arrow.DataType) string {
+	switch dt.ID() {
+	case arrow.EXTENSION:
+		return toSnowflakeType(dt.(arrow.ExtensionType).StorageType())
+	case arrow.DICTIONARY:
+		return toSnowflakeType(dt.(*arrow.DictionaryType).ValueType)
+	case arrow.RUN_END_ENCODED:
+		return toSnowflakeType(dt.(*arrow.RunEndEncodedType).Encoded())
+	case arrow.INT8, arrow.INT16, arrow.INT32, arrow.INT64,
+		arrow.UINT8, arrow.UINT16, arrow.UINT32, arrow.UINT64:
+		return "integer"
+	case arrow.FLOAT32, arrow.FLOAT16, arrow.FLOAT64:
+		return "double"
+	case arrow.DECIMAL, arrow.DECIMAL256:
+		dec := dt.(arrow.DecimalType)
+		return fmt.Sprintf("NUMERIC(%d,%d)", dec.GetPrecision(), dec.GetScale())
+	case arrow.STRING, arrow.LARGE_STRING:
+		return "text"
+	case arrow.BINARY, arrow.LARGE_BINARY:
+		return "binary"
+	case arrow.FIXED_SIZE_BINARY:
+		fsb := dt.(*arrow.FixedSizeBinaryType)
+		return fmt.Sprintf("binary(%d)", fsb.ByteWidth)
+	case arrow.BOOL:
+		return "boolean"
+	case arrow.TIME32, arrow.TIME64:
+		t := dt.(arrow.TemporalWithUnit)
+		prec := int(t.TimeUnit()) * 3
+		return fmt.Sprintf("time(%d)", prec)
+	case arrow.DATE32, arrow.DATE64:
+		return "date"
+	case arrow.TIMESTAMP:
+		ts := dt.(*arrow.TimestampType)
+		prec := int(ts.Unit) * 3
+		if ts.TimeZone == "" {
+			return fmt.Sprintf("timestamp_tz(%d)", prec)
+		}
+		return fmt.Sprintf("timestamp_ltz(%d)", prec)
+	case arrow.DENSE_UNION, arrow.SPARSE_UNION:
+		return "variant"
+	case arrow.LIST, arrow.LARGE_LIST, arrow.FIXED_SIZE_LIST:
+		return "array"
+	case arrow.STRUCT, arrow.MAP:
+		return "object"
+	}
+
+	return ""
+}
+
+func (st *statement) initIngest(ctx context.Context) (string, error) {
+	var (
+		createBldr, insertBldr strings.Builder
+	)
+
+	createBldr.WriteString("CREATE TABLE ")
+	createBldr.WriteString(st.targetTable)

Review Comment:
   The driver does not. And in my attempts to play with the worksheets on snowflake, i couldn't find a good way to actually use the double-quote escape character here, it just failed. So I'm not sure the best route there.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on pull request #586: WIP: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#issuecomment-1514922903

   @lidavidm I've filed https://issues.apache.org/jira/browse/INFRA-24477 to get it added.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175856062


##########
go/adbc/driver/snowflake/statement.go:
##########
@@ -0,0 +1,572 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql/driver"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+)
+
+const (
+	OptionStatementQueueSize = "adbc.flight.sql.rpc.queue_size"

Review Comment:
   Kind of bikeshedding, but maybe drop the 'sql'? 'adbc.rpc.result_queue_size'?



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175869788


##########
go/adbc/driver/snowflake/statement.go:
##########
@@ -0,0 +1,572 @@
+// 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 snowflake
+
+import (
+	"context"
+	"database/sql/driver"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+	"github.com/snowflakedb/gosnowflake"
+)
+
+const (
+	OptionStatementQueueSize = "adbc.flight.sql.rpc.queue_size"

Review Comment:
   sounds good, i'll update 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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1178107124


##########
c/validation/adbc_validation.cc:
##########
@@ -53,6 +54,15 @@ namespace {
   } while (false)
 }  // namespace
 
+/// case insensitive string compare
+bool iequals(std::string s1, std::string s2) {

Review Comment:
   it's not using string_view or a const string because i'm actually modifying the strings here. I'll change this to do the comparison without modifying the strings and have it use string_view



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175886069


##########
.github/workflows/native-unix.yml:
##########
@@ -189,6 +189,165 @@ jobs:
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_build.sh "$(pwd)" "$(pwd)/build"
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_test.sh "$(pwd)" "$(pwd)/build"
 
+  # ------------------------------------------------------------
+  # C/C++ driver service integration tests

Review Comment:
   rebased and pushed!



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1178103782


##########
.github/workflows/integration.yml:
##########
@@ -208,3 +208,161 @@ jobs:
           ADBC_POSTGRESQL_TEST_URI: "postgres://localhost:5432/postgres?user=postgres&password=password"
         run: |
           ./ci/scripts/python_test.sh "$(pwd)" "$(pwd)/build"
+
+  # ------------------------------------------------------------
+  # C/C++ driver service integration tests
+  # ------------------------------------------------------------
+  drivers-integration-conda:
+    name: "C/C++ (Conda) Driver Service Integration Tests"
+    runs-on: ubuntu-latest
+    services:
+      postgres:

Review Comment:
   good point, i'll do so



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1178105359


##########
c/validation/adbc_validation.cc:
##########
@@ -32,6 +32,7 @@
 #include <gtest/gtest-matchers.h>
 #include <gtest/gtest.h>
 #include <nanoarrow/nanoarrow.h>
+#include <algorithm>

Review Comment:
   i was relying on clang-format to sort the headers.... but i guess it didn't do it because I had put it in this block. I'll fix this.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1176691506


##########
.github/workflows/native-unix.yml:
##########
@@ -189,6 +189,165 @@ jobs:
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_build.sh "$(pwd)" "$(pwd)/build"
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_test.sh "$(pwd)" "$(pwd)/build"
 
+  # ------------------------------------------------------------
+  # C/C++ driver service integration tests

Review Comment:
   fair point, i'll shift it there.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1176676103


##########
LICENSE.txt:
##########


Review Comment:
   i mistyped the template file name -_- ugh. Fixed this.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175887660


##########
go/adbc/go.mod:
##########
@@ -22,51 +22,87 @@ go 1.18
 require (
 	github.com/apache/arrow/go/v12 v12.0.0-20230421000340-388f3a88c647
 	github.com/bluele/gcache v0.0.2
-	github.com/stretchr/testify v1.8.1
-	golang.org/x/exp v0.0.0-20230206171751-46f607a40771
+	github.com/snowflakedb/gosnowflake v1.6.19-0.20230228202026-7c7d86c6b3d0
+	github.com/stretchr/testify v1.8.2
+	golang.org/x/exp v0.0.0-20230420155640-133eef4313cb
 	golang.org/x/sync v0.1.0
-	golang.org/x/tools v0.6.0
-	google.golang.org/grpc v1.53.0
-	google.golang.org/protobuf v1.28.1
+	golang.org/x/tools v0.8.0
+	google.golang.org/grpc v1.54.0
+	google.golang.org/protobuf v1.30.0
 )
 
 require (
-	github.com/andybalholm/brotli v1.0.4 // indirect
+	github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect
+	github.com/99designs/keyring v1.2.2 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/azcore v1.5.0 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 // indirect
+	github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 // indirect
+	github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect
+	github.com/andybalholm/brotli v1.0.5 // indirect
+	github.com/apache/arrow/go/v11 v11.0.0 // indirect
 	github.com/apache/thrift v0.17.0 // indirect
+	github.com/aws/aws-sdk-go-v2 v1.17.8 // indirect
+	github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.10 // indirect
+	github.com/aws/aws-sdk-go-v2/credentials v1.13.20 // indirect
+	github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.63 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.32 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.26 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.24 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.11 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.27 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.26 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.1 // indirect
+	github.com/aws/aws-sdk-go-v2/service/s3 v1.32.0 // indirect
+	github.com/aws/smithy-go v1.13.5 // indirect
+	github.com/danieljoos/wincred v1.1.2 // indirect
 	github.com/davecgh/go-spew v1.1.1 // indirect
 	github.com/dustin/go-humanize v1.0.1 // indirect
-	github.com/goccy/go-json v0.10.0 // indirect
-	github.com/golang/protobuf v1.5.2 // indirect
+	github.com/dvsekhvalnov/jose2go v1.5.0 // indirect
+	github.com/form3tech-oss/jwt-go v3.2.5+incompatible // indirect
+	github.com/gabriel-vasile/mimetype v1.4.2 // indirect
+	github.com/goccy/go-json v0.10.2 // indirect
+	github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect
+	github.com/golang/protobuf v1.5.3 // indirect
 	github.com/golang/snappy v0.0.4 // indirect
-	github.com/google/flatbuffers v23.1.21+incompatible // indirect
+	github.com/google/flatbuffers v23.3.3+incompatible // indirect
 	github.com/google/uuid v1.3.0 // indirect
+	github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect
+	github.com/jmespath/go-jmespath v0.4.0 // indirect
 	github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 // indirect
 	github.com/klauspost/asmfmt v1.3.2 // indirect
-	github.com/klauspost/compress v1.15.15 // indirect
-	github.com/klauspost/cpuid/v2 v2.2.3 // indirect
+	github.com/klauspost/compress v1.16.5 // indirect
+	github.com/klauspost/cpuid/v2 v2.2.4 // indirect
 	github.com/kr/text v0.2.0 // indirect
-	github.com/mattn/go-isatty v0.0.17 // indirect
+	github.com/mattn/go-isatty v0.0.18 // indirect
 	github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect
 	github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect
+	github.com/mtibben/percent v0.2.1 // indirect
 	github.com/pierrec/lz4/v4 v4.1.17 // indirect
+	github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 // indirect
 	github.com/pmezard/go-difflib v1.0.0 // indirect
 	github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect
+	github.com/sirupsen/logrus v1.9.0 // indirect
 	github.com/zeebo/xxh3 v1.0.2 // indirect
-	golang.org/x/mod v0.8.0 // indirect
-	golang.org/x/net v0.7.0 // indirect
-	golang.org/x/sys v0.5.0 // indirect
-	golang.org/x/text v0.7.0 // indirect
+	golang.org/x/crypto v0.8.0 // indirect
+	golang.org/x/mod v0.10.0 // indirect
+	golang.org/x/net v0.9.0 // indirect
+	golang.org/x/sys v0.7.0 // indirect
+	golang.org/x/term v0.7.0 // indirect
+	golang.org/x/text v0.9.0 // indirect
 	golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
-	google.golang.org/genproto v0.0.0-20230209215440-0dfe4f8abfcc // indirect
+	gonum.org/v1/gonum v0.12.0 // indirect
+	google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect
 	gopkg.in/yaml.v3 v3.0.1 // indirect
-	lukechampine.com/uint128 v1.2.0 // indirect
+	lukechampine.com/uint128 v1.3.0 // indirect
 	modernc.org/cc/v3 v3.40.0 // indirect
 	modernc.org/ccgo/v3 v3.16.13 // indirect
-	modernc.org/libc v1.22.2 // indirect
+	modernc.org/libc v1.22.4 // indirect
 	modernc.org/mathutil v1.5.0 // indirect
 	modernc.org/memory v1.5.0 // indirect
 	modernc.org/opt v0.1.3 // indirect
-	modernc.org/sqlite v1.20.4 // indirect
+	modernc.org/sqlite v1.21.2 // indirect
 	modernc.org/strutil v1.1.3 // indirect
 	modernc.org/token v1.1.0 // indirect
 )
+
+replace github.com/snowflakedb/gosnowflake => github.com/zeroshade/gosnowflake v0.0.0-20230419185854-4870fd63961f

Review Comment:
   Where's the script for the dependency analyzer?



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] lidavidm commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1175981769


##########
LICENSE.txt:
##########


Review Comment:
   Did you mean to delete the entire file?



##########
.github/workflows/native-unix.yml:
##########
@@ -189,6 +189,165 @@ jobs:
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_build.sh "$(pwd)" "$(pwd)/build"
           env BUILD_ALL=0 BUILD_DRIVER_MANAGER=1 ./ci/scripts/cpp_test.sh "$(pwd)" "$(pwd)/build"
 
+  # ------------------------------------------------------------
+  # C/C++ driver service integration tests

Review Comment:
   Shouldn't this all go in the integration.yml pipeline now? (I split it out a while back to prevent this pipeline from getting unwieldy.)



##########
license.tpl:
##########
@@ -0,0 +1,321 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+--------------------------------------------------------------------------------
+
+This project includes code from Apache Arrow Nanoarrow.
+
+* c/vendor/nanoarrow is the source of nanoarrow
+
+Copyright: 2022 The Apache Software Foundation.
+Home page: https://arrow.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+The files python/*/*/_version.py and python/*/*/_static_version.py
+contain code from
+
+https://github.com/jbweston/miniver
+
+which is made available under the Creative Commons CC0 license.
+
+--------------------------------------------------------------------------------
+
+The files under ci/conda/.ci-support have the following license
+
+BSD 3-clause license
+Copyright (c) 2015-2022, conda-forge
+All rights reserved.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency Go is statically linked in certain binary distributions,
+like the Python wheels. The Go project is under the BSD 3-clause license +
+PATENTS weak patent termination clause
+(https://github.com/golang/go/blob/master/PATENTS).
+
+Copyright (c) 2009 The Go Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency libpq is statically linked in certain binary
+distributions, like the Python wheels. libpq has the following license:
+
+Portions Copyright © 1996-2022, The PostgreSQL Global Development Group
+
+Portions Copyright © 1994, The Regents of the University of California
+
+Permission to use, copy, modify, and distribute this software and its
+documentation for any purpose, without fee, and without a written
+agreement is hereby granted, provided that the above copyright notice
+and this paragraph and the following two paragraphs appear in all
+copies.
+
+IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY
+FOR DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES,
+INCLUDING LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND
+ITS DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN
+ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
+INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE
+PROVIDED HEREUNDER IS ON AN "AS IS" BASIS, AND THE UNIVERSITY OF
+CALIFORNIA HAS NO OBLIGATIONS TO PROVIDE MAINTENANCE, SUPPORT,
+UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency OpenSSL is statically linked in certain binary
+distributions, like the Python wheels. OpenSSL version 3 has the
+following license:
+
+Copyright 1995-2021 The OpenSSL Project Authors. All Rights Reserved.
+
+Licensed under the Apache License 2.0 (the "License").  You may not use
+this file except in compliance with the License.  You can obtain a copy
+in the file LICENSE in the source distribution or at
+https://www.openssl.org/source/license.html
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency SQLite is statically linked in certain binary
+distributions, like the Python wheels. SQLite is public domain.
+
+{{ range .}}
+--------------------------------------------------------------------------------
+
+3rdparty dependency {{ .Name }}
+is statically linked in certain binary distributions, like the Python wheels.

Review Comment:
   I suppose until we actually create a wheel or Conda package for the Snowflake driver, we don't need to do this...but maybe let's just take care of it now



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#issuecomment-1526408388

   @lidavidm My change on the snowflake repo has been merged and I've updated the go.mod here to point to that commit hash. They release on a monthly schedule so the next actual release which will include this change won't be until the end of May, at which point we can update the go.mod to point to that specific version tag.
   
   So this is good to go!


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-adbc] zeroshade commented on a diff in pull request #586: feat(go/adbc/driver): Adbc Driver for Snowflake

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #586:
URL: https://github.com/apache/arrow-adbc/pull/586#discussion_r1180552762


##########
ci/linux-packages/debian/rules:
##########
@@ -40,7 +40,8 @@ override_dh_auto_configure:
           -DADBC_DRIVER_MANAGER=ON                      \
           -DADBC_DRIVER_POSTGRESQL=ON                   \
           -DADBC_DRIVER_SQLITE=ON                       \
-          -DADBC_DRIVER_FLIGHTSQL=ON
+          -DADBC_DRIVER_FLIGHTSQL=ON										\
+					-DADBC_DRIVER_SNOWFLAKE=ON

Review Comment:
   fixed



-- 
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: github-unsubscribe@arrow.apache.org

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