You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ze...@apache.org on 2023/01/26 19:03:53 UTC

[arrow-adbc] branch main updated: feat(go/adbc/driver/flightsql): add timeout option handling (#390)

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

zeroshade 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 a848b20  feat(go/adbc/driver/flightsql): add timeout option handling (#390)
a848b20 is described below

commit a848b20c4a1b7ef0e2cd8305f29120a6b4fe13f7
Author: Matt Topol <zo...@gmail.com>
AuthorDate: Thu Jan 26 14:03:48 2023 -0500

    feat(go/adbc/driver/flightsql): add timeout option handling (#390)
    
    * feat(go/adbc/driver/flightsql): add timeout option handling
    
    * fix go.sum with go mod tidy
    
    * bump arrow version
    
    * settle down pre-commit
    
    * update arrow version, add go debug build options
    
    * update arrow version again
---
 c/cmake_modules/GoUtils.cmake                   |  22 ++-
 go/adbc/driver/flightsql/flightsql_adbc.go      | 162 +++++++++++++++++++++--
 go/adbc/driver/flightsql/flightsql_adbc_test.go | 169 ++++++++++++++++++++++++
 go/adbc/driver/flightsql/flightsql_statement.go |  50 +++++--
 go/adbc/driver/flightsql/record_reader.go       |   7 +-
 go/adbc/go.mod                                  |  18 +--
 go/adbc/go.sum                                  |  40 +++---
 7 files changed, 411 insertions(+), 57 deletions(-)

diff --git a/c/cmake_modules/GoUtils.cmake b/c/cmake_modules/GoUtils.cmake
index d936fdf..cf1a7e9 100644
--- a/c/cmake_modules/GoUtils.cmake
+++ b/c/cmake_modules/GoUtils.cmake
@@ -70,6 +70,16 @@ function(add_go_lib GO_MOD_DIR GO_LIBNAME)
     endif()
   endif()
 
+  # Go gcflags for disabling optimizations and inlining if debug
+  separate_arguments(GO_BUILD_FLAGS NATIVE_COMMAND
+                     "${GO_BUILD_FLAGS} $<$<CONFIG:DEBUG>:-gcflags=\"-N -l\">")
+
+  # if we're building debug mode then change the default CGO_CFLAGS and CGO_CXXFLAGS from "-g O2" to "-g3"
+  set(GO_ENV_VARS
+      "CGO_ENABLED=1 $<$<CONFIG:DEBUG>:CGO_CFLAGS=-g3> $<$<CONFIG:DEBUG>:CGO_CXXFLAGS=-g3>"
+  )
+  separate_arguments(GO_ENV_VARS NATIVE_COMMAND "${GO_ENV_VARS}")
+
   if(BUILD_SHARED)
     set(LIB_NAME_SHARED
         "${CMAKE_SHARED_LIBRARY_PREFIX}${GO_LIBNAME}${CMAKE_SHARED_LIBRARY_SUFFIX}")
@@ -86,7 +96,6 @@ function(add_go_lib GO_MOD_DIR GO_LIBNAME)
     endif()
 
     set(LIBOUT_SHARED "${CMAKE_CURRENT_BINARY_DIR}/${LIB_NAME_SHARED}")
-    set(GO_ENV_VARS "CGO_ENABLED=1")
 
     if(CMAKE_OSX_ARCHITECTURES STREQUAL "x86_64")
       list(APPEND GO_ENV_VARS "GOARCH=amd64")
@@ -97,10 +106,10 @@ function(add_go_lib GO_MOD_DIR GO_LIBNAME)
     add_custom_command(OUTPUT "${LIBOUT_SHARED}.${ADBC_FULL_SO_VERSION}"
                        WORKING_DIRECTORY ${GO_MOD_DIR}
                        DEPENDS ${ARG_SOURCES}
-                       COMMAND ${CMAKE_COMMAND} -E env "${GO_ENV_VARS}" ${GO_BIN} build
-                               "${GO_BUILD_TAGS}" "${GO_BUILD_FLAGS}" -o
-                               "${LIBOUT_SHARED}.${ADBC_FULL_SO_VERSION}"
-                               -buildmode=c-shared "${GO_LDFLAGS}" .
+                       COMMAND ${CMAKE_COMMAND} -E env ${GO_ENV_VARS} ${GO_BIN} build
+                               ${GO_BUILD_TAGS} "${GO_BUILD_FLAGS}" -o
+                               ${LIBOUT_SHARED}.${ADBC_FULL_SO_VERSION}
+                               -buildmode=c-shared ${GO_LDFLAGS} .
                        COMMAND ${CMAKE_COMMAND} -E remove -f
                                "${LIBOUT_SHARED}.${ADBC_SO_VERSION}.0.h"
                        COMMENT "Building Go Shared lib ${GO_LIBNAME}"
@@ -173,7 +182,8 @@ function(add_go_lib GO_MOD_DIR GO_LIBNAME)
     add_custom_command(OUTPUT "${LIBOUT_STATIC}"
                        WORKING_DIRECTORY ${GO_MOD_DIR}
                        DEPENDS ${ARG_SOURCES}
-                       COMMAND ${GO_BIN} build "${GO_BUILD_TAGS}" -o "${LIBOUT_STATIC}"
+                       COMMAND ${CMAKE_COMMAND} -E env "${GO_ENV_VARS}" ${GO_BIN} build
+                               "${GO_BUILD_TAGS}" -o "${LIBOUT_STATIC}"
                                -buildmode=c-archive "${GO_BUILD_FLAGS}" .
                        COMMAND ${CMAKE_COMMAND} -E remove -f "${LIBOUT_HEADER}"
                        COMMENT "Building Go Static lib ${GO_LIBNAME}"
diff --git a/go/adbc/driver/flightsql/flightsql_adbc.go b/go/adbc/driver/flightsql/flightsql_adbc.go
index 3d483eb..4c51a7b 100644
--- a/go/adbc/driver/flightsql/flightsql_adbc.go
+++ b/go/adbc/driver/flightsql/flightsql_adbc.go
@@ -36,11 +36,14 @@ import (
 	"context"
 	"crypto/tls"
 	"crypto/x509"
+	"errors"
 	"fmt"
 	"io"
+	"math"
 	"net/url"
 	"regexp"
 	"runtime/debug"
+	"strconv"
 	"strings"
 	"time"
 
@@ -68,6 +71,9 @@ const (
 	OptionSSLSkipVerify       = "adbc.flight.sql.client_option.tls_skip_verify"
 	OptionSSLRootCerts        = "adbc.flight.sql.client_option.tls_root_certs"
 	OptionAuthorizationHeader = "adbc.flight.sql.authorization_header"
+	OptionTimeoutFetch        = "adbc.flight.sql.rpc.timeout_seconds.fetch"
+	OptionTimeoutQuery        = "adbc.flight.sql.rpc.timeout_seconds.query"
+	OptionTimeoutUpdate       = "adbc.flight.sql.rpc.timeout_seconds.update"
 	OptionRPCCallHeaderPrefix = "adbc.flight.sql.rpc.call_header."
 	infoDriverName            = "ADBC Flight SQL Driver - Go"
 )
@@ -90,6 +96,14 @@ func init() {
 	}
 }
 
+func getTimeoutOptionValue(v string) (time.Duration, error) {
+	timeout, err := strconv.ParseFloat(v, 64)
+	if math.IsNaN(timeout) || math.IsInf(timeout, 0) || timeout < 0 {
+		return 0, errors.New("timeout must be positive and finite")
+	}
+	return time.Duration(timeout * float64(time.Second)), err
+}
+
 type Driver struct {
 	Alloc memory.Allocator
 }
@@ -123,6 +137,7 @@ type database struct {
 	creds      credentials.TransportCredentials
 	user, pass string
 	hdrs       metadata.MD
+	timeout    timeoutOption
 
 	alloc memory.Allocator
 }
@@ -216,6 +231,37 @@ func (d *database) SetOptions(cnOptions map[string]string) error {
 		delete(cnOptions, adbc.OptionKeyPassword)
 	}
 
+	var err error
+	if tv, ok := cnOptions[OptionTimeoutFetch]; ok {
+		if d.timeout.fetchTimeout, err = getTimeoutOptionValue(tv); err != nil {
+			return adbc.Error{
+				Msg: fmt.Sprintf("invalid timeout option value %s = %s : %s",
+					OptionTimeoutFetch, tv, err.Error()),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+	}
+
+	if tv, ok := cnOptions[OptionTimeoutQuery]; ok {
+		if d.timeout.queryTimeout, err = getTimeoutOptionValue(tv); err != nil {
+			return adbc.Error{
+				Msg: fmt.Sprintf("invalid timeout option value %s = %s : %s",
+					OptionTimeoutQuery, tv, err.Error()),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+	}
+
+	if tv, ok := cnOptions[OptionTimeoutUpdate]; ok {
+		if d.timeout.updateTimeout, err = getTimeoutOptionValue(tv); err != nil {
+			return adbc.Error{
+				Msg: fmt.Sprintf("invalid timeout option value %s = %s : %s",
+					OptionTimeoutUpdate, tv, err.Error()),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+	}
+
 	for key, val := range cnOptions {
 		if strings.HasPrefix(key, OptionRPCCallHeaderPrefix) {
 			d.hdrs.Append(strings.TrimPrefix(key, OptionRPCCallHeaderPrefix), val)
@@ -230,6 +276,57 @@ func (d *database) SetOptions(cnOptions map[string]string) error {
 	return nil
 }
 
+type timeoutOption struct {
+	grpc.EmptyCallOption
+
+	// timeout for DoGet requests
+	fetchTimeout time.Duration
+	// timeout for GetFlightInfo requests
+	queryTimeout time.Duration
+	// timeout for DoPut or DoAction requests
+	updateTimeout time.Duration
+}
+
+func getTimeout(method string, callOptions []grpc.CallOption) (time.Duration, bool) {
+	for _, opt := range callOptions {
+		if to, ok := opt.(timeoutOption); ok {
+			var tm time.Duration
+			switch {
+			case strings.HasSuffix(method, "DoGet"):
+				tm = to.fetchTimeout
+			case strings.HasSuffix(method, "GetFlightInfo"):
+				tm = to.queryTimeout
+			case strings.HasSuffix(method, "DoPut") || strings.HasSuffix(method, "DoAction"):
+				tm = to.updateTimeout
+			}
+
+			return tm, tm > 0
+		}
+	}
+
+	return 0, false
+}
+
+func unaryTimeoutInterceptor(ctx context.Context, method string, req, reply any, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+	if tm, ok := getTimeout(method, opts); ok {
+		ctx, cancel := context.WithTimeout(ctx, tm)
+		defer cancel()
+		return invoker(ctx, method, req, reply, cc, opts...)
+	}
+
+	return invoker(ctx, method, req, reply, cc, opts...)
+}
+
+func streamTimeoutInterceptor(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
+	if tm, ok := getTimeout(method, opts); ok {
+		ctx, cancel := context.WithTimeout(ctx, tm)
+		defer cancel()
+		return streamer(ctx, desc, cc, method, opts...)
+	}
+
+	return streamer(ctx, desc, cc, method, opts...)
+}
+
 type bearerAuthMiddleware struct {
 	hdrs metadata.MD
 }
@@ -241,6 +338,13 @@ func (b *bearerAuthMiddleware) StartCall(ctx context.Context) context.Context {
 
 func getFlightClient(ctx context.Context, loc string, d *database) (*flightsql.Client, error) {
 	authMiddle := &bearerAuthMiddleware{hdrs: d.hdrs.Copy()}
+	middleware := []flight.ClientMiddleware{
+		flight.CreateClientMiddleware(authMiddle),
+		{
+			Unary:  unaryTimeoutInterceptor,
+			Stream: streamTimeoutInterceptor,
+		},
+	}
 
 	uri, err := url.Parse(loc)
 	if err != nil {
@@ -251,8 +355,7 @@ func getFlightClient(ctx context.Context, loc string, d *database) (*flightsql.C
 		creds = insecure.NewCredentials()
 	}
 
-	cl, err := flightsql.NewClient(uri.Host, nil, []flight.ClientMiddleware{
-		flight.CreateClientMiddleware(authMiddle)}, grpc.WithTransportCredentials(creds))
+	cl, err := flightsql.NewClient(uri.Host, nil, middleware, grpc.WithTransportCredentials(creds))
 	if err != nil {
 		return nil, adbc.Error{
 			Msg:  err.Error(),
@@ -304,7 +407,8 @@ func (d *database) Open(ctx context.Context) (adbc.Connection, error) {
 			conn := client.(*flightsql.Client)
 			conn.Close()
 		}).Build()
-	return &cnxn{cl: cl, db: d, clientCache: cache, hdrs: make(metadata.MD)}, nil
+	return &cnxn{cl: cl, db: d, clientCache: cache,
+		hdrs: make(metadata.MD), timeouts: d.timeout}, nil
 }
 
 type cnxn struct {
@@ -313,6 +417,7 @@ type cnxn struct {
 	db          *database
 	clientCache gcache.Cache
 	hdrs        metadata.MD
+	timeouts    timeoutOption
 }
 
 var adbcToFlightSQLInfo = map[adbc.InfoCode]flightsql.SqlInfo{
@@ -321,9 +426,9 @@ var adbcToFlightSQLInfo = map[adbc.InfoCode]flightsql.SqlInfo{
 	adbc.InfoVendorArrowVersion: flightsql.SqlInfoFlightSqlServerArrowVersion,
 }
 
-func doGet(ctx context.Context, cl *flightsql.Client, endpoint *flight.FlightEndpoint, clientCache gcache.Cache) (rdr *flight.Reader, err error) {
+func doGet(ctx context.Context, cl *flightsql.Client, endpoint *flight.FlightEndpoint, clientCache gcache.Cache, opts ...grpc.CallOption) (rdr *flight.Reader, err error) {
 	if len(endpoint.Location) == 0 {
-		return cl.DoGet(ctx, endpoint.Ticket)
+		return cl.DoGet(ctx, endpoint.Ticket, opts...)
 	}
 
 	var (
@@ -337,7 +442,7 @@ func doGet(ctx context.Context, cl *flightsql.Client, endpoint *flight.FlightEnd
 		}
 
 		conn := cc.(*flightsql.Client)
-		rdr, err = conn.DoGet(ctx, endpoint.Ticket)
+		rdr, err = conn.DoGet(ctx, endpoint.Ticket, opts...)
 		if err != nil {
 			continue
 		}
@@ -360,6 +465,36 @@ func (c *cnxn) SetOption(key, value string) error {
 	}
 
 	switch key {
+	case OptionTimeoutFetch:
+		timeout, err := getTimeoutOptionValue(value)
+		if err != nil {
+			return adbc.Error{
+				Msg: fmt.Sprintf("invalid timeout option value %s = %s : %s",
+					OptionTimeoutFetch, value, err.Error()),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+		c.timeouts.fetchTimeout = timeout
+	case OptionTimeoutQuery:
+		timeout, err := getTimeoutOptionValue(value)
+		if err != nil {
+			return adbc.Error{
+				Msg: fmt.Sprintf("invalid timeout option value %s = %s : %s",
+					OptionTimeoutFetch, value, err.Error()),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+		c.timeouts.queryTimeout = timeout
+	case OptionTimeoutUpdate:
+		timeout, err := getTimeoutOptionValue(value)
+		if err != nil {
+			return adbc.Error{
+				Msg: fmt.Sprintf("invalid timeout option value %s = %s : %s",
+					OptionTimeoutFetch, value, err.Error()),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+		c.timeouts.updateTimeout = timeout
 	case adbc.OptionKeyAutoCommit:
 		return adbc.Error{
 			Msg:  "[Flight SQL] transactions not yet supported",
@@ -371,6 +506,8 @@ func (c *cnxn) SetOption(key, value string) error {
 			Code: adbc.StatusNotImplemented,
 		}
 	}
+
+	return nil
 }
 
 // GetInfo returns metadata about the database/driver.
@@ -436,13 +573,13 @@ func (c *cnxn) GetInfo(ctx context.Context, infoCodes []adbc.InfoCode) (array.Re
 	}
 
 	ctx = metadata.NewOutgoingContext(ctx, c.hdrs)
-	info, err := c.cl.GetSqlInfo(ctx, translated)
+	info, err := c.cl.GetSqlInfo(ctx, translated, c.timeouts)
 	if err != nil {
 		return nil, adbcFromFlightStatus(err)
 	}
 
 	for _, endpoint := range info.Endpoint {
-		rdr, err := doGet(ctx, c.cl, endpoint, c.clientCache)
+		rdr, err := doGet(ctx, c.cl, endpoint, c.clientCache, c.timeouts)
 		if err != nil {
 			return nil, adbcFromFlightStatus(err)
 		}
@@ -993,12 +1130,12 @@ func (c *cnxn) GetTableSchema(ctx context.Context, catalog *string, dbSchema *st
 	}
 
 	ctx = metadata.NewOutgoingContext(ctx, c.hdrs)
-	info, err := c.cl.GetTables(ctx, opts)
+	info, err := c.cl.GetTables(ctx, opts, c.timeouts)
 	if err != nil {
 		return nil, adbcFromFlightStatus(err)
 	}
 
-	rdr, err := doGet(ctx, c.cl, info.Endpoint[0], c.clientCache)
+	rdr, err := doGet(ctx, c.cl, info.Endpoint[0], c.clientCache, c.timeouts)
 	if err != nil {
 		return nil, adbcFromFlightStatus(err)
 	}
@@ -1045,7 +1182,7 @@ func (c *cnxn) GetTableSchema(ctx context.Context, catalog *string, dbSchema *st
 //
 func (c *cnxn) GetTableTypes(ctx context.Context) (array.RecordReader, error) {
 	ctx = metadata.NewOutgoingContext(ctx, c.hdrs)
-	info, err := c.cl.GetTableTypes(ctx)
+	info, err := c.cl.GetTableTypes(ctx, c.timeouts)
 	if err != nil {
 		return nil, adbcFromFlightStatus(err)
 	}
@@ -1085,6 +1222,7 @@ func (c *cnxn) NewStatement() (adbc.Statement, error) {
 		clientCache: c.clientCache,
 		hdrs:        c.hdrs.Copy(),
 		queueSize:   5,
+		timeouts:    c.timeouts,
 	}, nil
 }
 
@@ -1124,7 +1262,7 @@ func (c *cnxn) ReadPartition(ctx context.Context, serializedPartition []byte) (r
 	}
 
 	ctx = metadata.NewOutgoingContext(ctx, c.hdrs)
-	rdr, err = doGet(ctx, c.cl, info.Endpoint[0], c.clientCache)
+	rdr, err = doGet(ctx, c.cl, info.Endpoint[0], c.clientCache, c.timeouts)
 	if err != nil {
 		return nil, adbcFromFlightStatus(err)
 	}
diff --git a/go/adbc/driver/flightsql/flightsql_adbc_test.go b/go/adbc/driver/flightsql/flightsql_adbc_test.go
index 026807b..398f379 100644
--- a/go/adbc/driver/flightsql/flightsql_adbc_test.go
+++ b/go/adbc/driver/flightsql/flightsql_adbc_test.go
@@ -226,6 +226,7 @@ func TestADBCFlightSQL(t *testing.T) {
 	suite.Run(t, &SSLTests{Quirks: q})
 	suite.Run(t, &StatementTests{Quirks: q})
 	suite.Run(t, &HeaderTests{Quirks: q})
+	suite.Run(t, new(TimeoutTestSuite))
 }
 
 // Driver-specific tests
@@ -534,3 +535,171 @@ func (suite *HeaderTests) TestCombined() {
 	suite.Contains(suite.Quirks.middle.recordedHeaders.Get("x-header-two"), "value 2")
 	suite.Contains(suite.Quirks.middle.recordedHeaders.Get("x-header-three"), "value 3")
 }
+
+type TimeoutTestServer struct {
+	flightsql.BaseServer
+}
+
+func (ts *TimeoutTestServer) DoGetStatement(ctx context.Context, _ flightsql.StatementQueryTicket) (*arrow.Schema, <-chan flight.StreamChunk, error) {
+	// wait till the context is cancelled
+	<-ctx.Done()
+	return nil, nil, arrow.ErrNotImplemented
+}
+
+func (ts *TimeoutTestServer) DoPutCommandStatementUpdate(ctx context.Context, cmd flightsql.StatementUpdate) (int64, error) {
+	if cmd.GetQuery() == "timeout" {
+		<-ctx.Done()
+	}
+	return 0, arrow.ErrNotImplemented
+}
+
+func (ts *TimeoutTestServer) GetFlightInfoStatement(ctx context.Context, cmd flightsql.StatementQuery, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) {
+	switch cmd.GetQuery() {
+	case "timeout":
+		<-ctx.Done()
+	case "fetch":
+		tkt, _ := flightsql.CreateStatementQueryTicket([]byte("fetch"))
+		info := &flight.FlightInfo{
+			FlightDescriptor: desc,
+			Endpoint: []*flight.FlightEndpoint{
+				{Ticket: &flight.Ticket{Ticket: tkt}},
+			},
+			TotalRecords: -1,
+			TotalBytes:   -1,
+		}
+		return info, nil
+	}
+	return nil, arrow.ErrNotImplemented
+}
+
+type TimeoutTestSuite struct {
+	suite.Suite
+
+	s    flight.Server
+	db   adbc.Database
+	cnxn adbc.Connection
+}
+
+func (ts *TimeoutTestSuite) SetupSuite() {
+	ts.s = flight.NewServerWithMiddleware(nil)
+	ts.s.RegisterFlightService(flightsql.NewFlightServer(&TimeoutTestServer{}))
+	ts.Require().NoError(ts.s.Init("localhost:0"))
+	ts.s.SetShutdownOnSignals(os.Interrupt, os.Kill)
+	go func() {
+		_ = ts.s.Serve()
+	}()
+
+	uri := "grpc+tcp://" + ts.s.Addr().String()
+	var err error
+	ts.db, err = (driver.Driver{}).NewDatabase(map[string]string{
+		"uri": uri,
+	})
+	ts.Require().NoError(err)
+}
+
+func (ts *TimeoutTestSuite) SetupTest() {
+	var err error
+	ts.cnxn, err = ts.db.Open(context.Background())
+	ts.Require().NoError(err)
+}
+
+func (ts *TimeoutTestSuite) TearDownTest() {
+	ts.Require().NoError(ts.cnxn.Close())
+}
+
+func (ts *TimeoutTestSuite) TearDownSuite() {
+	ts.db = nil
+	ts.s.Shutdown()
+}
+
+func (ts *TimeoutTestSuite) TestInvalidValues() {
+	keys := []string{
+		"adbc.flight.sql.rpc.timeout_seconds.fetch",
+		"adbc.flight.sql.rpc.timeout_seconds.query",
+		"adbc.flight.sql.rpc.timeout_seconds.update",
+	}
+	values := []string{"1.1f", "asdf", "inf", "NaN", "-1"}
+
+	for _, k := range keys {
+		for _, v := range values {
+			ts.Run("key="+k+",val="+v, func() {
+				err := ts.cnxn.(adbc.PostInitOptions).SetOption(k, v)
+				var adbcErr adbc.Error
+				ts.ErrorAs(err, &adbcErr)
+				ts.Equal(adbc.StatusInvalidArgument, adbcErr.Code)
+				ts.ErrorContains(err, "invalid timeout option value")
+			})
+		}
+	}
+}
+
+func (ts *TimeoutTestSuite) TestRemoveTimeout() {
+	keys := []string{
+		"adbc.flight.sql.rpc.timeout_seconds.fetch",
+		"adbc.flight.sql.rpc.timeout_seconds.query",
+		"adbc.flight.sql.rpc.timeout_seconds.update",
+	}
+	for _, k := range keys {
+		ts.Run(k, func() {
+			ts.NoError(ts.cnxn.(adbc.PostInitOptions).SetOption(k, "1.0"))
+			ts.NoError(ts.cnxn.(adbc.PostInitOptions).SetOption(k, "0"))
+		})
+	}
+}
+
+func (ts *TimeoutTestSuite) TestDoActionTimeout() {
+	ts.NoError(ts.cnxn.(adbc.PostInitOptions).
+		SetOption("adbc.flight.sql.rpc.timeout_seconds.update", "0.1"))
+
+	stmt, err := ts.cnxn.NewStatement()
+	ts.Require().NoError(err)
+	defer stmt.Close()
+
+	ts.Require().NoError(stmt.SetSqlQuery("fetch"))
+	var adbcErr adbc.Error
+	ts.ErrorAs(stmt.Prepare(context.Background()), &adbcErr)
+	ts.Equal(adbc.StatusCancelled, adbcErr.Code)
+}
+
+func (ts *TimeoutTestSuite) TestDoGetTimeout() {
+	ts.NoError(ts.cnxn.(adbc.PostInitOptions).
+		SetOption("adbc.flight.sql.rpc.timeout_seconds.fetch", "0.1"))
+
+	stmt, err := ts.cnxn.NewStatement()
+	ts.Require().NoError(err)
+	defer stmt.Close()
+
+	ts.Require().NoError(stmt.SetSqlQuery("fetch"))
+	var adbcErr adbc.Error
+	_, _, err = stmt.ExecuteQuery(context.Background())
+	ts.ErrorAs(err, &adbcErr)
+	ts.Equal(adbc.StatusCancelled, adbcErr.Code)
+}
+
+func (ts *TimeoutTestSuite) TestDoPutTimeout() {
+	ts.NoError(ts.cnxn.(adbc.PostInitOptions).
+		SetOption("adbc.flight.sql.rpc.timeout_seconds.update", "0.1"))
+
+	stmt, err := ts.cnxn.NewStatement()
+	ts.Require().NoError(err)
+	defer stmt.Close()
+
+	ts.Require().NoError(stmt.SetSqlQuery("timeout"))
+	_, err = stmt.ExecuteUpdate(context.Background())
+	ts.Error(err)
+}
+
+func (ts *TimeoutTestSuite) TestGetFlightInfoTimeout() {
+	ts.NoError(ts.cnxn.(adbc.PostInitOptions).
+		SetOption("adbc.flight.sql.rpc.timeout_seconds.query", "0.1"))
+
+	stmt, err := ts.cnxn.NewStatement()
+	ts.Require().NoError(err)
+	defer stmt.Close()
+
+	ts.Require().NoError(stmt.SetSqlQuery("timeout"))
+	var adbcErr adbc.Error
+	_, _, err = stmt.ExecuteQuery(context.Background())
+	ts.ErrorAs(err, &adbcErr)
+	ts.NotEqual(adbc.StatusNotImplemented, adbcErr.Code)
+}
diff --git a/go/adbc/driver/flightsql/flightsql_statement.go b/go/adbc/driver/flightsql/flightsql_statement.go
index 0efbada..ccd0598 100644
--- a/go/adbc/driver/flightsql/flightsql_statement.go
+++ b/go/adbc/driver/flightsql/flightsql_statement.go
@@ -47,10 +47,11 @@ type statement struct {
 	query     string
 	prepared  *flightsql.PreparedStatement
 	queueSize int
+	timeouts  timeoutOption
 }
 
 func (s *statement) closePreparedStatement() error {
-	return s.prepared.Close(context.Background())
+	return s.prepared.Close(context.Background(), s.timeouts)
 }
 
 // Close releases any relevant resources associated with this statement
@@ -89,6 +90,36 @@ func (s *statement) SetOption(key string, val string) error {
 	}
 
 	switch key {
+	case OptionTimeoutFetch:
+		timeout, err := getTimeoutOptionValue(val)
+		if err != nil {
+			return adbc.Error{
+				Msg: fmt.Sprintf("invalid timeout option value %s = %s : %s",
+					OptionTimeoutFetch, val, err.Error()),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+		s.timeouts.fetchTimeout = timeout
+	case OptionTimeoutQuery:
+		timeout, err := getTimeoutOptionValue(val)
+		if err != nil {
+			return adbc.Error{
+				Msg: fmt.Sprintf("invalid timeout option value %s = %s : %s",
+					OptionTimeoutFetch, val, err.Error()),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+		s.timeouts.queryTimeout = timeout
+	case OptionTimeoutUpdate:
+		timeout, err := getTimeoutOptionValue(val)
+		if err != nil {
+			return adbc.Error{
+				Msg: fmt.Sprintf("invalid timeout option value %s = %s : %s",
+					OptionTimeoutFetch, val, err.Error()),
+				Code: adbc.StatusInvalidArgument,
+			}
+		}
+		s.timeouts.updateTimeout = timeout
 	case OptionStatementQueueSize:
 		var err error
 		var size int
@@ -111,6 +142,7 @@ func (s *statement) SetOption(key string, val string) error {
 			Code: adbc.StatusNotImplemented,
 		}
 	}
+	return nil
 }
 
 // SetSqlQuery sets the query string to be executed.
@@ -139,9 +171,9 @@ func (s *statement) ExecuteQuery(ctx context.Context) (rdr array.RecordReader, n
 	ctx = metadata.NewOutgoingContext(ctx, s.hdrs)
 	var info *flight.FlightInfo
 	if s.prepared != nil {
-		info, err = s.prepared.Execute(ctx)
+		info, err = s.prepared.Execute(ctx, s.timeouts)
 	} else if s.query != "" {
-		info, err = s.cl.Execute(ctx, 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",
@@ -154,7 +186,7 @@ func (s *statement) ExecuteQuery(ctx context.Context) (rdr array.RecordReader, n
 	}
 
 	nrec = info.TotalRecords
-	rdr, err = newRecordReader(ctx, s.alloc, s.cl, info, s.clientCache, s.queueSize)
+	rdr, err = newRecordReader(ctx, s.alloc, s.cl, info, s.clientCache, s.queueSize, s.timeouts)
 	return
 }
 
@@ -163,11 +195,11 @@ func (s *statement) ExecuteQuery(ctx context.Context) (rdr array.RecordReader, n
 func (s *statement) ExecuteUpdate(ctx context.Context) (int64, error) {
 	ctx = metadata.NewOutgoingContext(ctx, s.hdrs)
 	if s.prepared != nil {
-		return s.prepared.ExecuteUpdate(ctx)
+		return s.prepared.ExecuteUpdate(ctx, s.timeouts)
 	}
 
 	if s.query != "" {
-		return s.cl.ExecuteUpdate(ctx, s.query)
+		return s.cl.ExecuteUpdate(ctx, s.query, s.timeouts)
 	}
 
 	return -1, adbc.Error{
@@ -187,7 +219,7 @@ func (s *statement) Prepare(ctx context.Context) error {
 		}
 	}
 
-	prep, err := s.cl.Prepare(ctx, s.alloc, s.query)
+	prep, err := s.cl.Prepare(ctx, s.alloc, s.query, s.timeouts)
 	if err != nil {
 		return adbcFromFlightStatus(err)
 	}
@@ -299,9 +331,9 @@ func (s *statement) ExecutePartitions(ctx context.Context) (*arrow.Schema, adbc.
 	)
 
 	if s.prepared != nil {
-		info, err = s.prepared.Execute(ctx)
+		info, err = s.prepared.Execute(ctx, s.timeouts)
 	} else if s.query != "" {
-		info, err = s.cl.Execute(ctx, 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",
diff --git a/go/adbc/driver/flightsql/record_reader.go b/go/adbc/driver/flightsql/record_reader.go
index 4470855..c90e330 100644
--- a/go/adbc/driver/flightsql/record_reader.go
+++ b/go/adbc/driver/flightsql/record_reader.go
@@ -29,6 +29,7 @@ import (
 	"github.com/apache/arrow/go/v11/arrow/memory"
 	"github.com/bluele/gcache"
 	"golang.org/x/sync/errgroup"
+	"google.golang.org/grpc"
 )
 
 type reader struct {
@@ -44,7 +45,7 @@ type reader struct {
 
 // kicks off a goroutine for each endpoint and returns a reader which
 // gathers all of the records as they come in.
-func newRecordReader(ctx context.Context, alloc memory.Allocator, cl *flightsql.Client, info *flight.FlightInfo, clCache gcache.Cache, bufferSize int) (rdr array.RecordReader, err error) {
+func newRecordReader(ctx context.Context, alloc memory.Allocator, cl *flightsql.Client, info *flight.FlightInfo, clCache gcache.Cache, bufferSize int, opts ...grpc.CallOption) (rdr array.RecordReader, err error) {
 	endpoints := info.Endpoint
 	var schema *arrow.Schema
 	if len(endpoints) == 0 {
@@ -85,7 +86,7 @@ func newRecordReader(ctx context.Context, alloc memory.Allocator, cl *flightsql.
 				Code: adbc.StatusInvalidState}
 		}
 	} else {
-		rdr, err := doGet(ctx, cl, endpoints[0], clCache)
+		rdr, err := doGet(ctx, cl, endpoints[0], clCache, opts...)
 		if err != nil {
 			return nil, adbcFromFlightStatus(err)
 		}
@@ -125,7 +126,7 @@ func newRecordReader(ctx context.Context, alloc memory.Allocator, cl *flightsql.
 				defer close(chs[endpointIndex])
 			}
 
-			rdr, err := doGet(ctx, cl, endpoint, clCache)
+			rdr, err := doGet(ctx, cl, endpoint, clCache, opts...)
 			if err != nil {
 				return err
 			}
diff --git a/go/adbc/go.mod b/go/adbc/go.mod
index f6dc610..fbe6c08 100644
--- a/go/adbc/go.mod
+++ b/go/adbc/go.mod
@@ -20,12 +20,12 @@ module github.com/apache/arrow-adbc/go/adbc
 go 1.18
 
 require (
-	github.com/apache/arrow/go/v11 v11.0.0-20230123220137-8449c553710a
+	github.com/apache/arrow/go/v11 v11.0.0-20230126180639-8f537ca93922
 	github.com/bluele/gcache v0.0.2
 	github.com/stretchr/testify v1.8.0
 	golang.org/x/exp v0.0.0-20220827204233-334a2380cb91
-	golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde
-	golang.org/x/tools v0.1.12
+	golang.org/x/sync v0.1.0
+	golang.org/x/tools v0.5.0
 	google.golang.org/grpc v1.49.0
 	google.golang.org/protobuf v1.28.1
 )
@@ -37,7 +37,7 @@ require (
 	github.com/goccy/go-json v0.9.11 // indirect
 	github.com/golang/protobuf v1.5.2 // indirect
 	github.com/golang/snappy v0.0.4 // indirect
-	github.com/google/flatbuffers v2.0.8+incompatible // indirect
+	github.com/google/flatbuffers v23.1.21+incompatible // indirect
 	github.com/google/uuid v1.3.0 // indirect
 	github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 // indirect
 	github.com/klauspost/asmfmt v1.3.2 // indirect
@@ -51,11 +51,11 @@ require (
 	github.com/pmezard/go-difflib v1.0.0 // indirect
 	github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect
 	github.com/zeebo/xxh3 v1.0.2 // indirect
-	golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4 // indirect
-	golang.org/x/net v0.0.0-20220722155237-a158d28d115b // indirect
-	golang.org/x/sys v0.0.0-20220829200755-d48e67d00261 // indirect
-	golang.org/x/text v0.3.7 // indirect
-	golang.org/x/xerrors v0.0.0-20220609144429-65e65417b02f // indirect
+	golang.org/x/mod v0.7.0 // indirect
+	golang.org/x/net v0.5.0 // indirect
+	golang.org/x/sys v0.4.0 // indirect
+	golang.org/x/text v0.6.0 // indirect
+	golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
 	google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013 // indirect
 	gopkg.in/yaml.v3 v3.0.1 // indirect
 	lukechampine.com/uint128 v1.2.0 // indirect
diff --git a/go/adbc/go.sum b/go/adbc/go.sum
index c62eb5d..da79f28 100644
--- a/go/adbc/go.sum
+++ b/go/adbc/go.sum
@@ -3,8 +3,12 @@ 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-20230123220137-8449c553710a h1:46k0d+c2UMW3TBn5Q4x1rynQb7ISPQegpQCROt9w3Fw=
-github.com/apache/arrow/go/v11 v11.0.0-20230123220137-8449c553710a/go.mod h1:Eg5OsL5H+e299f7u5ssuXsuHQVEGC4xei5aX110hRiI=
+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/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=
@@ -37,8 +41,8 @@ github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw
 github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
 github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM=
 github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
-github.com/google/flatbuffers v2.0.8+incompatible h1:ivUb1cGomAB101ZM1T0nOiWz9pSrTMoa9+EiY7igmkM=
-github.com/google/flatbuffers v2.0.8+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8=
+github.com/google/flatbuffers v23.1.21+incompatible h1:bUqzx/MXCDxuS0hRJL2EfjyZL3uQrPbMocUa8zGqsTA=
+github.com/google/flatbuffers v23.1.21+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8=
 github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
 github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
 github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
@@ -95,8 +99,8 @@ golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTk
 golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU=
 golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
 golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
-golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4 h1:6zppjxzCulZykYSLyVDYbneBfbaBIQPYMevg0bEwv2s=
-golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4=
+golang.org/x/mod v0.7.0 h1:LapD9S96VoQRhi/GrNTqeBJFrUjs5UHCAtTlgwA5oZA=
+golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs=
 golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
 golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
 golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
@@ -104,15 +108,15 @@ golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn
 golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
 golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
 golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
-golang.org/x/net v0.0.0-20220722155237-a158d28d115b h1:PxfKdU9lEEDYjdIzOtC4qFWgkU2rGHdKlKowJSMN9h0=
-golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
+golang.org/x/net v0.5.0 h1:GyT4nK/YDHSqa1c4753ouYCDajOYKTja9Xb/OHtgvSw=
+golang.org/x/net v0.5.0/go.mod h1:DivGGAXEgPSlEBzxGzZI+ZLohi+xUj054jfeKui00ws=
 golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
 golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde h1:ejfdSekXMDxDLbRrJMwUk6KnSLZ2McaUCVcIKM+N6jc=
-golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.1.0 h1:wsuoTGHzEhffawBOhz5CYhcrV4IdKZbEyZjBMuTp12o=
+golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
 golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
 golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@@ -120,12 +124,12 @@ golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7w
 golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
 golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
 golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220829200755-d48e67d00261 h1:v6hYoSR9T5oet+pMXwUWkbiVqx/63mlHjefrHmxwfeY=
-golang.org/x/sys v0.0.0-20220829200755-d48e67d00261/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/sys v0.4.0 h1:Zr2JFtRQNX3BCZ8YtxRE9hNJYC8J6I1MVbMg6owUp18=
+golang.org/x/sys v0.4.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
 golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
 golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
-golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk=
-golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ=
+golang.org/x/text v0.6.0 h1:3XmdazWV+ubf7QgHSTWeykHOci5oeekaGJBLkrkaw4k=
+golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8=
 golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
 golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
 golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY=
@@ -133,14 +137,14 @@ golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3
 golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
 golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
 golang.org/x/tools v0.0.0-20201124115921-2c860bdd6e78/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
-golang.org/x/tools v0.1.12 h1:VveCTK38A2rkS8ZqFY25HIDFscX5X9OoEhJd3quQmXU=
-golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc=
+golang.org/x/tools v0.5.0 h1:+bSpV5HIeWkuvgaMfI3UmKRThoTA5ODJTUd8T17NO+4=
+golang.org/x/tools v0.5.0/go.mod h1:N+Kgy78s5I24c24dU8OfWNEotWjutIs8SnJvn5IDq+k=
 golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
 golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
 golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
 golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
-golang.org/x/xerrors v0.0.0-20220609144429-65e65417b02f h1:uF6paiQQebLeSXkrTqHqz0MXhXXS1KgF41eUdBNvxK0=
-golang.org/x/xerrors v0.0.0-20220609144429-65e65417b02f/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8=
+golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 h1:H2TDz8ibqkAF6YGhCdN3jS9O0/s90v0rJh3X/OLHEUk=
+golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8=
 gonum.org/v1/gonum v0.11.0 h1:f1IJhK4Km5tBJmaiJXtk/PkL4cdVX6J+tGiM187uT5E=
 google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
 google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=