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/13 18:54:35 UTC

[arrow-adbc] branch main updated: feat(go/adbc/driver/flightsql): Native Golang ADBC Flight SQL driver (#322)

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 cbb1a38  feat(go/adbc/driver/flightsql): Native Golang ADBC Flight SQL driver (#322)
cbb1a38 is described below

commit cbb1a38584152ef0bc71c415533bd3172cc73870
Author: Matt Topol <zo...@gmail.com>
AuthorDate: Fri Jan 13 13:54:29 2023 -0500

    feat(go/adbc/driver/flightsql): Native Golang ADBC Flight SQL driver (#322)
    
    * feat(go/flightsql): Native Golang FlightSQL ADBC driver
    
    * don't pass db opts to connection
    
    * fix doc for uri
    
    * connection caching, record reader
    
    * implement execute partitions
    
    * simple tests for driver/db and connection
    
    * add generated file to the rat exclude
    
    * updating from go mod tidy -v
    
    * fix linting, check error returns
    
    * some more statement tests
    
    * weird timing issue?
    
    * fix golangci-lint
    
    * add go option constants
---
 .pre-commit-config.yaml                         |   4 +-
 dev/release/rat_exclude_files.txt               |   1 +
 go/adbc/adbc.go                                 |  20 +-
 go/adbc/driver/flightsql/flightsql_adbc.go      | 577 ++++++++++++++++++++++++
 go/adbc/driver/flightsql/flightsql_adbc_test.go | 205 +++++++++
 go/adbc/driver/flightsql/flightsql_statement.go | 290 ++++++++++++
 go/adbc/driver/flightsql/record_reader.go       | 154 +++++++
 go/adbc/driver/flightsql/utils.go               |  57 +++
 go/adbc/drivermgr/wrapper.go                    | 132 ++++--
 go/adbc/drivermgr/wrapper_sqlite_test.go        |  25 +-
 go/adbc/go.mod                                  |  25 +-
 go/adbc/go.sum                                  | 219 ++-------
 go/adbc/infocode_string.go                      |  39 ++
 go/adbc/sqldriver/driver.go                     |  30 +-
 go/adbc/standard_schemas.go                     |  94 ++++
 go/adbc/validation/validation.go                | 424 +++++++++++++++++
 16 files changed, 2036 insertions(+), 260 deletions(-)

diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 8523fa2..1a0ef8b 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -52,8 +52,8 @@ repos:
     rev: v1.49.0
     hooks:
     - id: golangci-lint
-      entry: bash -c 'cd go/adbc && golangci-lint run --fix'
-      args: [--fix]
+      entry: bash -c 'cd go/adbc && golangci-lint run --fix --timeout 2m'
+      types_or: [go]
   - repo: https://github.com/macisamuele/language-formatters-pre-commit-hooks
     rev: v2.3.0
     hooks:
diff --git a/dev/release/rat_exclude_files.txt b/dev/release/rat_exclude_files.txt
index 9d68138..36b36e0 100644
--- a/dev/release/rat_exclude_files.txt
+++ b/dev/release/rat_exclude_files.txt
@@ -7,5 +7,6 @@ go/adbc/drivermgr/adbc.h
 go/adbc/drivermgr/adbc_driver_manager.cc
 go/adbc/drivermgr/adbc_driver_manager.h
 go/adbc/status_string.go
+go/adbc/infocode_string.go
 go/adbc/go.sum
 rat.txt
diff --git a/go/adbc/adbc.go b/go/adbc/adbc.go
index bcfcc40..7191b26 100644
--- a/go/adbc/adbc.go
+++ b/go/adbc/adbc.go
@@ -45,6 +45,7 @@ import (
 )
 
 //go:generate go run golang.org/x/tools/cmd/stringer -type Status -linecomment
+//go:generate go run golang.org/x/tools/cmd/stringer -type InfoCode -linecomment
 
 // Error is the detailed error for an operation
 type Error struct {
@@ -152,6 +153,9 @@ const (
 	OptionKeyReadOnly           = "adbc.connection.readonly"
 	OptionValueIngestModeCreate = "adbc.ingest.mode.create"
 	OptionValueIngestModeAppend = "adbc.ingest.mode.append"
+	OptionKeyURI                = "uri"
+	OptionKeyUsername           = "username"
+	OptionKeyPassword           = "password"
 )
 
 type OptionIsolationLevel string
@@ -183,6 +187,10 @@ const (
 // A driver can also optionally implement io.Closer if there is a need
 // or desire for it.
 type Driver interface {
+	NewDatabase(opts map[string]string) (Database, error)
+}
+
+type Database interface {
 	SetOptions(map[string]string) error
 	Open(ctx context.Context) (Connection, error)
 }
@@ -192,18 +200,18 @@ type InfoCode uint32
 const (
 	// The database vendor/product name (e.g. the server name)
 	// (type: utf8)
-	InfoVendorName InfoCode = 0
+	InfoVendorName InfoCode = 0 // VendorName
 	// The database vendor/product version (type: utf8)
-	InfoVendorVersion InfoCode = 1
+	InfoVendorVersion InfoCode = 1 // VendorVersion
 	// The database vendor/product Arrow library version (type: utf8)
-	InfoVendorArrowVersion InfoCode = 2
+	InfoVendorArrowVersion InfoCode = 2 // VendorArrowVersion
 
 	// The driver name (type: utf8)
-	InfoDriverName InfoCode = 100
+	InfoDriverName InfoCode = 100 // DriverName
 	// The driver version (type: utf8)
-	InfoDriverVersion InfoCode = 101
+	InfoDriverVersion InfoCode = 101 // DriverVersion
 	// The driver Arrow library version (type: utf8)
-	InfoDriverArrowVersion InfoCode = 102
+	InfoDriverArrowVersion InfoCode = 102 // DriverArrowVersion
 )
 
 type ObjectDepth int
diff --git a/go/adbc/driver/flightsql/flightsql_adbc.go b/go/adbc/driver/flightsql/flightsql_adbc.go
new file mode 100644
index 0000000..ade6bb2
--- /dev/null
+++ b/go/adbc/driver/flightsql/flightsql_adbc.go
@@ -0,0 +1,577 @@
+// 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 flightsql is an ADBC Driver Implementation for Flight SQL
+// natively in go.
+//
+// It can be used to register a driver for database/sql by importing
+// github.com/apache/arrow-adbc/go/adbc/sqldriver and running:
+//
+//     sql.Register("flightsql", sqldriver.Driver{flightsql.Driver{}})
+//
+// You can then open a flightsql connection with the database/sql
+// standard package by using:
+//
+//     db, err := sql.Open("flightsql", "uri=<flight sql db url>")
+//
+// The URI passed *must* contain a scheme, most likely "grpc+tcp://"
+package flightsql
+
+import (
+	"context"
+	"crypto/tls"
+	"io"
+	"net/url"
+	"runtime/debug"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v10/arrow"
+	"github.com/apache/arrow/go/v10/arrow/array"
+	"github.com/apache/arrow/go/v10/arrow/flight"
+	"github.com/apache/arrow/go/v10/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v10/arrow/memory"
+	"github.com/bluele/gcache"
+	"golang.org/x/exp/maps"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/credentials"
+	"google.golang.org/grpc/credentials/insecure"
+	"google.golang.org/grpc/metadata"
+	"google.golang.org/protobuf/proto"
+)
+
+const (
+	OptionSSLInsecure = "adbc.flight.sql.client_option.tls_skip_verify"
+	OptionSSLCertFile = "adbc.flight.sql.client_option.tls_root_certs"
+
+	infoDriverName = "ADBC Flight SQL Driver - Go"
+)
+
+var (
+	infoDriverVersion      string
+	infoDriverArrowVersion string
+)
+
+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/flightsql":
+				infoDriverVersion = dep.Version
+			case strings.HasPrefix(dep.Path, "github.com/apache/arrow/go/"):
+				infoDriverArrowVersion = dep.Version
+			}
+		}
+	}
+}
+
+type Driver struct {
+	Alloc memory.Allocator
+}
+
+func (d Driver) NewDatabase(opts map[string]string) (adbc.Database, error) {
+	uri, ok := opts[adbc.OptionKeyURI]
+	if !ok {
+		return nil, adbc.Error{
+			Msg:  "URI required for a FlightSQL DB",
+			Code: adbc.StatusInvalidArgument,
+		}
+	}
+
+	db := &database{alloc: d.Alloc}
+	if db.alloc == nil {
+		db.alloc = memory.DefaultAllocator
+	}
+
+	var err error
+	if db.uri, err = url.Parse(uri); err != nil {
+		return nil, adbc.Error{Msg: err.Error(), Code: adbc.StatusInvalidArgument}
+	}
+
+	return db, db.SetOptions(opts)
+}
+
+type database struct {
+	uri        *url.URL
+	creds      credentials.TransportCredentials
+	user, pass string
+
+	alloc memory.Allocator
+}
+
+func (d *database) SetOptions(cnOptions map[string]string) error {
+	if val, ok := cnOptions[OptionSSLInsecure]; ok && val == adbc.OptionValueEnabled {
+		d.creds = insecure.NewCredentials()
+	} else {
+		// option specified path to certificate file
+		if cert, ok := cnOptions[OptionSSLCertFile]; ok {
+			c, err := credentials.NewClientTLSFromFile(cert, "")
+			if err != nil {
+				return adbc.Error{
+					Msg:  "invalid SSL certificate passed",
+					Code: adbc.StatusInvalidArgument,
+				}
+			}
+			d.creds = c
+		} else if d.creds == nil {
+			// use local root certificates
+			d.creds = credentials.NewTLS(&tls.Config{})
+		}
+	}
+
+	if u, ok := cnOptions[adbc.OptionKeyUsername]; ok {
+		d.user = u
+	}
+
+	if p, ok := cnOptions[adbc.OptionKeyPassword]; ok {
+		d.pass = p
+	}
+
+	return nil
+}
+
+type bearerAuthMiddleware struct {
+	token string
+}
+
+func (b *bearerAuthMiddleware) StartCall(ctx context.Context) context.Context {
+	if b.token != "" {
+		return metadata.AppendToOutgoingContext(ctx, "authorization", b.token)
+	}
+
+	return ctx
+}
+
+func getFlightClient(ctx context.Context, loc string, d *database) (*flightsql.Client, error) {
+	authMiddle := &bearerAuthMiddleware{}
+
+	cl, err := flightsql.NewClient(d.uri.Host, nil, []flight.ClientMiddleware{
+		flight.CreateClientMiddleware(authMiddle)}, grpc.WithTransportCredentials(d.creds))
+	if err != nil {
+		return nil, adbc.Error{
+			Msg:  err.Error(),
+			Code: adbc.StatusIO,
+		}
+	}
+
+	cl.Alloc = d.alloc
+	if d.user != "" {
+		ctx, err = cl.Client.AuthenticateBasicToken(ctx, d.user, d.pass)
+		if err != nil {
+			return nil, adbc.Error{
+				Msg:  err.Error(),
+				Code: adbc.StatusUnauthenticated,
+			}
+		}
+
+		if md, ok := metadata.FromOutgoingContext(ctx); ok {
+			authMiddle.token = md.Get("Authorization")[0]
+		}
+	}
+
+	return cl, nil
+}
+
+func (d *database) Open(ctx context.Context) (adbc.Connection, error) {
+	cl, err := getFlightClient(ctx, d.uri.Host, d)
+	if err != nil {
+		return nil, err
+	}
+
+	cache := gcache.New(20).LRU().
+		Expiration(5 * time.Minute).
+		LoaderFunc(func(loc interface{}) (interface{}, error) {
+			uri, ok := loc.(string)
+			if !ok {
+				return nil, adbc.Error{Code: adbc.StatusInternal}
+			}
+
+			cl, err := getFlightClient(context.Background(), uri, d)
+			if err != nil {
+				return nil, err
+			}
+
+			cl.Alloc = d.alloc
+			return cl, nil
+		}).
+		EvictedFunc(func(_, client interface{}) {
+			conn := client.(*flightsql.Client)
+			conn.Close()
+		}).Build()
+	return &cnxn{cl: cl, db: d, clientCache: cache}, nil
+}
+
+type cnxn struct {
+	cl *flightsql.Client
+
+	db          *database
+	clientCache gcache.Cache
+}
+
+var adbcToFlightSQLInfo = map[adbc.InfoCode]flightsql.SqlInfo{
+	adbc.InfoVendorName:         flightsql.SqlInfoFlightSqlServerName,
+	adbc.InfoVendorVersion:      flightsql.SqlInfoFlightSqlServerVersion,
+	adbc.InfoVendorArrowVersion: flightsql.SqlInfoFlightSqlServerArrowVersion,
+}
+
+func doGet(ctx context.Context, cl *flightsql.Client, endpoint *flight.FlightEndpoint, clientCache gcache.Cache) (rdr *flight.Reader, err error) {
+	if len(endpoint.Location) == 0 {
+		return cl.DoGet(ctx, endpoint.Ticket)
+	}
+
+	var (
+		cc interface{}
+	)
+
+	for _, loc := range endpoint.Location {
+		cc, err = clientCache.Get(loc)
+		if err != nil {
+			continue
+		}
+
+		conn := cc.(*flightsql.Client)
+		rdr, err = conn.DoGet(ctx, endpoint.Ticket)
+		if err != nil {
+			continue
+		}
+
+		return
+	}
+
+	return nil, err
+}
+
+// GetInfo returns metadata about the database/driver.
+//
+// The result is an Arrow dataset with the following schema:
+//
+//    Field Name									| Field Type
+//    ----------------------------|-----------------------------
+//    info_name					   				| uint32 not null
+//    info_value									| INFO_SCHEMA
+//
+// INFO_SCHEMA is a dense union with members:
+//
+// 		Field Name (Type Code)			| Field Type
+//		----------------------------|-----------------------------
+//		string_value (0)						| utf8
+//		bool_value (1)							| bool
+//		int64_value (2)							| int64
+//		int32_bitmask (3)						| int32
+//		string_list (4)							| list<utf8>
+//		int32_to_int32_list_map (5)	| map<int32, list<int32>>
+//
+// Each metadatum is identified by an integer code. The recognized
+// codes are defined as constants. Codes [0, 10_000) are reserved
+// for ADBC usage. Drivers/vendors will ignore requests for unrecognized
+// codes (the row will be omitted from the result).
+func (c *cnxn) GetInfo(ctx context.Context, infoCodes []adbc.InfoCode) (array.RecordReader, error) {
+	const strValTypeID arrow.UnionTypeCode = 0
+
+	if len(infoCodes) == 0 {
+		infoCodes = maps.Keys(adbcToFlightSQLInfo)
+	}
+
+	bldr := array.NewRecordBuilder(c.cl.Alloc, adbc.GetInfoSchema)
+	defer bldr.Release()
+	bldr.Reserve(len(infoCodes))
+
+	infoNameBldr := bldr.Field(0).(*array.Uint32Builder)
+	infoValueBldr := bldr.Field(1).(*array.DenseUnionBuilder)
+	strInfoBldr := infoValueBldr.Child(0).(*array.StringBuilder)
+
+	translated := make([]flightsql.SqlInfo, 0, len(infoCodes))
+	for _, code := range infoCodes {
+		if t, ok := adbcToFlightSQLInfo[code]; ok {
+			translated = append(translated, t)
+			continue
+		}
+
+		switch code {
+		case adbc.InfoDriverName:
+			infoNameBldr.Append(uint32(code))
+			infoValueBldr.Append(strValTypeID)
+			strInfoBldr.Append(infoDriverName)
+		case adbc.InfoDriverVersion:
+			infoNameBldr.Append(uint32(code))
+			infoValueBldr.Append(strValTypeID)
+			strInfoBldr.Append(infoDriverVersion)
+		case adbc.InfoDriverArrowVersion:
+			infoNameBldr.Append(uint32(code))
+			infoValueBldr.Append(strValTypeID)
+			strInfoBldr.Append(infoDriverArrowVersion)
+		}
+	}
+
+	info, err := c.cl.GetSqlInfo(ctx, translated)
+	if err != nil {
+		return nil, adbcFromFlightStatus(err)
+	}
+
+	for _, endpoint := range info.Endpoint {
+		rdr, err := doGet(ctx, c.cl, endpoint, c.clientCache)
+		if err != nil {
+			return nil, adbcFromFlightStatus(err)
+		}
+
+		for rdr.Next() {
+			rec := rdr.Record()
+			field := rec.Column(0).(*array.Uint32)
+			info := rec.Column(1).(*array.DenseUnion)
+
+			for i := 0; i < int(rec.NumRows()); i++ {
+				switch flightsql.SqlInfo(field.Value(i)) {
+				case flightsql.SqlInfoFlightSqlServerName:
+					infoNameBldr.Append(uint32(adbc.InfoVendorName))
+				case flightsql.SqlInfoFlightSqlServerVersion:
+					infoNameBldr.Append(uint32(adbc.InfoVendorVersion))
+				case flightsql.SqlInfoFlightSqlServerArrowVersion:
+					infoNameBldr.Append(uint32(adbc.InfoVendorArrowVersion))
+				}
+
+				infoValueBldr.Append(info.TypeCode(i))
+				// we know we're only doing string fields here right now
+				v := info.Field(info.ChildID(i)).(*array.String).
+					Value(int(info.ValueOffset(i)))
+				strInfoBldr.Append(v)
+			}
+		}
+
+		if rdr.Err() != nil {
+			return nil, adbcFromFlightStatus(rdr.Err())
+		}
+	}
+
+	final := bldr.NewRecord()
+	defer final.Release()
+	return array.NewRecordReader(adbc.GetInfoSchema, []arrow.Record{final})
+}
+
+// GetObjects gets a hierarchical view of all catalogs, database schemas,
+// tables, and columns.
+//
+// The result is an Arrow Dataset with the following schema:
+//
+//		Field Name									| Field Type
+//		----------------------------|----------------------------
+//		catalog_name								| utf8
+//		catalog_db_schemas					| list<DB_SCHEMA_SCHEMA>
+//
+// DB_SCHEMA_SCHEMA is a Struct with the fields:
+//
+//		Field Name									| Field Type
+//		----------------------------|----------------------------
+//		db_schema_name							| utf8
+//		db_schema_tables						|	list<TABLE_SCHEMA>
+//
+// TABLE_SCHEMA is a Struct with the fields:
+//
+//		Field Name									| Field Type
+//		----------------------------|----------------------------
+//		table_name									| utf8 not null
+//		table_type									|	utf8 not null
+//		table_columns								| list<COLUMN_SCHEMA>
+//		table_constraints						| list<CONSTRAINT_SCHEMA>
+//
+// COLUMN_SCHEMA is a Struct with the fields:
+//
+//		Field Name 									| Field Type					| Comments
+//		----------------------------|---------------------|---------
+//		column_name									| utf8 not null				|
+//		ordinal_position						| int32								| (1)
+//		remarks											| utf8								| (2)
+//		xdbc_data_type							| int16								| (3)
+//		xdbc_type_name							| utf8								| (3)
+//		xdbc_column_size						| int32								| (3)
+//		xdbc_decimal_digits					| int16								| (3)
+//		xdbc_num_prec_radix					| int16								| (3)
+//		xdbc_nullable								| int16								| (3)
+//		xdbc_column_def							| utf8								| (3)
+//		xdbc_sql_data_type					| int16								| (3)
+//		xdbc_datetime_sub						| int16								| (3)
+//		xdbc_char_octet_length			| int32								| (3)
+//		xdbc_is_nullable						| utf8								| (3)
+//		xdbc_scope_catalog					| utf8								| (3)
+//		xdbc_scope_schema						| utf8								| (3)
+//		xdbc_scope_table						| utf8								| (3)
+//		xdbc_is_autoincrement				| bool								| (3)
+//		xdbc_is_generatedcolumn			| bool								| (3)
+//
+// 1. The column's ordinal position in the table (starting from 1).
+// 2. Database-specific description of the column.
+// 3. Optional Value. Should be null if not supported by the driver.
+//	  xdbc_values are meant to provide JDBC/ODBC-compatible metadata
+//		in an agnostic manner.
+//
+// CONSTRAINT_SCHEMA is a Struct with the fields:
+//
+//		Field Name									| Field Type					| Comments
+//		----------------------------|---------------------|---------
+//		constraint_name							| utf8								|
+//		constraint_type							| utf8 not null				| (1)
+//		constraint_column_names			| list<utf8> not null | (2)
+//		constraint_column_usage			| list<USAGE_SCHEMA>	| (3)
+//
+// 1. One of 'CHECK', 'FOREIGN KEY', 'PRIMARY KEY', or 'UNIQUE'.
+// 2. The columns on the current table that are constrained, in order.
+// 3. For FOREIGN KEY only, the referenced table and columns.
+//
+// USAGE_SCHEMA is a Struct with fields:
+//
+//		Field Name									|	Field Type
+//		----------------------------|----------------------------
+//		fk_catalog									| utf8
+//		fk_db_schema								| utf8
+//		fk_table										| utf8 not null
+//		fk_column_name							| utf8 not null
+//
+// For the parameters: If nil is passed, then that parameter will not
+// be filtered by at all. If an empty string, then only objects without
+// that property (ie: catalog or db schema) will be returned.
+//
+// tableName and columnName must be either nil (do not filter by
+// table name or column name) or non-empty.
+//
+// All non-empty, non-nil strings should be a search pattern (as described
+// earlier).
+func (c *cnxn) GetObjects(ctx context.Context, depth adbc.ObjectDepth, catalog *string, dbSchema *string, tableName *string, columnName *string, tableType []string) (array.RecordReader, error) {
+	panic("not implemented") // TODO: Implement
+}
+
+func (c *cnxn) GetTableSchema(ctx context.Context, catalog *string, dbSchema *string, tableName string) (*arrow.Schema, error) {
+	opts := &flightsql.GetTablesOpts{
+		Catalog:                catalog,
+		DbSchemaFilterPattern:  dbSchema,
+		TableNameFilterPattern: &tableName,
+		IncludeSchema:          true,
+	}
+
+	info, err := c.cl.GetTables(ctx, opts)
+	if err != nil {
+		return nil, adbcFromFlightStatus(err)
+	}
+
+	rdr, err := doGet(ctx, c.cl, info.Endpoint[0], c.clientCache)
+	if err != nil {
+		return nil, adbcFromFlightStatus(err)
+	}
+	defer rdr.Release()
+
+	rec, err := rdr.Read()
+	if err != nil {
+		if err == io.EOF {
+			return nil, adbc.Error{
+				Msg:  "No table found",
+				Code: adbc.StatusNotFound,
+			}
+		}
+		return nil, adbcFromFlightStatus(err)
+	}
+
+	// returned schema should be
+	//    0: catalog_name: utf8
+	//    1: db_schema_name: utf8
+	//    2: table_name: utf8 not null
+	//    3: table_type: utf8 not null
+	//    4: table_schema: bytes not null
+	schemaBytes := rec.Column(4).(*array.Binary).Value(0)
+	s, err := flight.DeserializeSchema(schemaBytes, c.db.alloc)
+	if err != nil {
+		return nil, adbcFromFlightStatus(err)
+	}
+	return s, nil
+}
+
+// GetTableTypes returns a list of the table types in the database.
+//
+// The result is an arrow dataset with the following schema:
+//
+//		Field Name			| Field Type
+//		----------------|--------------
+//		table_type			| utf8 not null
+//
+func (c *cnxn) GetTableTypes(ctx context.Context) (array.RecordReader, error) {
+	info, err := c.cl.GetTableTypes(ctx)
+	if err != nil {
+		return nil, adbcFromFlightStatus(err)
+	}
+
+	return newRecordReader(ctx, c.db.alloc, c.cl, info, c.clientCache)
+}
+
+// Commit commits any pending transactions on this connection, it should
+// only be used if autocommit is disabled.
+//
+// Behavior is undefined if this is mixed with SQL transaction statements.
+func (c *cnxn) Commit(_ context.Context) error {
+	return adbc.Error{
+		Msg:  "[Flight SQL] Transaction methods are not implemented yet",
+		Code: adbc.StatusNotImplemented}
+}
+
+// Rollback rolls back any pending transactions. Only used if autocommit
+// is disabled.
+//
+// Behavior is undefined if this is mixed with SQL transaction statements.
+func (c *cnxn) Rollback(_ context.Context) error {
+	return adbc.Error{
+		Msg:  "[Flight SQL] Transaction methods are not implemented yet",
+		Code: adbc.StatusNotImplemented}
+}
+
+// NewStatement initializes a new statement object tied to this connection
+func (c *cnxn) NewStatement() (adbc.Statement, error) {
+	return &statement{
+		alloc:       c.db.alloc,
+		cl:          c.cl,
+		clientCache: c.clientCache,
+	}, nil
+}
+
+// Close closes this connection and releases any associated resources.
+func (c *cnxn) Close() error {
+	if c.cl == nil {
+		return adbc.Error{
+			Msg:  "[Flight SQL Connection] trying to close already closed connection",
+			Code: adbc.StatusInvalidState,
+		}
+	}
+	err := c.cl.Close()
+	c.cl = nil
+	return err
+}
+
+// ReadPartition constructs a statement for a partition of a query. The
+// results can then be read independently using the returned RecordReader.
+//
+// A partition can be retrieved by using ExecutePartitions on a statement.
+func (c *cnxn) ReadPartition(ctx context.Context, serializedPartition []byte) (rdr array.RecordReader, err error) {
+	var endpoint flight.FlightEndpoint
+	if err := proto.Unmarshal(serializedPartition, &endpoint); err != nil {
+		return nil, adbc.Error{
+			Msg:  err.Error(),
+			Code: adbc.StatusInvalidArgument,
+		}
+	}
+
+	rdr, err = doGet(ctx, c.cl, &endpoint, c.clientCache)
+	if err != nil {
+		return nil, adbcFromFlightStatus(err)
+	}
+	return rdr, nil
+}
diff --git a/go/adbc/driver/flightsql/flightsql_adbc_test.go b/go/adbc/driver/flightsql/flightsql_adbc_test.go
new file mode 100644
index 0000000..e4cf5f3
--- /dev/null
+++ b/go/adbc/driver/flightsql/flightsql_adbc_test.go
@@ -0,0 +1,205 @@
+// 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 flightsql_test
+
+import (
+	"context"
+	"fmt"
+	"io"
+	"os"
+	"strings"
+	"testing"
+
+	"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/v10/arrow"
+	"github.com/apache/arrow/go/v10/arrow/array"
+	"github.com/apache/arrow/go/v10/arrow/flight"
+	"github.com/apache/arrow/go/v10/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v10/arrow/flight/flightsql/example"
+	"github.com/apache/arrow/go/v10/arrow/memory"
+	"github.com/stretchr/testify/require"
+	"github.com/stretchr/testify/suite"
+)
+
+type FlightSQLQuirks struct {
+	srv *example.SQLiteFlightSQLServer
+	s   flight.Server
+
+	done chan bool
+	mem  *memory.CheckedAllocator
+}
+
+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(nil)
+	s.srv, err = example.NewSQLiteFlightSQLServer()
+	require.NoError(t, err)
+	s.srv.Alloc = s.mem
+
+	s.s.RegisterFlightService(flightsql.NewFlightServer(s.srv))
+	require.NoError(t, s.s.Init("localhost:0"))
+	s.s.SetShutdownOnSignals(os.Interrupt, os.Kill)
+	go func() {
+		defer close(s.done)
+		_ = s.s.Serve()
+	}()
+
+	return driver.Driver{Alloc: s.mem}
+}
+
+func (s *FlightSQLQuirks) TearDownDriver(t *testing.T, _ adbc.Driver) {
+	s.s.Shutdown()
+	<-s.done
+	s.srv = nil
+	s.mem.AssertSize(t, 0)
+}
+
+func (s *FlightSQLQuirks) DatabaseOptions() map[string]string {
+	return map[string]string{
+		adbc.OptionKeyURI:        "grpc+tcp://" + s.s.Addr().String(),
+		driver.OptionSSLInsecure: adbc.OptionValueEnabled,
+	}
+}
+
+func (s *FlightSQLQuirks) getSqlTypeFromArrowType(dt arrow.DataType) string {
+	switch dt.ID() {
+	case arrow.INT8, arrow.INT16, arrow.INT32, arrow.INT64:
+		return "integer"
+	case arrow.FLOAT32:
+		return "float"
+	case arrow.FLOAT64:
+		return "double"
+	case arrow.STRING:
+		return "text"
+	default:
+		return ""
+	}
+}
+
+type srvQuery string
+
+func (s srvQuery) GetQuery() string { return string(s) }
+
+func writeTo(arr arrow.Array, idx int, w io.Writer) {
+	switch arr := arr.(type) {
+	case *array.Int8:
+		fmt.Fprint(w, arr.Value(idx))
+	case *array.Uint8:
+		fmt.Fprint(w, arr.Value(idx))
+	case *array.Int16:
+		fmt.Fprint(w, arr.Value(idx))
+	case *array.Uint16:
+		fmt.Fprint(w, arr.Value(idx))
+	case *array.Int32:
+		fmt.Fprint(w, arr.Value(idx))
+	case *array.Uint32:
+		fmt.Fprint(w, arr.Value(idx))
+	case *array.Int64:
+		fmt.Fprint(w, arr.Value(idx))
+	case *array.Uint64:
+		fmt.Fprint(w, arr.Value(idx))
+	case *array.Float32:
+		fmt.Fprint(w, arr.Value(idx))
+	case *array.Float64:
+		fmt.Fprint(w, arr.Value(idx))
+	case *array.String:
+		fmt.Fprintf(w, `"%s"`, arr.Value(idx))
+	}
+}
+
+func (s *FlightSQLQuirks) CreateSampleTable(tableName string, r arrow.Record) error {
+	var b strings.Builder
+	b.WriteString("CREATE TABLE ")
+	b.WriteString(tableName)
+	b.WriteString(" (")
+	for i := 0; i < int(r.NumCols()); i++ {
+		if i != 0 {
+			b.WriteString(", ")
+		}
+		f := r.Schema().Field(i)
+		b.WriteString(f.Name)
+		b.WriteByte(' ')
+		b.WriteString(s.getSqlTypeFromArrowType(f.Type))
+	}
+	b.WriteString(")")
+
+	_, err := s.srv.DoPutCommandStatementUpdate(context.Background(), srvQuery(b.String()))
+	if err != nil {
+		return err
+	}
+
+	insertQueryPrefix := "INSERT INTO " + tableName + " VALUES ("
+	for i := 0; i < int(r.NumRows()); i++ {
+		b.Reset()
+		b.WriteString(insertQueryPrefix)
+
+		for j := 0; j < int(r.NumCols()); j++ {
+			if j != 0 {
+				b.WriteString(", ")
+			}
+
+			col := r.Column(j)
+			writeTo(col, j, &b)
+		}
+
+		b.WriteString(")")
+		_, err := s.srv.DoPutCommandStatementUpdate(context.Background(), srvQuery(b.String()))
+		if err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+func (s *FlightSQLQuirks) BindParameter(_ int) string         { return "?" }
+func (s *FlightSQLQuirks) SupportsConcurrentStatements() bool { return true }
+func (s *FlightSQLQuirks) SupportsPartitionedData() bool      { return true }
+func (s *FlightSQLQuirks) SupportsTransactions() bool         { return false }
+func (s *FlightSQLQuirks) SupportsGetParameterSchema() bool   { return false }
+func (s *FlightSQLQuirks) GetMetadata(code adbc.InfoCode) interface{} {
+	switch code {
+	case adbc.InfoDriverName:
+		return "ADBC Flight SQL Driver - Go"
+	// runtime/debug.ReadBuildInfo doesn't currently work for tests
+	// github.com/golang/go/issues/33976
+	case adbc.InfoDriverVersion:
+		return ""
+	case adbc.InfoDriverArrowVersion:
+		return ""
+	case adbc.InfoVendorName:
+		return "db_name"
+	case adbc.InfoVendorVersion:
+		return "sqlite 3"
+	case adbc.InfoVendorArrowVersion:
+		return "10.0.1"
+	}
+
+	return nil
+}
+
+func TestADBCFlightSQL(t *testing.T) {
+	q := &FlightSQLQuirks{}
+	suite.Run(t, &validation.DatabaseTests{Quirks: q})
+	suite.Run(t, &validation.ConnectionTests{Quirks: q})
+	suite.Run(t, &validation.StatementTests{Quirks: q})
+}
diff --git a/go/adbc/driver/flightsql/flightsql_statement.go b/go/adbc/driver/flightsql/flightsql_statement.go
new file mode 100644
index 0000000..04f2904
--- /dev/null
+++ b/go/adbc/driver/flightsql/flightsql_statement.go
@@ -0,0 +1,290 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package flightsql
+
+import (
+	"context"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v10/arrow"
+	"github.com/apache/arrow/go/v10/arrow/array"
+	"github.com/apache/arrow/go/v10/arrow/flight"
+	"github.com/apache/arrow/go/v10/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v10/arrow/memory"
+	"github.com/bluele/gcache"
+	"google.golang.org/protobuf/proto"
+)
+
+type statement struct {
+	alloc       memory.Allocator
+	cl          *flightsql.Client
+	clientCache gcache.Cache
+
+	query    string
+	prepared *flightsql.PreparedStatement
+}
+
+func (s *statement) closePreparedStatement() error {
+	return s.prepared.Close(context.Background())
+}
+
+// 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 (s *statement) Close() (err error) {
+	if s.prepared != nil {
+		err = s.closePreparedStatement()
+		s.prepared = nil
+	}
+
+	if s.cl == nil {
+		return adbc.Error{
+			Msg:  "[Flight SQL Statement] cannot close already closed statement",
+			Code: adbc.StatusInvalidState,
+		}
+	}
+
+	s.cl = nil
+	s.clientCache = nil
+
+	return err
+}
+
+// SetOption sets a string option on this statement
+func (s *statement) SetOption(key string, val string) error {
+	return adbc.Error{
+		Msg:  "[FlightSQL Statement] SetOption not implemented",
+		Code: adbc.StatusNotImplemented,
+	}
+}
+
+// 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 (s *statement) SetSqlQuery(query string) error {
+	if s.prepared != nil {
+		if err := s.closePreparedStatement(); err != nil {
+			return err
+		}
+		s.prepared = nil
+	}
+
+	s.query = query
+	return nil
+}
+
+// ExecuteQuery executes the current query or prepared statement
+// and returnes a RecordReader for the results along with the number
+// of rows affected if known, otherwise it will be -1.
+//
+// This invalidates any prior result sets on this statement.
+func (s *statement) ExecuteQuery(ctx context.Context) (rdr array.RecordReader, nrec int64, err error) {
+	var info *flight.FlightInfo
+	if s.prepared != nil {
+		info, err = s.prepared.Execute(ctx)
+	} else if s.query != "" {
+		info, err = s.cl.Execute(ctx, s.query)
+	} else {
+		return nil, -1, adbc.Error{
+			Msg:  "[Flight SQL Statement] cannot call ExecuteQuery without a query or prepared statement",
+			Code: adbc.StatusInvalidState,
+		}
+	}
+
+	if err != nil {
+		return nil, -1, adbcFromFlightStatus(err)
+	}
+
+	nrec = info.TotalRecords
+	rdr, err = newRecordReader(ctx, s.alloc, s.cl, info, s.clientCache)
+	return
+}
+
+// ExecuteUpdate executes a statement that does not generate a result
+// set. It returns the number of rows affected if known, otherwise -1.
+func (s *statement) ExecuteUpdate(ctx context.Context) (int64, error) {
+	if s.prepared != nil {
+		return s.prepared.ExecuteUpdate(ctx)
+	}
+
+	if s.query != "" {
+		return s.cl.ExecuteUpdate(ctx, s.query)
+	}
+
+	return -1, adbc.Error{
+		Msg:  "[Flight SQL Statement] cannot call ExecuteUpdate without a query or prepared statement",
+		Code: adbc.StatusInvalidState,
+	}
+}
+
+// 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 {
+	if s.query == "" {
+		return adbc.Error{
+			Msg:  "[FlightSQL Statement] must call SetSqlQuery before Prepare",
+			Code: adbc.StatusInvalidArgument,
+		}
+	}
+
+	prep, err := s.cl.Prepare(ctx, s.alloc, s.query)
+	if err != nil {
+		return adbcFromFlightStatus(err)
+	}
+	s.prepared = prep
+	return nil
+}
+
+// SetSubstraitPlan allows setting a serialized Substrait execution
+// plan into the query or for querying Substrait-related metadata.
+//
+// Drivers are not required to support both SQL and Substrait semantics.
+// If they do, it may be via converting between representations internally.
+//
+// Like SetSqlQuery, after this is called the query can be executed
+// 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,
+	}
+}
+
+// Bind uses an arrow record batch to bind parameters to the query.
+//
+// This can be used for bulk inserts or for prepared statements.
+// The driver will call release on the passed in Record when it is done,
+// but it may not do this until the statement is closed or another
+// record is bound.
+func (s *statement) Bind(_ context.Context, values arrow.Record) error {
+	// TODO: handle bulk insert situation
+
+	if s.prepared == nil {
+		return adbc.Error{
+			Msg:  "[Flight SQL Statement] must call Prepare before calling Bind",
+			Code: adbc.StatusInvalidState}
+	}
+
+	s.prepared.SetParameters(values)
+	return nil
+}
+
+// BindStream uses a record batch stream to bind parameters for this
+// query. This can be used for bulk inserts or prepared statements.
+//
+// The driver will call Release on the record reader, but may not do this
+// until Close is called.
+func (s *statement) BindStream(ctx context.Context, stream array.RecordReader) error {
+	return adbc.Error{
+		Msg:  "[Flight SQL Statement] BindStream not yet implemented",
+		Code: adbc.StatusNotImplemented,
+	}
+}
+
+// GetParameterSchema returns an Arrow schema representation of
+// the expected parameters to be bound.
+//
+// This retrieves an Arrow Schema describing the number, names, and
+// types of the parameters in a parameterized statement. The fields
+// of the schema should be in order of the ordinal position of the
+// parameters; named parameters should appear only once.
+//
+// If the parameter does not have a name, or a name cannot be determined,
+// the name of the corresponding field in the schema will be an empty
+// string. If the type cannot be determined, the type of the corresponding
+// field will be NA (NullType).
+//
+// This should be called only after calling Prepare.
+//
+// This should return an error with StatusNotImplemented if the schema
+// cannot be determined.
+func (s *statement) GetParameterSchema() (*arrow.Schema, error) {
+	if s.prepared == nil {
+		return nil, adbc.Error{
+			Msg:  "[Flight SQL Statement] must call Prepare before GetParameterSchema",
+			Code: adbc.StatusInvalidState,
+		}
+	}
+
+	ret := s.prepared.ParameterSchema()
+	if ret == nil {
+		return nil, adbc.Error{Code: adbc.StatusNotImplemented}
+	}
+
+	return ret, nil
+}
+
+// ExecutePartitions executes the current statement and gets the results
+// as a partitioned result set.
+//
+// It returns the Schema of the result set (if available, nil otherwise),
+// the collection of partition descriptors and the number of rows affected,
+// if known. If unknown, the number of rows affected will be -1.
+//
+// If the driver does not support partitioned results, this will return
+// an error with a StatusNotImplemented code.
+func (s *statement) ExecutePartitions(ctx context.Context) (*arrow.Schema, adbc.Partitions, int64, error) {
+	var (
+		info *flight.FlightInfo
+		out  adbc.Partitions
+		sc   *arrow.Schema
+		err  error
+	)
+
+	if s.prepared != nil {
+		info, err = s.prepared.Execute(ctx)
+	} else if s.query != "" {
+		info, err = s.cl.Execute(ctx, s.query)
+	} else {
+		return nil, out, -1, adbc.Error{
+			Msg:  "[Flight SQL Statement] cannot call ExecuteQuery without a query or prepared statement",
+			Code: adbc.StatusInvalidState,
+		}
+	}
+
+	if err != nil {
+		return nil, out, -1, adbcFromFlightStatus(err)
+	}
+
+	if len(info.Schema) > 0 {
+		sc, err = flight.DeserializeSchema(info.Schema, s.alloc)
+		if err != nil {
+			return nil, out, -1, adbcFromFlightStatus(err)
+		}
+	}
+
+	out.NumPartitions = uint64(len(info.Endpoint))
+	out.PartitionIDs = make([][]byte, out.NumPartitions)
+	for i, e := range info.Endpoint {
+		data, err := proto.Marshal(e)
+		if err != nil {
+			return sc, out, -1, adbc.Error{
+				Msg:  err.Error(),
+				Code: adbc.StatusInternal,
+			}
+		}
+
+		out.PartitionIDs[i] = data
+	}
+
+	return sc, out, info.TotalRecords, nil
+}
diff --git a/go/adbc/driver/flightsql/record_reader.go b/go/adbc/driver/flightsql/record_reader.go
new file mode 100644
index 0000000..09b82cf
--- /dev/null
+++ b/go/adbc/driver/flightsql/record_reader.go
@@ -0,0 +1,154 @@
+// 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 flightsql
+
+import (
+	"context"
+	"sync"
+	"sync/atomic"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v10/arrow"
+	"github.com/apache/arrow/go/v10/arrow/array"
+	"github.com/apache/arrow/go/v10/arrow/flight"
+	"github.com/apache/arrow/go/v10/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v10/arrow/memory"
+	"github.com/bluele/gcache"
+)
+
+type reader struct {
+	refCount int64
+	schema   *arrow.Schema
+	ch       chan arrow.Record
+	rec      arrow.Record
+
+	cancelFn context.CancelFunc
+}
+
+// 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) (rdr array.RecordReader, err error) {
+	var cancelFn context.CancelFunc
+	ctx, cancelFn = context.WithCancel(ctx)
+	ch := make(chan arrow.Record, 5)
+
+	var wg sync.WaitGroup
+	defer func() {
+		if err != nil {
+			cancelFn()
+			for rec := range ch {
+				rec.Release()
+			}
+		}
+	}()
+
+	wg.Add(len(info.Endpoint))
+
+	go func() {
+		wg.Wait()
+		close(ch)
+	}()
+
+	endpoints := info.Endpoint
+
+	var schema *arrow.Schema
+	if info.Schema != nil {
+		schema, err = flight.DeserializeSchema(info.Schema, alloc)
+		if err != nil {
+			return nil, adbc.Error{
+				Msg:  err.Error(),
+				Code: adbc.StatusInvalidState}
+		}
+	} else {
+		rdr, err := doGet(ctx, cl, endpoints[0], clCache)
+		if err != nil {
+			return nil, adbcFromFlightStatus(err)
+		}
+		schema = rdr.Schema()
+		go func() {
+			defer wg.Done()
+			defer rdr.Release()
+			for rdr.Next() && ctx.Err() == nil {
+				rec := rdr.Record()
+				rec.Retain()
+				ch <- rec
+			}
+		}()
+
+		endpoints = endpoints[1:]
+	}
+
+	for _, ep := range endpoints {
+		go func(endpoint *flight.FlightEndpoint) {
+			defer wg.Done()
+
+			rdr, err := doGet(ctx, cl, endpoint, clCache)
+			if err != nil {
+				return
+			}
+			defer rdr.Release()
+
+			for rdr.Next() && ctx.Err() == nil {
+				rec := rdr.Record()
+				rec.Retain()
+				ch <- rec
+			}
+		}(ep)
+	}
+
+	return &reader{
+		refCount: 1,
+		ch:       ch,
+		cancelFn: cancelFn,
+		schema:   schema,
+	}, nil
+}
+
+func (r *reader) Retain() {
+	atomic.AddInt64(&r.refCount, 1)
+}
+
+func (r *reader) Release() {
+	if atomic.AddInt64(&r.refCount, -1) == 0 {
+		if r.rec != nil {
+			r.rec.Release()
+		}
+		r.cancelFn()
+		for rec := range r.ch {
+			rec.Release()
+		}
+	}
+}
+
+func (r *reader) Next() bool {
+	if r.rec != nil {
+		r.rec.Release()
+		r.rec = nil
+	}
+
+	r.rec = <-r.ch
+	return r.rec != nil
+}
+
+func (r *reader) Schema() *arrow.Schema {
+	return r.schema
+}
+
+func (r *reader) Record() arrow.Record {
+	return r.rec
+}
diff --git a/go/adbc/driver/flightsql/utils.go b/go/adbc/driver/flightsql/utils.go
new file mode 100644
index 0000000..d855fe5
--- /dev/null
+++ b/go/adbc/driver/flightsql/utils.go
@@ -0,0 +1,57 @@
+// 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 flightsql
+
+import (
+	"github.com/apache/arrow-adbc/go/adbc"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+func adbcFromFlightStatus(err error) error {
+	var adbcCode adbc.Status
+	switch status.Code(err) {
+	case codes.OK:
+		return nil
+	case codes.Canceled:
+		adbcCode = adbc.StatusCancelled
+	case codes.Unknown:
+		adbcCode = adbc.StatusUnknown
+	case codes.Internal:
+		adbcCode = adbc.StatusInternal
+	case codes.InvalidArgument:
+		adbcCode = adbc.StatusInvalidArgument
+	case codes.AlreadyExists:
+		adbcCode = adbc.StatusAlreadyExists
+	case codes.NotFound:
+		adbcCode = adbc.StatusNotFound
+	case codes.Unauthenticated:
+		adbcCode = adbc.StatusUnauthenticated
+	case codes.Unavailable:
+		adbcCode = adbc.StatusIO
+	case codes.PermissionDenied:
+		adbcCode = adbc.StatusUnauthorized
+	default:
+		adbcCode = adbc.StatusUnknown
+	}
+
+	return adbc.Error{
+		Msg:  err.Error(),
+		Code: adbcCode,
+	}
+}
diff --git a/go/adbc/drivermgr/wrapper.go b/go/adbc/drivermgr/wrapper.go
index 3b60696..a220291 100644
--- a/go/adbc/drivermgr/wrapper.go
+++ b/go/adbc/drivermgr/wrapper.go
@@ -22,6 +22,9 @@ package drivermgr
 // #include <stdlib.h>
 //
 // void releaseErr(struct AdbcError* err) { err->release(err); }
+// struct ArrowArray* allocArr() {
+//     return (struct ArrowArray*)malloc(sizeof(struct ArrowArray));
+// }
 //
 import "C"
 import (
@@ -39,7 +42,73 @@ type option struct {
 	key, val *C.char
 }
 
-type Driver struct {
+func convOptions(incoming map[string]string, existing map[string]option) {
+	for k, v := range incoming {
+		o, ok := existing[k]
+		if !ok {
+			o.key = C.CString(k)
+			o.val = C.CString(v)
+			existing[k] = o
+			continue
+		}
+
+		C.free(unsafe.Pointer(o.val))
+		o.val = C.CString(v)
+		existing[k] = o
+	}
+}
+
+type Driver struct{}
+
+func (d Driver) NewDatabase(opts map[string]string) (adbc.Database, error) {
+	dbOptions := make(map[string]option)
+	convOptions(opts, dbOptions)
+
+	db := &Database{
+		options: make(map[string]option),
+	}
+
+	var err C.struct_AdbcError
+	db.db = (*C.struct_AdbcDatabase)(unsafe.Pointer(C.malloc(C.sizeof_struct_AdbcDatabase)))
+	if code := adbc.Status(C.AdbcDatabaseNew(db.db, &err)); code != adbc.StatusOK {
+		return nil, toAdbcError(code, &err)
+	}
+
+	for _, o := range dbOptions {
+		if code := adbc.Status(C.AdbcDatabaseSetOption(db.db, o.key, o.val, &err)); code != adbc.StatusOK {
+			errOut := toAdbcError(code, &err)
+			C.AdbcDatabaseRelease(db.db, &err)
+			db.db = nil
+			return nil, errOut
+		}
+	}
+
+	if code := adbc.Status(C.AdbcDatabaseInit(db.db, &err)); code != adbc.StatusOK {
+		errOut := toAdbcError(code, &err)
+		C.AdbcDatabaseRelease(db.db, &err)
+		db.db = nil
+		return nil, errOut
+	}
+
+	runtime.SetFinalizer(db, func(db *Database) {
+		if db.db != nil {
+			var err C.struct_AdbcError
+			code := adbc.Status(C.AdbcDatabaseRelease(db.db, &err))
+			if code != adbc.StatusOK {
+				panic(toAdbcError(code, &err))
+			}
+		}
+
+		for _, o := range db.options {
+			C.free(unsafe.Pointer(o.key))
+			C.free(unsafe.Pointer(o.val))
+		}
+	})
+
+	return db, nil
+}
+
+type Database struct {
 	options map[string]option
 	db      *C.struct_AdbcDatabase
 }
@@ -57,7 +126,7 @@ func toAdbcError(code adbc.Status, e *C.struct_AdbcError) error {
 	return err
 }
 
-func (d *Driver) SetOptions(options map[string]string) error {
+func (d *Database) SetOptions(options map[string]string) error {
 	if d.options == nil {
 		d.options = make(map[string]option)
 	}
@@ -78,44 +147,20 @@ func (d *Driver) SetOptions(options map[string]string) error {
 	return nil
 }
 
-func (d *Driver) Open(context.Context) (adbc.Connection, error) {
+func (d *Database) Open(context.Context) (adbc.Connection, error) {
 	var err C.struct_AdbcError
-	if d.db == nil {
-		d.db = (*C.struct_AdbcDatabase)(unsafe.Pointer(C.malloc(C.sizeof_struct_AdbcDatabase)))
-		if code := adbc.Status(C.AdbcDatabaseNew(d.db, &err)); code != adbc.StatusOK {
-			return nil, toAdbcError(code, &err)
-		}
 
-		for _, o := range d.options {
-			if code := adbc.Status(C.AdbcDatabaseSetOption(d.db, o.key, o.val, &err)); code != adbc.StatusOK {
-				errOut := toAdbcError(code, &err)
-				C.AdbcDatabaseRelease(d.db, &err)
-				d.db = nil
-				return nil, errOut
-			}
-		}
+	var c C.struct_AdbcConnection
+	if code := adbc.Status(C.AdbcConnectionNew(&c, &err)); code != adbc.StatusOK {
+		return nil, toAdbcError(code, &err)
+	}
 
-		if code := adbc.Status(C.AdbcDatabaseInit(d.db, &err)); code != adbc.StatusOK {
+	for _, o := range d.options {
+		if code := adbc.Status(C.AdbcConnectionSetOption(&c, o.key, o.val, &err)); code != adbc.StatusOK {
 			errOut := toAdbcError(code, &err)
-			C.AdbcDatabaseRelease(d.db, &err)
-			d.db = nil
+			C.AdbcConnectionRelease(&c, &err)
 			return nil, errOut
 		}
-
-		runtime.SetFinalizer(d, func(drv *Driver) {
-			if drv.db != nil {
-				var err C.struct_AdbcError
-				code := adbc.Status(C.AdbcDatabaseRelease(drv.db, &err))
-				if code != adbc.StatusOK {
-					panic(toAdbcError(code, &err))
-				}
-			}
-		})
-	}
-
-	var c C.struct_AdbcConnection
-	if code := adbc.Status(C.AdbcConnectionNew(&c, &err)); code != adbc.StatusOK {
-		return nil, toAdbcError(code, &err)
 	}
 
 	if code := adbc.Status(C.AdbcConnectionInit(&c, d.db, &err)); code != adbc.StatusOK {
@@ -280,13 +325,22 @@ func (s *stmt) SetSubstraitPlan(plan []byte) error {
 
 func (s *stmt) Bind(_ context.Context, values arrow.Record) error {
 	var (
-		arr    cdata.CArrowArray
-		schema cdata.CArrowSchema
-		err    C.struct_AdbcError
+		arr    = C.allocArr()
+		schema C.struct_ArrowSchema
+
+		cdArr    = (*cdata.CArrowArray)(unsafe.Pointer(arr))
+		cdSchema = (*cdata.CArrowSchema)(unsafe.Pointer(&schema))
+		err      C.struct_AdbcError
 	)
-	cdata.ExportArrowRecordBatch(values, &arr, &schema)
+	cdata.ExportArrowRecordBatch(values, cdArr, cdSchema)
+	defer func() {
+		cdata.ReleaseCArrowArray(cdArr)
+		cdata.ReleaseCArrowSchema(cdSchema)
+
+		C.free(unsafe.Pointer(arr))
+	}()
 
-	code := adbc.Status(C.AdbcStatementBind(s.st, (*C.struct_ArrowArray)(unsafe.Pointer(&arr)), (*C.struct_ArrowSchema)(unsafe.Pointer(&schema)), &err))
+	code := adbc.Status(C.AdbcStatementBind(s.st, arr, &schema, &err))
 	if code != adbc.StatusOK {
 		return toAdbcError(code, &err)
 	}
diff --git a/go/adbc/drivermgr/wrapper_sqlite_test.go b/go/adbc/drivermgr/wrapper_sqlite_test.go
index 8ec3af0..929b1d4 100644
--- a/go/adbc/drivermgr/wrapper_sqlite_test.go
+++ b/go/adbc/drivermgr/wrapper_sqlite_test.go
@@ -40,19 +40,22 @@ type DriverMgrSuite struct {
 
 	ctx  context.Context
 	drv  adbc.Driver
+	db   adbc.Database
 	conn adbc.Connection
 }
 
 func (dm *DriverMgrSuite) SetupSuite() {
 	dm.ctx = context.TODO()
-	dm.drv = &drivermgr.Driver{}
-	dm.NoError(dm.drv.SetOptions(map[string]string{
+	dm.drv = drivermgr.Driver{}
+	var err error
+	dm.db, err = dm.drv.NewDatabase(map[string]string{
 		"driver": "adbc_driver_sqlite",
-	}))
+	})
+	dm.NoError(err)
 }
 
 func (dm *DriverMgrSuite) SetupTest() {
-	cnxn, err := dm.drv.Open(dm.ctx)
+	cnxn, err := dm.db.Open(dm.ctx)
 	dm.Require().NoError(err)
 	dm.conn = cnxn
 }
@@ -197,22 +200,22 @@ func TestDriverMgr(t *testing.T) {
 func TestDriverMgrCustomInitFunc(t *testing.T) {
 	// explicitly set entrypoint
 	var drv drivermgr.Driver
-	assert.NoError(t, drv.SetOptions(map[string]string{
+	db, err := drv.NewDatabase(map[string]string{
 		"driver":     "adbc_driver_sqlite",
 		"entrypoint": "AdbcDriverInit",
-	}))
-	cnxn, err := drv.Open(context.Background())
+	})
+	assert.NoError(t, err)
+	cnxn, err := db.Open(context.Background())
 	assert.NoError(t, err)
 	require.NoError(t, cnxn.Close())
 
 	// set invalid entrypoint
 	drv = drivermgr.Driver{}
-	assert.NoError(t, drv.SetOptions(map[string]string{
+	db, err = drv.NewDatabase(map[string]string{
 		"driver":     "adbc_driver_sqlite",
 		"entrypoint": "ThisSymbolDoesNotExist",
-	}))
-	cnxn, err = drv.Open(context.Background())
-	assert.Nil(t, cnxn)
+	})
+	assert.Nil(t, db)
 	var exp *adbc.Error
 	assert.ErrorAs(t, err, &exp)
 	assert.Equal(t, adbc.StatusInternal, exp.Code)
diff --git a/go/adbc/go.mod b/go/adbc/go.mod
index e045401..b186883 100644
--- a/go/adbc/go.mod
+++ b/go/adbc/go.mod
@@ -20,9 +20,13 @@ module github.com/apache/arrow-adbc/go/adbc
 go 1.18
 
 require (
-	github.com/apache/arrow/go/v10 v10.0.0-20220907151818-ff3aa3b7bb31
+	github.com/apache/arrow/go/v10 v10.0.1
+	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/tools v0.1.12
+	google.golang.org/grpc v1.49.0
+	google.golang.org/protobuf v1.28.1
 )
 
 require (
@@ -30,18 +34,37 @@ require (
 	github.com/apache/thrift v0.16.0 // indirect
 	github.com/davecgh/go-spew v1.1.1 // indirect
 	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/uuid v1.3.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.9 // indirect
 	github.com/klauspost/cpuid/v2 v2.0.9 // indirect
+	github.com/kr/text v0.2.0 // indirect
+	github.com/mattn/go-isatty v0.0.16 // indirect
 	github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect
 	github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect
 	github.com/pierrec/lz4/v4 v4.1.15 // indirect
 	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
+	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
+	modernc.org/cc/v3 v3.36.3 // indirect
+	modernc.org/ccgo/v3 v3.16.9 // indirect
+	modernc.org/libc v1.17.1 // indirect
+	modernc.org/mathutil v1.5.0 // indirect
+	modernc.org/memory v1.2.1 // indirect
+	modernc.org/opt v0.1.3 // indirect
+	modernc.org/sqlite v1.18.1 // indirect
+	modernc.org/strutil v1.1.3 // indirect
+	modernc.org/token v1.0.0 // indirect
 )
diff --git a/go/adbc/go.sum b/go/adbc/go.sum
index 4c4003b..85ebb30 100644
--- a/go/adbc/go.sum
+++ b/go/adbc/go.sum
@@ -1,257 +1,136 @@
 cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
-cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
-dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
-gioui.org v0.0.0-20210308172011-57750fc8a0a6/go.mod h1:RSH6KIUZ0p2xy5zHDxgAM4zumjgTw83q2ge/PI+yyw8=
-git.sr.ht/~sbinet/gg v0.3.1/go.mod h1:KGYtlADtqsqANL9ueOFkWymvzUvLMQllU5Ixo+8v3pc=
 github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
-github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo=
 github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU=
-github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk=
-github.com/ajstarks/deck v0.0.0-20200831202436-30c9fc6549a9/go.mod h1:JynElWSGnm/4RlzPXRlREEwqTHAN3T56Bv2ITsFT3gY=
-github.com/ajstarks/deck/generate v0.0.0-20210309230005-c3f852c02e19/go.mod h1:T13YZdzov6OU0A1+RfKZiZN9ca6VeKdBdyDV+BY97Tk=
-github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw=
-github.com/ajstarks/svgo v0.0.0-20211024235047-1546f124cd8b/go.mod h1:1KcenG0jGWcpt8ov532z81sp/kMMUG485J2InIOyADM=
 github.com/andybalholm/brotli v1.0.4 h1:V7DdXeJtZscaqfNuAdSRuRFzuiKlHSC/Zh3zl9qY3JY=
 github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig=
-github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY=
-github.com/apache/arrow/go/v10 v10.0.0-20220830153009-74dae618ed8d h1:c8cgjN3/nnAyOvIguA/V7yutV9B1AnnvDK6lmk3NuLY=
-github.com/apache/arrow/go/v10 v10.0.0-20220830153009-74dae618ed8d/go.mod h1:CjQdSNiJ0z9Mk+SQdLic1TTBuvUnVEqDluS7mOvUv0E=
-github.com/apache/arrow/go/v10 v10.0.0-20220907151818-ff3aa3b7bb31 h1:a4X4QOo6Q08PyUdQpV90jffGYNsC225avLc5gg9hEHI=
-github.com/apache/arrow/go/v10 v10.0.0-20220907151818-ff3aa3b7bb31/go.mod h1:AT/kJeoralrEDorj059JQcKfDnwWEeO6KcVMzGhwpmQ=
+github.com/apache/arrow/go/v10 v10.0.1 h1:n9dERvixoC/1JjDmBcs9FPaEryoANa2sCgVFo6ez9cI=
+github.com/apache/arrow/go/v10 v10.0.1/go.mod h1:YvhnlEePVnBS4+0z3fhPfUy7W1Ikj0Ih0vcRo/gZ1M0=
 github.com/apache/thrift v0.16.0 h1:qEy6UW60iVOlUy+b9ZR0d5WzUWYGOo4HfopoyBaNmoY=
 github.com/apache/thrift v0.16.0/go.mod h1:PHK3hniurgQaNMZYaCLEqXKsYK8upmhPbmdP2FXSqgU=
-github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8=
-github.com/boombuler/barcode v1.0.1/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8=
+github.com/bluele/gcache v0.0.2 h1:WcbfdXICg7G/DGBh1PFfcirkWOQV+v077yF1pSy3DGw=
+github.com/bluele/gcache v0.0.2/go.mod h1:m15KV+ECjptwSPxKhOhQoAFQVtUFjTVkc3H8o0t/fp0=
 github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
-github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
 github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
-github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc=
-github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk=
-github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI=
-github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs=
-github.com/cncf/xds/go v0.0.0-20211001041855-01bcc9b48dfe/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs=
-github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs=
 github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
 github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
 github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
 github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
-github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE=
+github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo=
 github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
-github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
 github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
-github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98=
-github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk=
-github.com/envoyproxy/go-control-plane v0.10.2-0.20220325020618-49ff273808a1/go.mod h1:KJwIaB5Mv44NWtYuAOFCVOjcI94vtpEz2JU/D2v6IjE=
 github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
-github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k=
-github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k=
-github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
-github.com/go-fonts/dejavu v0.1.0/go.mod h1:4Wt4I4OU2Nq9asgDCteaAaWZOV24E+0/Pwo0gppep4g=
-github.com/go-fonts/latin-modern v0.2.0/go.mod h1:rQVLdDMK+mK1xscDwsqM5J8U2jrRa3T0ecnM9pNujks=
-github.com/go-fonts/liberation v0.1.1/go.mod h1:K6qoJYypsmfVjWg8KOVDQhLc8UDgIK2HYqyqAO9z7GY=
-github.com/go-fonts/liberation v0.2.0/go.mod h1:K6qoJYypsmfVjWg8KOVDQhLc8UDgIK2HYqyqAO9z7GY=
-github.com/go-fonts/stix v0.1.0/go.mod h1:w/c1f0ldAUlJmLBvlbkvVXLAD+tAMqobIIQpmnUIzUY=
-github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
-github.com/go-latex/latex v0.0.0-20210118124228-b3d85cf34e07/go.mod h1:CO1AlKB2CSIqUrmQPqA0gdRIlnLEY0gK5JGjh37zN5U=
-github.com/go-latex/latex v0.0.0-20210823091927-c0d11ff05a81/go.mod h1:SX0U8uGpxhq9o2S/CELCSUxEWWAuoCUcVCQWv7G2OCk=
-github.com/go-pdf/fpdf v0.5.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M=
-github.com/go-pdf/fpdf v0.6.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M=
-github.com/goccy/go-json v0.9.10/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I=
 github.com/goccy/go-json v0.9.11 h1:/pAaQDLHEoCq/5FFmSKBswWmK6H0e8g4159Kc/X/nqk=
 github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I=
-github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k=
 github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
 github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
 github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8=
 github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
 github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
-github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw=
 github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
 github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
 github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
 github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
 github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
 github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8=
-github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
-github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
 github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
+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.6+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8=
 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/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=
 github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
-github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
 github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
 github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
-github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
-github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE=
-github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY=
-github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
+github.com/google/go-cmp v0.5.8 h1:e6P7q2lk1O+qJJb4BtCQXlK8vWEO8V1ZeuEdJNOqZyg=
+github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I=
 github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw=
-github.com/jung-kurt/gofpdf v1.0.0/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes=
-github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes=
+github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 h1:Z9n2FFNUXsshfwJMBgNA0RU6/i7WVaAegv3PtuIHPMs=
 github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51/go.mod h1:CzGEWj7cYgsdH8dAjBGEr58BoE7ScuLd+fwFZ44+/x8=
-github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
 github.com/klauspost/asmfmt v1.3.2 h1:4Ri7ox3EwapiOjCki+hw14RyKk201CN4rzyCJRFLpK4=
 github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE=
 github.com/klauspost/compress v1.15.9 h1:wKRjX6JRtDdrE9qwa4b/Cip7ACOshUI4smpCQanqjSY=
 github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU=
 github.com/klauspost/cpuid/v2 v2.0.9 h1:lgaqFMSdTdQYdZ04uHyN2d/eKdOMyi2YLSvlQIBFYa4=
 github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg=
-github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI=
-github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
-github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
-github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk=
-github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
-github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE=
-github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
+github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0=
+github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
 github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
 github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU=
-github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94=
+github.com/mattn/go-isatty v0.0.16 h1:bq3VjFmv/sOjHtdEhmkEV4x1AJtvUvOJ2PFAZ5+peKQ=
 github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM=
-github.com/mattn/go-sqlite3 v1.14.12/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU=
-github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU=
+github.com/mattn/go-sqlite3 v1.14.14 h1:qZgc/Rwetq+MtyE18WhzjokPD93dNqLGNT3QJuLvBGw=
 github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs=
 github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcsrzbxHt8iiaC+zU4b1ylILSosueou12R++wfY=
 github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8DFdX7uMikMLXX4oubIzJF4kv/wI=
 github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE=
-github.com/phpdave11/gofpdf v1.4.2/go.mod h1:zpO6xFn9yxo3YLyMvW8HcKWVdbNqgIfOOp2dXMnm1mY=
-github.com/phpdave11/gofpdi v1.0.12/go.mod h1:vBmVV0Do6hSBHC8uKUQ71JGW+ZGQq74llk/7bXwjDoI=
-github.com/phpdave11/gofpdi v1.0.13/go.mod h1:vBmVV0Do6hSBHC8uKUQ71JGW+ZGQq74llk/7bXwjDoI=
 github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0=
 github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4=
-github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
-github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
 github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
 github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
 github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
+github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 h1:OdAsTTz6OkFY5QxjkYwrChwuRruF69c169dPK26NUlk=
 github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo=
-github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ=
-github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc=
-github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfFZQK844Gfx8o5WFuvpxWRwnSoipWe/p622j1v06w=
-github.com/ruudk/golang-pdf417 v0.0.0-20201230142125-a7e3863a1245/go.mod h1:pQAZKsJ8yyVxGRWYNEm9oFB8ieLgKFnamEyDmSA0BRk=
+github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8=
 github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+github.com/stretchr/objx v0.4.0 h1:M2gUjqZET1qApGOWNSnZ49BAIMX4F/1plDv3+l31EJ4=
 github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
-github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
-github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA=
-github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
 github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
 github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk=
 github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU=
 github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
-github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k=
-github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY=
 github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ=
-github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0=
 github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0=
 github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA=
-go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI=
 golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
-golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
 golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
 golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
-golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
-golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
-golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
 golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
-golang.org/x/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
-golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
-golang.org/x/exp v0.0.0-20191002040644-a1355ae1e2c3/go.mod h1:NOZ3BPKG0ec/BKJQgnvsSFpcKLM5xXVWnvZS97DWHgE=
-golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e h1:+WEEuIdZHnUeJJmEUjyYC2gfUMj69yZXw17EnHg/otA=
-golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e/go.mod h1:Kr81I6Kryrl9sr8s2FK3vxD90NdsKWRuOIl2O4CvYbA=
+golang.org/x/exp v0.0.0-20220827204233-334a2380cb91 h1:tnebWN09GYg9OLPss1KXj8txwZc6X6uMr6VFdcGNbHw=
 golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE=
-golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs=
-golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js=
-golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
-golang.org/x/image v0.0.0-20190910094157-69e4b8554b2a/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
-golang.org/x/image v0.0.0-20200119044424-58c23975cae1/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
-golang.org/x/image v0.0.0-20200430140353-33d19683fad8/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
-golang.org/x/image v0.0.0-20200618115811-c13761719519/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
-golang.org/x/image v0.0.0-20201208152932-35266b937fa6/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
-golang.org/x/image v0.0.0-20210216034530-4410531fe030/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
-golang.org/x/image v0.0.0-20210607152325-775e3b0c77b9/go.mod h1:023OzeP/+EPmXeapQh35lcL3II3LrY8Ic+EFFKVhULM=
-golang.org/x/image v0.0.0-20210628002857-a66eb6448b8d/go.mod h1:023OzeP/+EPmXeapQh35lcL3II3LrY8Ic+EFFKVhULM=
-golang.org/x/image v0.0.0-20211028202545-6944b10bf410/go.mod h1:023OzeP/+EPmXeapQh35lcL3II3LrY8Ic+EFFKVhULM=
-golang.org/x/image v0.0.0-20220302094943-723b81ca9867/go.mod h1:023OzeP/+EPmXeapQh35lcL3II3LrY8Ic+EFFKVhULM=
 golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
 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/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o=
-golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY=
 golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
-golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro=
-golang.org/x/mod v0.6.0-dev.0.20220106191415-9b9b3d81d5e3/go.mod h1:3p9vT2HGsQu2K1YbXdKPJLVgG5VJdoTa1poYQBtP1AY=
 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/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-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
 golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
 golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
 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-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
 golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
-golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
-golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
+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/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
-golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
 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-20181221193216-37e7f081c4d4/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-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/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-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
 golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
 golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
 golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20210304124612-50617c2ba197/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
 golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220808155132-1c4a2a72c664/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/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
-golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8=
 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.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
-golang.org/x/text v0.3.6/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/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
 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-20190206041539-40960b6deb8e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
 golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY=
 golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
 golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
-golang.org/x/tools v0.0.0-20190927191325-030b2cf1153e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
 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.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0=
-golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU=
-golang.org/x/tools v0.1.10/go.mod h1:Uh6Zz+xoGYZom868N8YTex3t7RhtHDBrE8Gzo9bV56E=
 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/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
@@ -260,28 +139,17 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T
 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=
-gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo=
-gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0=
-gonum.org/v1/gonum v0.9.3/go.mod h1:TZumC3NeyVQskjXqmyWt4S3bINhy7B4eYwW69EbyX+0=
 gonum.org/v1/gonum v0.11.0 h1:f1IJhK4Km5tBJmaiJXtk/PkL4cdVX6J+tGiM187uT5E=
-gonum.org/v1/gonum v0.11.0/go.mod h1:fSG4YDCxxUZQJ7rKsQrj0gMOg00Il0Z96/qMA4bVQhA=
-gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw=
-gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc=
-gonum.org/v1/plot v0.9.0/go.mod h1:3Pcqqmp6RHvJI72kgb8fThyUnav364FOsdDo2aGW5lY=
-gonum.org/v1/plot v0.10.1/go.mod h1:VZW5OlhkL1mysU9vaqNHnsy86inf6Ot+jB3r+BczCEo=
 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=
 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
 google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
-google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
+google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013 h1:+kGHl1aib/qcwaRi1CbqBZ1rk19r85MNUf8HaBghugY=
 google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo=
 google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
 google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
-google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY=
 google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
-google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0=
-google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU=
-google.golang.org/grpc v1.48.0/go.mod h1:vN9eftEi1UMyUsIF80+uQXhHjbXYbm0uXoFCACuMGWk=
+google.golang.org/grpc v1.49.0 h1:WTLtQzmQori5FUH25Pq4WT22oCsv8USpQ+F6rqtsmxw=
 google.golang.org/grpc v1.49.0/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCDK+GI=
 google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
 google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0=
@@ -289,62 +157,49 @@ google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQ
 google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE=
 google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo=
 google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
-google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
 google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
-google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c=
 google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw=
 google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
-google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
+google.golang.org/protobuf v1.28.1 h1:d0NfwRgPtno5B1Wa6L2DAG+KivqkdutMf1UhdNx175w=
 google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I=
 gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
-gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY=
-gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
-gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=
-gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI=
-gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
 gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
 gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
 gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
 honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
 honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
-honnef.co/go/tools v0.1.3/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las=
 lukechampine.com/uint128 v1.1.1/go.mod h1:c4eWIwlEGaxC/+H1VguhU4PHXNWDCDMUlWdIWl2j1gk=
+lukechampine.com/uint128 v1.2.0 h1:mBi/5l91vocEN8otkC5bDLhi2KdCticRiwbdB0O+rjI=
 lukechampine.com/uint128 v1.2.0/go.mod h1:c4eWIwlEGaxC/+H1VguhU4PHXNWDCDMUlWdIWl2j1gk=
-modernc.org/cc/v3 v3.36.0/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI=
-modernc.org/cc/v3 v3.36.1/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI=
 modernc.org/cc/v3 v3.36.2/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI=
+modernc.org/cc/v3 v3.36.3 h1:uISP3F66UlixxWEcKuIWERa4TwrZENHSL8tWxZz8bHg=
 modernc.org/cc/v3 v3.36.3/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI=
-modernc.org/ccgo/v3 v3.0.0-20220428102840-41399a37e894/go.mod h1:eI31LL8EwEBKPpNpA4bU1/i+sKOwOrQy8D87zWUcRZc=
-modernc.org/ccgo/v3 v3.0.0-20220430103911-bc99d88307be/go.mod h1:bwdAnOoaIt8Ax9YdWGjxWsdkPcZyRPHqrOvJxaKAKGw=
-modernc.org/ccgo/v3 v3.16.4/go.mod h1:tGtX0gE9Jn7hdZFeU88slbTh1UtCYKusWOoCJuvkWsQ=
-modernc.org/ccgo/v3 v3.16.6/go.mod h1:tGtX0gE9Jn7hdZFeU88slbTh1UtCYKusWOoCJuvkWsQ=
-modernc.org/ccgo/v3 v3.16.8/go.mod h1:zNjwkizS+fIFDrDjIAgBSCLkWbJuHF+ar3QRn+Z9aws=
+modernc.org/ccgo/v3 v3.16.9 h1:AXquSwg7GuMk11pIdw7fmO1Y/ybgazVkMhsZWCV0mHM=
 modernc.org/ccgo/v3 v3.16.9/go.mod h1:zNMzC9A9xeNUepy6KuZBbugn3c0Mc9TeiJO4lgvkJDo=
+modernc.org/ccorpus v1.11.6 h1:J16RXiiqiCgua6+ZvQot4yUuUy8zxgqbqEEUuGPlISk=
 modernc.org/ccorpus v1.11.6/go.mod h1:2gEUTrWqdpH2pXsmTM1ZkjeSrUWDpjMu2T6m29L/ErQ=
+modernc.org/httpfs v1.0.6 h1:AAgIpFZRXuYnkjftxTAZwMIiwEqAfk8aVB2/oA6nAeM=
 modernc.org/httpfs v1.0.6/go.mod h1:7dosgurJGp0sPaRanU53W4xZYKh14wfzX420oZADeHM=
-modernc.org/libc v0.0.0-20220428101251-2d5f3daf273b/go.mod h1:p7Mg4+koNjc8jkqwcoFBJx7tXkpj00G77X7A72jXPXA=
-modernc.org/libc v1.16.0/go.mod h1:N4LD6DBE9cf+Dzf9buBlzVJndKr/iJHG97vGLHYnb5A=
-modernc.org/libc v1.16.1/go.mod h1:JjJE0eu4yeK7tab2n4S1w8tlWd9MxXLRzheaRnAKymU=
-modernc.org/libc v1.16.7/go.mod h1:hYIV5VZczAmGZAnG15Vdngn5HSF5cSkbvfz2B7GRuVU=
-modernc.org/libc v1.16.17/go.mod h1:hYIV5VZczAmGZAnG15Vdngn5HSF5cSkbvfz2B7GRuVU=
-modernc.org/libc v1.16.19/go.mod h1:p7Mg4+koNjc8jkqwcoFBJx7tXkpj00G77X7A72jXPXA=
 modernc.org/libc v1.17.0/go.mod h1:XsgLldpP4aWlPlsjqKRdHPqCxCjISdHfM/yeWC5GyW0=
+modernc.org/libc v1.17.1 h1:Q8/Cpi36V/QBfuQaFVeisEBs3WqoGAJprZzmf7TfEYI=
 modernc.org/libc v1.17.1/go.mod h1:FZ23b+8LjxZs7XtFMbSzL/EhPxNbfZbErxEHc7cbD9s=
 modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E=
 modernc.org/mathutil v1.4.1/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E=
+modernc.org/mathutil v1.5.0 h1:rV0Ko/6SfM+8G+yKiyI830l3Wuz1zRutdslNoQ0kfiQ=
 modernc.org/mathutil v1.5.0/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E=
-modernc.org/memory v1.1.1/go.mod h1:/0wo5ibyrQiaoUoH7f9D8dnglAmILJ5/cxZlRECf+Nw=
 modernc.org/memory v1.2.0/go.mod h1:/0wo5ibyrQiaoUoH7f9D8dnglAmILJ5/cxZlRECf+Nw=
+modernc.org/memory v1.2.1 h1:dkRh86wgmq/bJu2cAS2oqBCz/KsMZU7TUM4CibQ7eBs=
 modernc.org/memory v1.2.1/go.mod h1:PkUhL0Mugw21sHPeskwZW4D6VscE/GQJOnIpCnW6pSU=
 modernc.org/opt v0.1.1/go.mod h1:WdSiB5evDcignE70guQKxYUl14mgWtbClRi5wmkkTX0=
+modernc.org/opt v0.1.3 h1:3XOZf2yznlhC+ibLltsDGzABUGVx8J6pnFMS3E4dcq4=
 modernc.org/opt v0.1.3/go.mod h1:WdSiB5evDcignE70guQKxYUl14mgWtbClRi5wmkkTX0=
-modernc.org/sqlite v1.18.0/go.mod h1:B9fRWZacNxJBHoCJZQr1R54zhVn3fjfl0aszflrTSxY=
+modernc.org/sqlite v1.18.1 h1:ko32eKt3jf7eqIkCgPAeHMBXw3riNSLhl2f3loEF7o8=
 modernc.org/sqlite v1.18.1/go.mod h1:6ho+Gow7oX5V+OiOQ6Tr4xeqbx13UZ6t+Fw9IRUG4d4=
 modernc.org/strutil v1.1.1/go.mod h1:DE+MQQ/hjKBZS2zNInV5hhcipt5rLPWkmpbGeW5mmdw=
-modernc.org/strutil v1.1.2/go.mod h1:OYajnUAcI/MX+XD/Wx7v1bbdvcQSvxgtb0gC+u3d3eg=
+modernc.org/strutil v1.1.3 h1:fNMm+oJklMGYfU9Ylcywl0CO5O6nTfaowNsh2wpPjzY=
 modernc.org/strutil v1.1.3/go.mod h1:MEHNA7PdEnEwLvspRMtWTNnp2nnyvMfkimT1NKNAGbw=
-modernc.org/tcl v1.13.1/go.mod h1:XOLfOwzhkljL4itZkK6T72ckMgvj0BDsnKNdZVUOecw=
+modernc.org/tcl v1.13.1 h1:npxzTwFTZYM8ghWicVIX1cRWzj7Nd8i6AqqX2p+IYao=
+modernc.org/token v1.0.0 h1:a0jaWiNMDhDUtqOj09wvjWWAqd3q7WpBulmL9H2egsk=
 modernc.org/token v1.0.0/go.mod h1:UGzOrNV1mAFSEB63lOFHIpNRUVMvYTc6yu1SMY/XTDM=
-modernc.org/z v1.5.1/go.mod h1:eWFB510QWW5Th9YGZT81s+LwvaAs3Q2yr4sP0rmLkv8=
-rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4=
+modernc.org/z v1.5.1 h1:RTNHdsrOpeoSeOF4FbzTo8gBYByaJ5xT7NgZ9ZqRiJM=
diff --git a/go/adbc/infocode_string.go b/go/adbc/infocode_string.go
new file mode 100644
index 0000000..73af20c
--- /dev/null
+++ b/go/adbc/infocode_string.go
@@ -0,0 +1,39 @@
+// Code generated by "stringer -type InfoCode -linecomment"; DO NOT EDIT.
+
+package adbc
+
+import "strconv"
+
+func _() {
+	// An "invalid array index" compiler error signifies that the constant values have changed.
+	// Re-run the stringer command to generate them again.
+	var x [1]struct{}
+	_ = x[InfoVendorName-0]
+	_ = x[InfoVendorVersion-1]
+	_ = x[InfoVendorArrowVersion-2]
+	_ = x[InfoDriverName-100]
+	_ = x[InfoDriverVersion-101]
+	_ = x[InfoDriverArrowVersion-102]
+}
+
+const (
+	_InfoCode_name_0 = "VendorNameVendorVersionVendorArrowVersion"
+	_InfoCode_name_1 = "DriverNameDriverVersionDriverArrowVersion"
+)
+
+var (
+	_InfoCode_index_0 = [...]uint8{0, 10, 23, 41}
+	_InfoCode_index_1 = [...]uint8{0, 10, 23, 41}
+)
+
+func (i InfoCode) String() string {
+	switch {
+	case i <= 2:
+		return _InfoCode_name_0[_InfoCode_index_0[i]:_InfoCode_index_0[i+1]]
+	case 100 <= i && i <= 102:
+		i -= 100
+		return _InfoCode_name_1[_InfoCode_index_1[i]:_InfoCode_index_1[i+1]]
+	default:
+		return "InfoCode(" + strconv.FormatInt(int64(i), 10) + ")"
+	}
+}
diff --git a/go/adbc/sqldriver/driver.go b/go/adbc/sqldriver/driver.go
index 1a17912..f955717 100644
--- a/go/adbc/sqldriver/driver.go
+++ b/go/adbc/sqldriver/driver.go
@@ -87,7 +87,8 @@ func parseConnectStr(str string) (ret map[string]string, err error) {
 }
 
 type connector struct {
-	driver adbc.Driver
+	db  adbc.Database
+	drv adbc.Driver
 }
 
 // Connect returns a connection to the database. Connect may
@@ -103,17 +104,17 @@ type connector struct {
 //
 // The returned connection is only used by one goroutine at a time.
 func (c *connector) Connect(ctx context.Context) (driver.Conn, error) {
-	cnxn, err := c.driver.Open(ctx)
+	cnxn, err := c.db.Open(ctx)
 	if err != nil {
 		return nil, err
 	}
 
-	return &conn{Conn: cnxn, drv: c.driver}, nil
+	return &conn{Conn: cnxn, drv: c.db}, nil
 }
 
 // Driver returns the underlying Driver of the connector,
 // mainly to maintain compatibility with the Driver method on sql.DB
-func (c *connector) Driver() driver.Driver { return &Driver{c.driver} }
+func (c *connector) Driver() driver.Driver { return Driver{c.drv} }
 
 type Driver struct {
 	Driver adbc.Driver
@@ -129,21 +130,11 @@ type Driver struct {
 //
 // The returned connection is only used by one goroutine at a time.
 func (d Driver) Open(name string) (driver.Conn, error) {
-	opts, err := parseConnectStr(name)
-	if err != nil {
-		return nil, err
-	}
-	err = d.Driver.SetOptions(opts)
-	if err != nil {
-		return nil, err
-	}
-
-	cnxn, err := d.Driver.Open(context.Background())
+	connector, err := d.OpenConnector(name)
 	if err != nil {
 		return nil, err
 	}
-
-	return &conn{Conn: cnxn, drv: d.Driver}, nil
+	return connector.Connect(context.Background())
 }
 
 // OpenConnector expects the same format as driver.Open
@@ -152,12 +143,13 @@ func (d Driver) OpenConnector(name string) (driver.Connector, error) {
 	if err != nil {
 		return nil, err
 	}
-	err = d.Driver.SetOptions(opts)
+
+	db, err := d.Driver.NewDatabase(opts)
 	if err != nil {
 		return nil, err
 	}
 
-	return &connector{d.Driver}, nil
+	return &connector{db, d.Driver}, nil
 }
 
 type ctxOptsKey struct{}
@@ -178,7 +170,7 @@ func GetOptionsFromCtx(ctx context.Context) map[string]string {
 // multiple goroutines. It is assumed to be stateful.
 type conn struct {
 	Conn adbc.Connection
-	drv  adbc.Driver
+	drv  adbc.Database
 }
 
 // Close invalidates and potentially stops any current prepared
diff --git a/go/adbc/standard_schemas.go b/go/adbc/standard_schemas.go
new file mode 100644
index 0000000..bd648eb
--- /dev/null
+++ b/go/adbc/standard_schemas.go
@@ -0,0 +1,94 @@
+// 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 adbc
+
+import "github.com/apache/arrow/go/v10/arrow"
+
+var (
+	GetInfoSchema = arrow.NewSchema([]arrow.Field{
+		{Name: "info_name", Type: arrow.PrimitiveTypes.Uint32},
+		{Name: "info_value", Type: arrow.DenseUnionOf(
+			[]arrow.Field{
+				{Name: "string_value", Type: arrow.BinaryTypes.String, Nullable: true},
+				{Name: "bool_value", Type: arrow.FixedWidthTypes.Boolean, Nullable: true},
+				{Name: "int64_value", Type: arrow.PrimitiveTypes.Int64, Nullable: true},
+				{Name: "int32_bitmask", Type: arrow.PrimitiveTypes.Int32, Nullable: true},
+				{Name: "string_list", Type: arrow.ListOf(arrow.BinaryTypes.String), Nullable: true},
+				{Name: "int32_to_int32_list_map",
+					Type: arrow.MapOf(arrow.PrimitiveTypes.Int32,
+						arrow.ListOf(arrow.PrimitiveTypes.Int32)), Nullable: true},
+			},
+			[]arrow.UnionTypeCode{0, 1, 2, 3, 4, 5},
+		)},
+	}, nil)
+
+	TableTypesSchema = arrow.NewSchema([]arrow.Field{{Name: "table_type", Type: arrow.BinaryTypes.String}}, nil)
+
+	UsageSchema = arrow.StructOf(
+		arrow.Field{Name: "fk_catalog", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "fk_db_schema", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "fk_table", Type: arrow.BinaryTypes.String},
+		arrow.Field{Name: "fk_column_name", Type: arrow.BinaryTypes.String},
+	)
+
+	ConstraintSchema = arrow.StructOf(
+		arrow.Field{Name: "constraint_name", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "constraint_type", Type: arrow.BinaryTypes.String},
+		arrow.Field{Name: "constraint_column_names", Type: arrow.ListOf(arrow.BinaryTypes.String)},
+		arrow.Field{Name: "constraint_column_usage", Type: arrow.ListOf(UsageSchema), Nullable: true},
+	)
+
+	ColumnSchema = arrow.StructOf(
+		arrow.Field{Name: "column_name", Type: arrow.BinaryTypes.String},
+		arrow.Field{Name: "ordinal_position", Type: arrow.PrimitiveTypes.Int32, Nullable: true},
+		arrow.Field{Name: "remarks", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "xdbc_data_type", Type: arrow.PrimitiveTypes.Int16, Nullable: true},
+		arrow.Field{Name: "xdbc_type_name", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "xdbc_column_size", Type: arrow.PrimitiveTypes.Int32, Nullable: true},
+		arrow.Field{Name: "xdbc_decimal_digits", Type: arrow.PrimitiveTypes.Int16, Nullable: true},
+		arrow.Field{Name: "xdbc_num_prec_radix", Type: arrow.PrimitiveTypes.Int16, Nullable: true},
+		arrow.Field{Name: "xdbc_nullable", Type: arrow.PrimitiveTypes.Int16, Nullable: true},
+		arrow.Field{Name: "xdbc_column_def", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "xdbc_sql_data_type", Type: arrow.PrimitiveTypes.Int16, Nullable: true},
+		arrow.Field{Name: "xdbc_datetime_sub", Type: arrow.PrimitiveTypes.Int16, Nullable: true},
+		arrow.Field{Name: "xdbc_char_octet_length", Type: arrow.PrimitiveTypes.Int32, Nullable: true},
+		arrow.Field{Name: "xdbc_is_nullable", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "xdbc_scope_catalog", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "xdbc_scope_schema", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "xdbc_scope_table", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "xdbc_is_autoincrement", Type: arrow.FixedWidthTypes.Boolean, Nullable: true},
+		arrow.Field{Name: "xdbc_is_generatedcolumn", Type: arrow.FixedWidthTypes.Boolean, Nullable: true},
+	)
+
+	TableSchema = arrow.StructOf(
+		arrow.Field{Name: "table_name", Type: arrow.BinaryTypes.String},
+		arrow.Field{Name: "table_type", Type: arrow.BinaryTypes.String},
+		arrow.Field{Name: "table_columns", Type: arrow.ListOf(ColumnSchema), Nullable: true},
+		arrow.Field{Name: "table_constraints", Type: arrow.ListOf(ConstraintSchema), Nullable: true},
+	)
+
+	DBSchemaSchema = arrow.StructOf(
+		arrow.Field{Name: "db_schema_name", Type: arrow.BinaryTypes.String, Nullable: true},
+		arrow.Field{Name: "db_schema_tables", Type: arrow.ListOf(TableSchema), Nullable: true},
+	)
+
+	GetObjectsSchema = arrow.NewSchema([]arrow.Field{
+		{Name: "catalog_name", Type: arrow.BinaryTypes.String, Nullable: true},
+		{Name: "catalog_db_schemas", Type: arrow.ListOf(DBSchemaSchema), Nullable: true},
+	}, nil)
+)
diff --git a/go/adbc/validation/validation.go b/go/adbc/validation/validation.go
new file mode 100644
index 0000000..99ec5fb
--- /dev/null
+++ b/go/adbc/validation/validation.go
@@ -0,0 +1,424 @@
+// 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 validation is a driver-agnostic test suite intended to aid in
+// driver development for ADBC drivers. It provides a series of utilities
+// and defined tests that can be used to validate a driver follows the
+// correct and expected behavior.
+package validation
+
+import (
+	"context"
+	"strings"
+	"testing"
+
+	"github.com/apache/arrow-adbc/go/adbc"
+	"github.com/apache/arrow/go/v10/arrow"
+	"github.com/apache/arrow/go/v10/arrow/array"
+	"github.com/apache/arrow/go/v10/arrow/flight/flightsql"
+	"github.com/apache/arrow/go/v10/arrow/memory"
+	"github.com/stretchr/testify/suite"
+)
+
+type DriverQuirks interface {
+	// Called in SetupTest to initialize anything needed for testing
+	SetupDriver(*testing.T) adbc.Driver
+	// Called in TearDownTest to clean up anything necessary in between tests
+	TearDownDriver(*testing.T, adbc.Driver)
+	// Return the list of key/value pairs of options to pass when
+	// calling NewDatabase
+	DatabaseOptions() map[string]string
+	// Return the SQL to reference the bind parameter for a given index
+	BindParameter(index int) string
+	// Whether two statements can be used at the same time on a single connection
+	SupportsConcurrentStatements() bool
+	// Whether AdbcStatementExecutePartitions should work
+	SupportsPartitionedData() bool
+	// Whether transactions are supported (Commit/Rollback on connection)
+	SupportsTransactions() bool
+	// Whether retrieving the schema of prepared statement params is supported
+	SupportsGetParameterSchema() bool
+	// Expected Metadata responses
+	GetMetadata(adbc.InfoCode) interface{}
+	// Create a sample table from an arrow record
+	CreateSampleTable(tableName string, r arrow.Record) error
+}
+
+type DatabaseTests struct {
+	suite.Suite
+
+	Driver adbc.Driver
+	Quirks DriverQuirks
+}
+
+func (d *DatabaseTests) SetupTest() {
+	d.Driver = d.Quirks.SetupDriver(d.T())
+}
+
+func (d *DatabaseTests) TearDownTest() {
+	d.Quirks.TearDownDriver(d.T(), d.Driver)
+	d.Driver = nil
+}
+
+func (d *DatabaseTests) TestNewDatabase() {
+	db, err := d.Driver.NewDatabase(d.Quirks.DatabaseOptions())
+	d.NoError(err)
+	d.NotNil(db)
+	d.Implements((*adbc.Database)(nil), db)
+}
+
+type ConnectionTests struct {
+	suite.Suite
+
+	Driver adbc.Driver
+	Quirks DriverQuirks
+
+	DB adbc.Database
+}
+
+func (c *ConnectionTests) SetupTest() {
+	c.Driver = c.Quirks.SetupDriver(c.T())
+	var err error
+	c.DB, err = c.Driver.NewDatabase(c.Quirks.DatabaseOptions())
+	c.Require().NoError(err)
+}
+
+func (c *ConnectionTests) TearDownTest() {
+	c.Quirks.TearDownDriver(c.T(), c.Driver)
+	c.Driver = nil
+	c.DB = nil
+}
+
+func (c *ConnectionTests) TestNewConn() {
+	cnxn, err := c.DB.Open(context.Background())
+	c.NoError(err)
+	c.NotNil(cnxn)
+
+	c.NoError(cnxn.Close())
+}
+
+func (c *ConnectionTests) TestCloseConnTwice() {
+	cnxn, err := c.DB.Open(context.Background())
+	c.NoError(err)
+	c.NotNil(cnxn)
+
+	c.NoError(cnxn.Close())
+	err = cnxn.Close()
+	var adbcError adbc.Error
+	c.ErrorAs(err, &adbcError)
+	c.Equal(adbc.StatusInvalidState, adbcError.Code)
+}
+
+func (c *ConnectionTests) TestConcurrent() {
+	cnxn, _ := c.DB.Open(context.Background())
+	cnxn2, err := c.DB.Open(context.Background())
+	c.Require().NoError(err)
+
+	c.NoError(cnxn.Close())
+	c.NoError(cnxn2.Close())
+}
+
+func (c *ConnectionTests) TestAutocommitDefault() {
+	ctx := context.Background()
+	// even if not supported, drivers should act as if autocommit
+	// is enabled, and return INVALID_STATE if the client tries to
+	// commit or rollback
+	cnxn, _ := c.DB.Open(ctx)
+	defer cnxn.Close()
+
+	expectedCode := adbc.StatusInvalidState
+	if !c.Quirks.SupportsTransactions() {
+		expectedCode = adbc.StatusNotImplemented
+	}
+
+	var adbcError adbc.Error
+	err := cnxn.Commit(ctx)
+	c.ErrorAs(err, &adbcError)
+	c.Equal(expectedCode, adbcError.Code)
+	err = cnxn.Rollback(ctx)
+	c.ErrorAs(err, &adbcError)
+	c.Equal(expectedCode, adbcError.Code)
+
+	// if the driver supports setting options after init, it should error
+	// on an invalid option value for autocommit
+	if cnxnopts, ok := cnxn.(adbc.PostInitOptions); ok {
+		c.Error(cnxnopts.SetOption(adbc.OptionKeyAutoCommit, "invalid"))
+	}
+}
+
+func (c *ConnectionTests) TestAutocommitToggle() {
+	ctx := context.Background()
+	cnxn, _ := c.DB.Open(ctx)
+	defer cnxn.Close()
+
+	if !c.Quirks.SupportsTransactions() {
+		return
+	}
+
+	// if the connection doesn't support setting options after init
+	// then there's nothing to test here
+	cnxnopt, ok := cnxn.(adbc.PostInitOptions)
+	if !ok {
+		return
+	}
+
+	// it is ok to enable autocommit when it is already enabled
+	c.NoError(cnxnopt.SetOption(adbc.OptionKeyAutoCommit, adbc.OptionValueEnabled))
+	c.NoError(cnxnopt.SetOption(adbc.OptionKeyAutoCommit, adbc.OptionValueDisabled))
+
+	// it is ok to disable autocommit when it isn't enabled
+	c.NoError(cnxnopt.SetOption(adbc.OptionKeyAutoCommit, adbc.OptionValueDisabled))
+}
+
+func (c *ConnectionTests) TestMetadataGetInfo() {
+	ctx := context.Background()
+	cnxn, _ := c.DB.Open(ctx)
+	defer cnxn.Close()
+
+	info := []adbc.InfoCode{
+		adbc.InfoDriverName,
+		adbc.InfoDriverVersion,
+		adbc.InfoDriverArrowVersion,
+		adbc.InfoVendorName,
+		adbc.InfoVendorVersion,
+		adbc.InfoVendorArrowVersion,
+	}
+
+	rdr, err := cnxn.GetInfo(ctx, info)
+	c.Require().NoError(err)
+	defer rdr.Release()
+
+	c.Truef(adbc.GetInfoSchema.Equal(rdr.Schema()), "expected: %s\ngot: %s",
+		adbc.GetInfoSchema, rdr.Schema())
+
+	for rdr.Next() {
+		rec := rdr.Record()
+		codeCol := rec.Column(0).(*array.Uint32)
+		valUnion := rec.Column(1).(*array.DenseUnion)
+		for i := 0; i < int(rec.NumRows()); i++ {
+			code := codeCol.Value(i)
+
+			child := valUnion.Field(valUnion.ChildID(i))
+			// currently we only define utf8 values for metadata
+			c.Equal(c.Quirks.GetMetadata(adbc.InfoCode(code)), child.(*array.String).Value(i), adbc.InfoCode(code).String())
+		}
+	}
+}
+
+func (c *ConnectionTests) TestMetadataGetTableSchema() {
+	rec, _, err := array.RecordFromJSON(memory.DefaultAllocator, arrow.NewSchema(
+		[]arrow.Field{
+			{Name: "ints", Type: arrow.PrimitiveTypes.Int64, Nullable: true},
+			{Name: "strings", Type: arrow.BinaryTypes.String, Nullable: true},
+		}, nil), strings.NewReader(`[
+			{"ints": 42, "strings": "foo"},
+			{"ints": -42, "strings": null},
+			{"ints": null, "strings": ""}
+		]`))
+	c.Require().NoError(err)
+	defer rec.Release()
+
+	ctx := context.Background()
+	cnxn, _ := c.DB.Open(ctx)
+	defer cnxn.Close()
+
+	c.Require().NoError(c.Quirks.CreateSampleTable("sample_test", rec))
+
+	sc, err := cnxn.GetTableSchema(ctx, nil, nil, "sample_test")
+	c.Require().NoError(err)
+
+	expectedSchema := arrow.NewSchema([]arrow.Field{
+		{Name: "ints", Type: arrow.PrimitiveTypes.Int64,
+			Metadata: arrow.MetadataFrom(map[string]string{
+				flightsql.ScaleKey: "15", flightsql.IsReadOnlyKey: "0", flightsql.IsAutoIncrementKey: "0",
+				flightsql.TableNameKey: "sample_test", flightsql.PrecisionKey: "10"})},
+		{Name: "strings", Type: arrow.BinaryTypes.String,
+			Metadata: arrow.MetadataFrom(map[string]string{
+				flightsql.ScaleKey: "15", flightsql.IsReadOnlyKey: "0", flightsql.IsAutoIncrementKey: "0",
+				flightsql.TableNameKey: "sample_test"})},
+	}, nil)
+
+	c.Truef(expectedSchema.Equal(sc), "expected: %s\ngot: %s", expectedSchema, sc)
+}
+
+func (c *ConnectionTests) TestMetadataGetTableTypes() {
+	ctx := context.Background()
+	cnxn, _ := c.DB.Open(ctx)
+	defer cnxn.Close()
+
+	rdr, err := cnxn.GetTableTypes(ctx)
+	c.Require().NoError(err)
+	defer rdr.Release()
+
+	c.Truef(adbc.TableTypesSchema.Equal(rdr.Schema()), "expected: %s\ngot: %s", adbc.TableTypesSchema, rdr.Schema())
+	c.True(rdr.Next())
+}
+
+type StatementTests struct {
+	suite.Suite
+
+	Driver adbc.Driver
+	Quirks DriverQuirks
+
+	DB   adbc.Database
+	Cnxn adbc.Connection
+	ctx  context.Context
+}
+
+func (s *StatementTests) SetupTest() {
+	s.Driver = s.Quirks.SetupDriver(s.T())
+	var err error
+	s.DB, err = s.Driver.NewDatabase(s.Quirks.DatabaseOptions())
+	s.Require().NoError(err)
+	s.ctx = context.Background()
+	s.Cnxn, err = s.DB.Open(s.ctx)
+	s.Require().NoError(err)
+}
+
+func (s *StatementTests) TearDownTest() {
+	s.Require().NoError(s.Cnxn.Close())
+	s.Quirks.TearDownDriver(s.T(), s.Driver)
+	s.Cnxn = nil
+	s.DB = nil
+	s.Driver = nil
+}
+
+func (s *StatementTests) TestNewStatement() {
+	stmt, err := s.Cnxn.NewStatement()
+	s.NoError(err)
+	s.NotNil(stmt)
+	s.NoError(stmt.Close())
+
+	var adbcError adbc.Error
+	s.ErrorAs(stmt.Close(), &adbcError)
+	s.Equal(adbc.StatusInvalidState, adbcError.Code)
+
+	stmt, err = s.Cnxn.NewStatement()
+	s.NoError(err)
+	_, _, err = stmt.ExecuteQuery(s.ctx)
+	s.ErrorAs(err, &adbcError)
+	s.Equal(adbc.StatusInvalidState, adbcError.Code)
+}
+
+func (s *StatementTests) TestSqlPartitionedInts() {
+	stmt, err := s.Cnxn.NewStatement()
+	s.Require().NoError(err)
+	defer stmt.Close()
+
+	s.NoError(stmt.SetSqlQuery("SELECT 42"))
+
+	var adbcError adbc.Error
+	if !s.Quirks.SupportsPartitionedData() {
+		_, _, _, err := stmt.ExecutePartitions(s.ctx)
+		s.ErrorAs(err, &adbcError)
+		s.Equal(adbc.StatusNotImplemented, adbcError.Code)
+		return
+	}
+
+	sc, part, rows, err := stmt.ExecutePartitions(s.ctx)
+	s.Require().NoError(err)
+
+	s.EqualValues(1, part.NumPartitions)
+	s.Len(part.PartitionIDs, 1)
+	s.True(rows == 1 || rows == -1, rows)
+
+	if sc != nil {
+		s.Len(sc.Fields(), 1)
+	}
+
+	cxn, err := s.DB.Open(s.ctx)
+	s.Require().NoError(err)
+	defer cxn.Close()
+
+	rdr, err := cxn.ReadPartition(s.ctx, part.PartitionIDs[0])
+	s.Require().NoError(err)
+	defer rdr.Release()
+
+	sc = rdr.Schema()
+	s.Require().NotNil(sc)
+	s.Len(sc.Fields(), 1)
+
+	s.True(rdr.Next())
+	rec := rdr.Record()
+	s.EqualValues(1, rec.NumCols())
+	s.EqualValues(1, rec.NumRows())
+
+	switch arr := rec.Column(0).(type) {
+	case *array.Int32:
+		s.EqualValues(42, arr.Value(0))
+	case *array.Int64:
+		s.EqualValues(42, arr.Value(0))
+	}
+
+	s.False(rdr.Next())
+}
+
+func (s *StatementTests) TestSQLPrepareGetParameterSchema() {
+	stmt, err := s.Cnxn.NewStatement()
+	s.NoError(err)
+	defer stmt.Close()
+
+	query := "SELECT " + s.Quirks.BindParameter(0) + ", " + s.Quirks.BindParameter(1)
+	s.NoError(stmt.SetSqlQuery(query))
+	s.NoError(stmt.Prepare(s.ctx))
+
+	sc, err := stmt.GetParameterSchema()
+	if !s.Quirks.SupportsGetParameterSchema() {
+		var adbcError adbc.Error
+		s.ErrorAs(err, &adbcError)
+		s.Equal(adbc.StatusNotImplemented, adbcError.Code)
+		return
+	}
+	s.NoError(err)
+
+	// it's allowed to be nil as some systems don't provide param schemas
+	if sc != nil {
+		s.Len(sc.Fields(), 2)
+	}
+}
+
+func (s *StatementTests) TestSQLPrepareSelectNoParams() {
+	stmt, err := s.Cnxn.NewStatement()
+	s.NoError(err)
+	defer stmt.Close()
+
+	s.NoError(stmt.SetSqlQuery("SELECT 1"))
+	s.NoError(stmt.Prepare(s.ctx))
+
+	rdr, n, err := stmt.ExecuteQuery(s.ctx)
+	s.Require().NoError(err)
+	s.True(n == 1 || n == -1)
+	defer rdr.Release()
+
+	sc := rdr.Schema()
+	s.Require().NotNil(sc)
+	s.Len(sc.Fields(), 1)
+
+	s.True(rdr.Next())
+	rec := rdr.Record()
+	s.EqualValues(1, rec.NumCols())
+	s.EqualValues(1, rec.NumRows())
+
+	switch arr := rec.Column(0).(type) {
+	case *array.Int32:
+		s.EqualValues(1, arr.Value(0))
+	case *array.Int64:
+		s.EqualValues(1, arr.Value(0))
+	}
+
+	s.False(rdr.Next())
+}