You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by li...@apache.org on 2023/01/30 22:13:30 UTC

[arrow-adbc] branch main updated: test(go/adbc/driver/flightsql): add basic tests for TLS, Substrait (#403)

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

lidavidm pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-adbc.git


The following commit(s) were added to refs/heads/main by this push:
     new 83eb57e  test(go/adbc/driver/flightsql): add basic tests for TLS, Substrait (#403)
83eb57e is described below

commit 83eb57e9a6a938db1af0a68aad487020905ec040
Author: David Li <li...@gmail.com>
AuthorDate: Mon Jan 30 17:13:24 2023 -0500

    test(go/adbc/driver/flightsql): add basic tests for TLS, Substrait (#403)
    
    Fixes #380.
---
 .env                                            |   2 +-
 go/adbc/adbc.go                                 |   4 +-
 go/adbc/driver/flightsql/flightsql_adbc.go      |  19 +--
 go/adbc/driver/flightsql/flightsql_adbc_test.go | 163 +++++++++++++++++++++---
 go/adbc/driver/flightsql/flightsql_statement.go | 121 ++++++++++++------
 go/adbc/driver/flightsql/record_reader.go       |  10 +-
 go/adbc/driver/flightsql/record_reader_test.go  |  12 +-
 go/adbc/driver/flightsql/utils.go               |   4 +
 go/adbc/drivermgr/wrapper.go                    |   6 +-
 go/adbc/drivermgr/wrapper_sqlite_test.go        |   6 +-
 go/adbc/go.mod                                  |   2 +-
 go/adbc/go.sum                                  |   8 +-
 go/adbc/pkg/_tmpl/driver.go.tmpl                |   4 +-
 go/adbc/pkg/flightsql/driver.go                 |   4 +-
 go/adbc/sqldriver/driver.go                     |  10 +-
 go/adbc/sqldriver/driver_internals_test.go      |   6 +-
 go/adbc/standard_schemas.go                     |   2 +-
 go/adbc/validation/validation.go                |   8 +-
 18 files changed, 281 insertions(+), 110 deletions(-)

diff --git a/.env b/.env
index 44ec3d8..2b73edb 100644
--- a/.env
+++ b/.env
@@ -33,7 +33,7 @@ MANYLINUX=2014
 MAVEN=3.5.4
 PYTHON=3.10
 GO=1.19.5
-ARROW_MAJOR_VERSION=11
+ARROW_MAJOR_VERSION=12
 
 # Used through docker-compose.yml and serves as the default version for the
 # ci/scripts/install_vcpkg.sh script.
diff --git a/go/adbc/adbc.go b/go/adbc/adbc.go
index 2f5d7bd..2ecd415 100644
--- a/go/adbc/adbc.go
+++ b/go/adbc/adbc.go
@@ -40,8 +40,8 @@ import (
 	"context"
 	"fmt"
 
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
 )
 
 //go:generate go run golang.org/x/tools/cmd/stringer -type Status -linecomment
diff --git a/go/adbc/driver/flightsql/flightsql_adbc.go b/go/adbc/driver/flightsql/flightsql_adbc.go
index 5ffa88b..c0b70fa 100644
--- a/go/adbc/driver/flightsql/flightsql_adbc.go
+++ b/go/adbc/driver/flightsql/flightsql_adbc.go
@@ -48,13 +48,13 @@ import (
 	"time"
 
 	"github.com/apache/arrow-adbc/go/adbc"
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/flight"
-	"github.com/apache/arrow/go/v11/arrow/flight/flightsql"
-	"github.com/apache/arrow/go/v11/arrow/flight/flightsql/schema_ref"
-	"github.com/apache/arrow/go/v11/arrow/ipc"
-	"github.com/apache/arrow/go/v11/arrow/memory"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/flight"
+	"github.com/apache/arrow/go/v12/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v12/arrow/flight/flightsql/schema_ref"
+	"github.com/apache/arrow/go/v12/arrow/ipc"
+	"github.com/apache/arrow/go/v12/arrow/memory"
 	"github.com/bluele/gcache"
 	"golang.org/x/exp/maps"
 	"google.golang.org/grpc"
@@ -131,9 +131,10 @@ func (d Driver) NewDatabase(opts map[string]string) (adbc.Database, error) {
 		return nil, adbc.Error{Msg: err.Error(), Code: adbc.StatusInvalidArgument}
 	}
 
-	// Use WithBlock to surface connection errors eagerly
+	// Do not set WithBlock since it converts some types of connection
+	// errors to infinite hangs
 	// Use WithMaxMsgSize(16 MiB) since Flight services tend to send large messages
-	db.dialOpts.block = true
+	db.dialOpts.block = false
 	db.dialOpts.maxMsgSize = 16 * 1024 * 1024
 
 	return db, db.SetOptions(opts)
diff --git a/go/adbc/driver/flightsql/flightsql_adbc_test.go b/go/adbc/driver/flightsql/flightsql_adbc_test.go
index 38a2f00..23fc933 100644
--- a/go/adbc/driver/flightsql/flightsql_adbc_test.go
+++ b/go/adbc/driver/flightsql/flightsql_adbc_test.go
@@ -18,24 +18,36 @@
 package flightsql_test
 
 import (
+	"bytes"
 	"context"
+	"crypto/rand"
+	"crypto/rsa"
+	"crypto/tls"
+	"crypto/x509"
+	"crypto/x509/pkix"
+	"encoding/pem"
 	"fmt"
 	"io"
+	"math/big"
+	"net"
 	"os"
 	"strings"
 	"testing"
+	"time"
 
 	"github.com/apache/arrow-adbc/go/adbc"
 	driver "github.com/apache/arrow-adbc/go/adbc/driver/flightsql"
 	"github.com/apache/arrow-adbc/go/adbc/validation"
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/flight"
-	"github.com/apache/arrow/go/v11/arrow/flight/flightsql"
-	"github.com/apache/arrow/go/v11/arrow/flight/flightsql/example"
-	"github.com/apache/arrow/go/v11/arrow/memory"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/flight"
+	"github.com/apache/arrow/go/v12/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v12/arrow/flight/flightsql/example"
+	"github.com/apache/arrow/go/v12/arrow/memory"
 	"github.com/stretchr/testify/require"
 	"github.com/stretchr/testify/suite"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/credentials"
 	"google.golang.org/grpc/metadata"
 	"google.golang.org/protobuf/proto"
 )
@@ -57,6 +69,7 @@ type FlightSQLQuirks struct {
 	srv    *example.SQLiteFlightSQLServer
 	s      flight.Server
 	middle HeaderServerMiddleware
+	opts   []grpc.ServerOption
 
 	done chan bool
 	mem  *memory.CheckedAllocator
@@ -68,7 +81,7 @@ func (s *FlightSQLQuirks) SetupDriver(t *testing.T) adbc.Driver {
 	s.done = make(chan bool)
 	var err error
 	s.mem = memory.NewCheckedAllocator(memory.DefaultAllocator)
-	s.s = flight.NewServerWithMiddleware([]flight.ServerMiddleware{flight.CreateServerMiddleware(&s.middle)})
+	s.s = flight.NewServerWithMiddleware([]flight.ServerMiddleware{flight.CreateServerMiddleware(&s.middle)}, s.opts...)
 	s.srv, err = example.NewSQLiteFlightSQLServer()
 	require.NoError(t, err)
 	s.srv.Alloc = s.mem
@@ -210,7 +223,7 @@ func (s *FlightSQLQuirks) GetMetadata(code adbc.InfoCode) interface{} {
 	case adbc.InfoVendorVersion:
 		return "sqlite 3"
 	case adbc.InfoVendorArrowVersion:
-		return "11.0.0-SNAPSHOT"
+		return "12.0.0-SNAPSHOT"
 	}
 
 	return nil
@@ -223,11 +236,12 @@ func TestADBCFlightSQL(t *testing.T) {
 	suite.Run(t, &validation.StatementTests{Quirks: q})
 
 	suite.Run(t, &DefaultDialOptionsTests{Quirks: q})
+	suite.Run(t, &HeaderTests{Quirks: q})
+	suite.Run(t, &OptionTests{Quirks: q})
 	suite.Run(t, &PartitionTests{Quirks: q})
-	suite.Run(t, &SSLTests{Quirks: q})
 	suite.Run(t, &StatementTests{Quirks: q})
-	suite.Run(t, &HeaderTests{Quirks: q})
-	suite.Run(t, new(TimeoutTestSuite))
+	suite.Run(t, &TimeoutTestSuite{})
+	suite.Run(t, &TLSTests{Quirks: &FlightSQLQuirks{}})
 }
 
 // Driver-specific tests
@@ -325,23 +339,23 @@ func (suite *DefaultDialOptionsTests) TestMaxIncomingMessageSizeLow() {
 	suite.NoError(reader.Err())
 }
 
-type SSLTests struct {
+type OptionTests struct {
 	suite.Suite
 
 	Driver adbc.Driver
 	Quirks validation.DriverQuirks
 }
 
-func (suite *SSLTests) SetupTest() {
+func (suite *OptionTests) SetupTest() {
 	suite.Driver = suite.Quirks.SetupDriver(suite.T())
 }
 
-func (suite *SSLTests) TearDownTest() {
+func (suite *OptionTests) TearDownTest() {
 	suite.Quirks.TearDownDriver(suite.T(), suite.Driver)
 	suite.Driver = nil
 }
 
-func (suite *SSLTests) TestMutualTLS() {
+func (suite *OptionTests) TestMutualTLS() {
 	// Just checks that the option is accepted - doesn't actually configure TLS
 	options := suite.Quirks.DatabaseOptions()
 
@@ -358,7 +372,7 @@ func (suite *SSLTests) TestMutualTLS() {
 	suite.Require().ErrorContains(err, "Must provide both")
 }
 
-func (suite *SSLTests) TestOverrideHostname() {
+func (suite *OptionTests) TestOverrideHostname() {
 	// Just checks that the option is accepted - doesn't actually configure TLS
 	options := suite.Quirks.DatabaseOptions()
 	options["adbc.flight.sql.client_option.tls_override_hostname"] = "hostname"
@@ -366,7 +380,7 @@ func (suite *SSLTests) TestOverrideHostname() {
 	suite.Require().NoError(err)
 }
 
-func (suite *SSLTests) TestRootCerts() {
+func (suite *OptionTests) TestRootCerts() {
 	// Just checks that the option is accepted - doesn't actually configure TLS
 	options := suite.Quirks.DatabaseOptions()
 	options["adbc.flight.sql.client_option.tls_root_certs"] = "these are not valid certs"
@@ -374,7 +388,7 @@ func (suite *SSLTests) TestRootCerts() {
 	suite.Require().ErrorContains(err, "Invalid value for database option 'adbc.flight.sql.client_option.tls_root_certs': failed to append certificates")
 }
 
-func (suite *SSLTests) TestSkipVerify() {
+func (suite *OptionTests) TestSkipVerify() {
 	options := suite.Quirks.DatabaseOptions()
 	options["adbc.flight.sql.client_option.tls_skip_verify"] = "true"
 	_, err := suite.Driver.NewDatabase(options)
@@ -391,7 +405,7 @@ func (suite *SSLTests) TestSkipVerify() {
 	suite.Require().ErrorContains(err, "Invalid value for database option 'adbc.flight.sql.client_option.tls_skip_verify': 'invalid'")
 }
 
-func (suite *SSLTests) TestUnknownOption() {
+func (suite *OptionTests) TestUnknownOption() {
 	options := suite.Quirks.DatabaseOptions()
 	options["unknown option"] = "unknown value"
 	_, err := suite.Driver.NewDatabase(options)
@@ -497,6 +511,17 @@ func (suite *StatementTests) TestQueueSizeOption() {
 	suite.Require().NoError(err)
 }
 
+func (suite *StatementTests) TestSubstrait() {
+	err := suite.Stmt.SetSubstraitPlan([]byte("foo"))
+	suite.Require().NoError(err)
+
+	_, _, err = suite.Stmt.ExecuteQuery(context.Background())
+	suite.Require().ErrorContains(err, "Substrait is not yet implemented")
+	var adbcError adbc.Error
+	suite.ErrorAs(err, &adbcError)
+	suite.Equal(adbc.StatusNotImplemented, adbcError.Code)
+}
+
 func (suite *StatementTests) TestUnknownOption() {
 	err := suite.Stmt.SetOption("unknown option", "")
 	suite.Require().ErrorContains(err, "Unknown statement option 'unknown option'")
@@ -797,3 +822,103 @@ func (ts *TimeoutTestSuite) TestGetFlightInfoTimeout() {
 	ts.ErrorAs(err, &adbcErr)
 	ts.NotEqual(adbc.StatusNotImplemented, adbcErr.Code)
 }
+
+type TLSTests struct {
+	suite.Suite
+
+	Driver adbc.Driver
+	Quirks *FlightSQLQuirks
+
+	DB   adbc.Database
+	Cnxn adbc.Connection
+	Stmt adbc.Statement
+	ctx  context.Context
+}
+
+func (suite *TLSTests) SetupTest() {
+	var err error
+
+	// Generate a self-signed certificate in-process for testing
+	privKey, err := rsa.GenerateKey(rand.Reader, 2048)
+	suite.Require().NoError(err)
+	certTemplate := x509.Certificate{
+		SerialNumber: big.NewInt(1),
+		Subject: pkix.Name{
+			Organization: []string{"Unit Tests Incorporated"},
+		},
+		IPAddresses:           []net.IP{net.IPv4(127, 0, 0, 1), net.IPv6loopback},
+		NotBefore:             time.Now(),
+		NotAfter:              time.Now().Add(time.Hour),
+		KeyUsage:              x509.KeyUsageKeyEncipherment,
+		ExtKeyUsage:           []x509.ExtKeyUsage{x509.ExtKeyUsageServerAuth},
+		BasicConstraintsValid: true,
+	}
+	certDer, err := x509.CreateCertificate(rand.Reader, &certTemplate, &certTemplate, &privKey.PublicKey, privKey)
+	suite.Require().NoError(err)
+	buffer := &bytes.Buffer{}
+	suite.Require().NoError(pem.Encode(buffer, &pem.Block{Type: "CERTIFICATE", Bytes: certDer}))
+	certBytes := make([]byte, buffer.Len())
+	copy(certBytes, buffer.Bytes())
+	buffer.Reset()
+	suite.Require().NoError(pem.Encode(buffer, &pem.Block{Type: "RSA PRIVATE KEY", Bytes: x509.MarshalPKCS1PrivateKey(privKey)}))
+	keyBytes := make([]byte, buffer.Len())
+	copy(keyBytes, buffer.Bytes())
+
+	cert, err := tls.X509KeyPair(certBytes, keyBytes)
+	suite.Require().NoError(err)
+
+	suite.Require().NoError(err)
+	tlsConfig := &tls.Config{Certificates: []tls.Certificate{cert}}
+	tlsCreds := credentials.NewTLS(tlsConfig)
+	suite.Quirks.opts = []grpc.ServerOption{grpc.Creds(tlsCreds)}
+
+	suite.Driver = suite.Quirks.SetupDriver(suite.T())
+	suite.DB, err = suite.Driver.NewDatabase(map[string]string{
+		adbc.OptionKeyURI: "grpc+tls://" + suite.Quirks.s.Addr().String(),
+		"adbc.flight.sql.client_option.tls_skip_verify": "true",
+	})
+
+	suite.Require().NoError(err)
+	suite.ctx = context.Background()
+	suite.Cnxn, err = suite.DB.Open(suite.ctx)
+	suite.Require().NoError(err)
+	suite.Stmt, err = suite.Cnxn.NewStatement()
+	suite.Require().NoError(err)
+}
+
+func (suite *TLSTests) TearDownTest() {
+	suite.Require().NoError(suite.Stmt.Close())
+	suite.Require().NoError(suite.Cnxn.Close())
+	suite.Quirks.TearDownDriver(suite.T(), suite.Driver)
+	suite.Cnxn = nil
+	suite.DB = nil
+	suite.Driver = nil
+}
+
+func (suite *TLSTests) TestSimpleQuery() {
+	suite.NoError(suite.Stmt.SetSqlQuery("SELECT 1"))
+	reader, _, err := suite.Stmt.ExecuteQuery(suite.ctx)
+	suite.NoError(err)
+	defer reader.Release()
+
+	for reader.Next() {
+	}
+	suite.NoError(reader.Err())
+}
+
+func (suite *TLSTests) TestInvalidOptions() {
+	db, err := suite.Driver.NewDatabase(map[string]string{
+		adbc.OptionKeyURI: "grpc+tls://" + suite.Quirks.s.Addr().String(),
+		"adbc.flight.sql.client_option.tls_skip_verify": "false",
+	})
+	suite.Require().NoError(err)
+
+	cnxn, err := db.Open(suite.ctx)
+	suite.Require().NoError(err)
+	stmt, err := cnxn.NewStatement()
+	suite.Require().NoError(err)
+
+	suite.NoError(stmt.SetSqlQuery("SELECT 1"))
+	_, _, err = stmt.ExecuteQuery(suite.ctx)
+	suite.Contains(err.Error(), "Unavailable")
+}
diff --git a/go/adbc/driver/flightsql/flightsql_statement.go b/go/adbc/driver/flightsql/flightsql_statement.go
index 45010d3..44ac9d8 100644
--- a/go/adbc/driver/flightsql/flightsql_statement.go
+++ b/go/adbc/driver/flightsql/flightsql_statement.go
@@ -24,12 +24,13 @@ import (
 	"strings"
 
 	"github.com/apache/arrow-adbc/go/adbc"
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/flight"
-	"github.com/apache/arrow/go/v11/arrow/flight/flightsql"
-	"github.com/apache/arrow/go/v11/arrow/memory"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/flight"
+	"github.com/apache/arrow/go/v12/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v12/arrow/memory"
 	"github.com/bluele/gcache"
+	"google.golang.org/grpc"
 	"google.golang.org/grpc/metadata"
 	"google.golang.org/protobuf/proto"
 )
@@ -38,13 +39,76 @@ const (
 	OptionStatementQueueSize = "adbc.flight.sql.rpc.queue_size"
 )
 
+type sqlOrSubstrait struct {
+	sqlQuery      string
+	substraitPlan []byte
+}
+
+func (s *sqlOrSubstrait) setSqlQuery(query string) {
+	s.sqlQuery = query
+	s.substraitPlan = nil
+}
+
+func (s *sqlOrSubstrait) setSubstraitPlan(plan []byte) {
+	s.sqlQuery = ""
+	s.substraitPlan = plan
+}
+
+func (s *sqlOrSubstrait) execute(ctx context.Context, cl *flightsql.Client, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
+	if s.sqlQuery != "" {
+		return cl.Execute(ctx, s.sqlQuery, opts...)
+	} else if s.substraitPlan != nil {
+		// TODO(apache/arrow#33935): Substrait not supported upstream
+		return nil, adbc.Error{
+			Code: adbc.StatusNotImplemented,
+			Msg:  "Substrait is not yet implemented",
+		}
+	}
+	return nil, adbc.Error{
+		Code: adbc.StatusInvalidState,
+		Msg:  "[Flight SQL Statement] cannot call ExecuteQuery without a query or prepared statement",
+	}
+}
+
+func (s *sqlOrSubstrait) executeUpdate(ctx context.Context, cl *flightsql.Client, opts ...grpc.CallOption) (int64, error) {
+	if s.sqlQuery != "" {
+		return cl.ExecuteUpdate(ctx, s.sqlQuery, opts...)
+	} else if s.substraitPlan != nil {
+		// TODO: Substrait not supported upstream
+		return -1, adbc.Error{
+			Code: adbc.StatusNotImplemented,
+			Msg:  "Substrait is not yet implemented",
+		}
+	}
+	return -1, adbc.Error{
+		Code: adbc.StatusInvalidState,
+		Msg:  "[Flight SQL Statement] cannot call ExecuteUpdate without a query or prepared statement",
+	}
+}
+
+func (s *sqlOrSubstrait) prepare(ctx context.Context, cl *flightsql.Client, alloc memory.Allocator, opts ...grpc.CallOption) (*flightsql.PreparedStatement, error) {
+	if s.sqlQuery != "" {
+		return cl.Prepare(ctx, alloc, s.sqlQuery, opts...)
+	} else if s.substraitPlan != nil {
+		// TODO: Substrait not supported upstream
+		return nil, adbc.Error{
+			Code: adbc.StatusNotImplemented,
+			Msg:  "Substrait is not yet implemented",
+		}
+	}
+	return nil, adbc.Error{
+		Code: adbc.StatusInvalidState,
+		Msg:  "[FlightSQL Statement] must call SetSqlQuery before Prepare",
+	}
+}
+
 type statement struct {
 	alloc       memory.Allocator
 	cl          *flightsql.Client
 	clientCache gcache.Cache
 
 	hdrs      metadata.MD
-	query     string
+	query     sqlOrSubstrait
 	prepared  *flightsql.PreparedStatement
 	queueSize int
 	timeouts  timeoutOption
@@ -158,7 +222,7 @@ func (s *statement) SetSqlQuery(query string) error {
 		s.prepared = nil
 	}
 
-	s.query = query
+	s.query.setSqlQuery(query)
 	return nil
 }
 
@@ -172,13 +236,8 @@ func (s *statement) ExecuteQuery(ctx context.Context) (rdr array.RecordReader, n
 	var info *flight.FlightInfo
 	if s.prepared != nil {
 		info, err = s.prepared.Execute(ctx, s.timeouts)
-	} else if s.query != "" {
-		info, err = s.cl.Execute(ctx, s.query, s.timeouts)
 	} else {
-		return nil, -1, adbc.Error{
-			Msg:  "[Flight SQL Statement] cannot call ExecuteQuery without a query or prepared statement",
-			Code: adbc.StatusInvalidState,
-		}
+		info, err = s.query.execute(ctx, s.cl, s.timeouts)
 	}
 
 	if err != nil {
@@ -198,28 +257,14 @@ func (s *statement) ExecuteUpdate(ctx context.Context) (int64, error) {
 		return s.prepared.ExecuteUpdate(ctx, s.timeouts)
 	}
 
-	if s.query != "" {
-		return s.cl.ExecuteUpdate(ctx, s.query, s.timeouts)
-	}
-
-	return -1, adbc.Error{
-		Msg:  "[Flight SQL Statement] cannot call ExecuteUpdate without a query or prepared statement",
-		Code: adbc.StatusInvalidState,
-	}
+	return s.query.executeUpdate(ctx, s.cl, s.timeouts)
 }
 
 // Prepare turns this statement into a prepared statement to be executed
 // multiple times. This invalidates any prior result sets.
 func (s *statement) Prepare(ctx context.Context) error {
 	ctx = metadata.NewOutgoingContext(ctx, s.hdrs)
-	if s.query == "" {
-		return adbc.Error{
-			Msg:  "[FlightSQL Statement] must call SetSqlQuery before Prepare",
-			Code: adbc.StatusInvalidState,
-		}
-	}
-
-	prep, err := s.cl.Prepare(ctx, s.alloc, s.query, s.timeouts)
+	prep, err := s.query.prepare(ctx, s.cl, s.alloc, s.timeouts)
 	if err != nil {
 		return adbcFromFlightStatus(err)
 	}
@@ -237,10 +282,15 @@ func (s *statement) Prepare(ctx context.Context) error {
 // using any of the Execute methods. If the query is expected to be
 // executed repeatedly, Prepare should be called first on the statement.
 func (s *statement) SetSubstraitPlan(plan []byte) error {
-	return adbc.Error{
-		Msg:  "[FlightSQL Statement] SetSubstraitPlan not implemented",
-		Code: adbc.StatusNotImplemented,
+	if s.prepared != nil {
+		if err := s.closePreparedStatement(); err != nil {
+			return err
+		}
+		s.prepared = nil
 	}
+
+	s.query.setSubstraitPlan(plan)
+	return nil
 }
 
 // Bind uses an arrow record batch to bind parameters to the query.
@@ -332,13 +382,8 @@ func (s *statement) ExecutePartitions(ctx context.Context) (*arrow.Schema, adbc.
 
 	if s.prepared != nil {
 		info, err = s.prepared.Execute(ctx, s.timeouts)
-	} else if s.query != "" {
-		info, err = s.cl.Execute(ctx, s.query, s.timeouts)
 	} else {
-		return nil, out, -1, adbc.Error{
-			Msg:  "[Flight SQL Statement] cannot call ExecuteQuery without a query or prepared statement",
-			Code: adbc.StatusInvalidState,
-		}
+		info, err = s.query.execute(ctx, s.cl, s.timeouts)
 	}
 
 	if err != nil {
diff --git a/go/adbc/driver/flightsql/record_reader.go b/go/adbc/driver/flightsql/record_reader.go
index c90e330..042c661 100644
--- a/go/adbc/driver/flightsql/record_reader.go
+++ b/go/adbc/driver/flightsql/record_reader.go
@@ -22,11 +22,11 @@ import (
 	"sync/atomic"
 
 	"github.com/apache/arrow-adbc/go/adbc"
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/flight"
-	"github.com/apache/arrow/go/v11/arrow/flight/flightsql"
-	"github.com/apache/arrow/go/v11/arrow/memory"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/flight"
+	"github.com/apache/arrow/go/v12/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v12/arrow/memory"
 	"github.com/bluele/gcache"
 	"golang.org/x/sync/errgroup"
 	"google.golang.org/grpc"
diff --git a/go/adbc/driver/flightsql/record_reader_test.go b/go/adbc/driver/flightsql/record_reader_test.go
index 670d943..fd4d31a 100644
--- a/go/adbc/driver/flightsql/record_reader_test.go
+++ b/go/adbc/driver/flightsql/record_reader_test.go
@@ -24,12 +24,12 @@ import (
 	"testing"
 
 	"github.com/apache/arrow-adbc/go/adbc"
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/flight"
-	"github.com/apache/arrow/go/v11/arrow/flight/flightsql"
-	"github.com/apache/arrow/go/v11/arrow/ipc"
-	"github.com/apache/arrow/go/v11/arrow/memory"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/flight"
+	"github.com/apache/arrow/go/v12/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v12/arrow/ipc"
+	"github.com/apache/arrow/go/v12/arrow/memory"
 	"github.com/bluele/gcache"
 	"github.com/stretchr/testify/suite"
 	"google.golang.org/grpc"
diff --git a/go/adbc/driver/flightsql/utils.go b/go/adbc/driver/flightsql/utils.go
index a8b5eeb..e3dd654 100644
--- a/go/adbc/driver/flightsql/utils.go
+++ b/go/adbc/driver/flightsql/utils.go
@@ -24,6 +24,10 @@ import (
 )
 
 func adbcFromFlightStatus(err error) error {
+	if _, ok := err.(adbc.Error); ok {
+		return err
+	}
+
 	var adbcCode adbc.Status
 	switch status.Code(err) {
 	case codes.OK:
diff --git a/go/adbc/drivermgr/wrapper.go b/go/adbc/drivermgr/wrapper.go
index 22211fc..5f910b1 100644
--- a/go/adbc/drivermgr/wrapper.go
+++ b/go/adbc/drivermgr/wrapper.go
@@ -36,9 +36,9 @@ import (
 	"unsafe"
 
 	"github.com/apache/arrow-adbc/go/adbc"
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/cdata"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/cdata"
 )
 
 type option struct {
diff --git a/go/adbc/drivermgr/wrapper_sqlite_test.go b/go/adbc/drivermgr/wrapper_sqlite_test.go
index d5c0496..1b4795b 100644
--- a/go/adbc/drivermgr/wrapper_sqlite_test.go
+++ b/go/adbc/drivermgr/wrapper_sqlite_test.go
@@ -27,9 +27,9 @@ import (
 
 	"github.com/apache/arrow-adbc/go/adbc"
 	"github.com/apache/arrow-adbc/go/adbc/drivermgr"
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/memory"
+	"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/stretchr/testify/assert"
 	"github.com/stretchr/testify/require"
 	"github.com/stretchr/testify/suite"
diff --git a/go/adbc/go.mod b/go/adbc/go.mod
index fbe6c08..9e9ea91 100644
--- a/go/adbc/go.mod
+++ b/go/adbc/go.mod
@@ -20,7 +20,7 @@ module github.com/apache/arrow-adbc/go/adbc
 go 1.18
 
 require (
-	github.com/apache/arrow/go/v11 v11.0.0-20230126180639-8f537ca93922
+	github.com/apache/arrow/go/v12 v12.0.0-20230130180348-64a48c999345
 	github.com/bluele/gcache v0.0.2
 	github.com/stretchr/testify v1.8.0
 	golang.org/x/exp v0.0.0-20220827204233-334a2380cb91
diff --git a/go/adbc/go.sum b/go/adbc/go.sum
index da79f28..17c7412 100644
--- a/go/adbc/go.sum
+++ b/go/adbc/go.sum
@@ -3,12 +3,8 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03
 github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU=
 github.com/andybalholm/brotli v1.0.4 h1:V7DdXeJtZscaqfNuAdSRuRFzuiKlHSC/Zh3zl9qY3JY=
 github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig=
-github.com/apache/arrow/go/v11 v11.0.0-20230126145647-17ea6fcb5a38 h1:U2LbLZeKxttVkT1v1jjhM/Au2zQx7TZ9g1JCgmRhciM=
-github.com/apache/arrow/go/v11 v11.0.0-20230126145647-17ea6fcb5a38/go.mod h1:ypSOMuPhmiWVOBC7Or8cj/oPiROuXFS2ksXQWGoPRAo=
-github.com/apache/arrow/go/v11 v11.0.0-20230126170101-af400a81e698 h1:YYT0s7erFYXn8dQkjXv8NsenTedMKghVLsHQpl5h7bc=
-github.com/apache/arrow/go/v11 v11.0.0-20230126170101-af400a81e698/go.mod h1:ypSOMuPhmiWVOBC7Or8cj/oPiROuXFS2ksXQWGoPRAo=
-github.com/apache/arrow/go/v11 v11.0.0-20230126180639-8f537ca93922 h1:bTyKthK03dxd4t0gbASzx79uhNiUk16UKde38kUTmLo=
-github.com/apache/arrow/go/v11 v11.0.0-20230126180639-8f537ca93922/go.mod h1:ypSOMuPhmiWVOBC7Or8cj/oPiROuXFS2ksXQWGoPRAo=
+github.com/apache/arrow/go/v12 v12.0.0-20230130180348-64a48c999345 h1:cAh8divc591pQ1bx17osOVsBJjtfC5KFTuWaxNVXC4A=
+github.com/apache/arrow/go/v12 v12.0.0-20230130180348-64a48c999345/go.mod h1:qwJWRMGOu/SUTlFiQBgCjtOoxYqdG2KqXiDB3+e7BNA=
 github.com/apache/thrift v0.16.0 h1:qEy6UW60iVOlUy+b9ZR0d5WzUWYGOo4HfopoyBaNmoY=
 github.com/apache/thrift v0.16.0/go.mod h1:PHK3hniurgQaNMZYaCLEqXKsYK8upmhPbmdP2FXSqgU=
 github.com/bluele/gcache v0.0.2 h1:WcbfdXICg7G/DGBh1PFfcirkWOQV+v077yF1pSy3DGw=
diff --git a/go/adbc/pkg/_tmpl/driver.go.tmpl b/go/adbc/pkg/_tmpl/driver.go.tmpl
index 19101c5..8920915 100644
--- a/go/adbc/pkg/_tmpl/driver.go.tmpl
+++ b/go/adbc/pkg/_tmpl/driver.go.tmpl
@@ -39,8 +39,8 @@ import (
 	"unsafe"
 
 	"github.com/apache/arrow-adbc/go/adbc"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/cdata"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/cdata"
 )
 
 var drv = {{.Driver}}{}
diff --git a/go/adbc/pkg/flightsql/driver.go b/go/adbc/pkg/flightsql/driver.go
index 2fc916b..b48a51c 100644
--- a/go/adbc/pkg/flightsql/driver.go
+++ b/go/adbc/pkg/flightsql/driver.go
@@ -42,8 +42,8 @@ import (
 
 	"github.com/apache/arrow-adbc/go/adbc"
 	"github.com/apache/arrow-adbc/go/adbc/driver/flightsql"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/cdata"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/cdata"
 )
 
 var drv = flightsql.Driver{}
diff --git a/go/adbc/sqldriver/driver.go b/go/adbc/sqldriver/driver.go
index 9e13b27..e838db7 100644
--- a/go/adbc/sqldriver/driver.go
+++ b/go/adbc/sqldriver/driver.go
@@ -44,11 +44,11 @@ import (
 	"unsafe"
 
 	"github.com/apache/arrow-adbc/go/adbc"
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/decimal128"
-	"github.com/apache/arrow/go/v11/arrow/decimal256"
-	"github.com/apache/arrow/go/v11/arrow/memory"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/decimal128"
+	"github.com/apache/arrow/go/v12/arrow/decimal256"
+	"github.com/apache/arrow/go/v12/arrow/memory"
 )
 
 func getIsolationlevel(lvl sql.IsolationLevel) adbc.OptionIsolationLevel {
diff --git a/go/adbc/sqldriver/driver_internals_test.go b/go/adbc/sqldriver/driver_internals_test.go
index f913351..d35035e 100644
--- a/go/adbc/sqldriver/driver_internals_test.go
+++ b/go/adbc/sqldriver/driver_internals_test.go
@@ -25,9 +25,9 @@ import (
 	"time"
 
 	"github.com/apache/arrow-adbc/go/adbc"
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/memory"
+	"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/stretchr/testify/assert"
 	"github.com/stretchr/testify/require"
 )
diff --git a/go/adbc/standard_schemas.go b/go/adbc/standard_schemas.go
index 303c7fb..6bf6c52 100644
--- a/go/adbc/standard_schemas.go
+++ b/go/adbc/standard_schemas.go
@@ -17,7 +17,7 @@
 
 package adbc
 
-import "github.com/apache/arrow/go/v11/arrow"
+import "github.com/apache/arrow/go/v12/arrow"
 
 var (
 	GetInfoSchema = arrow.NewSchema([]arrow.Field{
diff --git a/go/adbc/validation/validation.go b/go/adbc/validation/validation.go
index da58544..e3e8212 100644
--- a/go/adbc/validation/validation.go
+++ b/go/adbc/validation/validation.go
@@ -27,10 +27,10 @@ import (
 	"testing"
 
 	"github.com/apache/arrow-adbc/go/adbc"
-	"github.com/apache/arrow/go/v11/arrow"
-	"github.com/apache/arrow/go/v11/arrow/array"
-	"github.com/apache/arrow/go/v11/arrow/flight/flightsql"
-	"github.com/apache/arrow/go/v11/arrow/memory"
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v12/arrow/memory"
 	"github.com/stretchr/testify/suite"
 )