You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "srebhan (via GitHub)" <gi...@apache.org> on 2023/02/24 14:02:33 UTC

[GitHub] [arrow] srebhan opened a new pull request, #34331: [Go][FlightRPC] Add driver for `database/sql` framework

srebhan opened a new pull request, #34331:
URL: https://github.com/apache/arrow/pull/34331

   ### Rationale for this change
   
   Using Golang's `database/sql` framework is well known, offers goodies like connection pooling and is easy to use. Therefore using FlightSQL trough this framework is a good starting point for users performing simple queries, inserts etc.
   
   ### What changes are included in this PR?
   
   This PR adds an `database/sql/driver` implementation currently supporting `sqlite` and `InfluxData IOx` (query only). Unit-tests are added using the SQLite server example implementation and the driver and driver settings are documented.
   
   ### Are these changes tested?
   
   Yes, a test-suite is added for the driver. Futhermore, the IOx backend is additionally tested against a real local instance using [this code](https://github.com/srebhan/go-flightsql-example).
   
   ### Are there any user-facing changes?
   
   This PR does not contain breaking changes. All modifications to the FlightSQL client code are transparent to the user.


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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1132238576


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}
+
+func fromArrowType(arr arrow.Array, idx int) (interface{}, error) {
+	switch c := arr.(type) {
+	case *array.Boolean:
+		return c.Value(idx), nil
+	case *array.Float16:
+		return float64(c.Value(idx).Float32()), nil
+	case *array.Float32:
+		return float64(c.Value(idx)), nil
+	case *array.Float64:
+		return c.Value(idx), nil
+	case *array.Int8:
+		return int64(c.Value(idx)), nil
+	case *array.Int16:
+		return int64(c.Value(idx)), nil
+	case *array.Int32:
+		return int64(c.Value(idx)), nil
+	case *array.Int64:
+		return c.Value(idx), nil
+	case *array.String:
+		return c.Value(idx), nil
+	case *array.Time32:
+		dt, ok := arr.DataType().(*arrow.Time32Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time32", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Time64:
+		dt, ok := arr.DataType().(*arrow.Time64Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time64", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Timestamp:
+		dt, ok := arr.DataType().(*arrow.TimestampType)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching timestamp", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	}
+
+	return nil, fmt.Errorf("type %T: %w", arr, ErrNotSupported)
+}
+
+func toArrowDataType(value interface{}) (arrow.DataType, error) {
+	switch value.(type) {
+	case bool:
+		return &arrow.BooleanType{}, nil
+	case float32:
+		return &arrow.Float32Type{}, nil
+	case float64:
+		return &arrow.Float64Type{}, nil
+	case int8:
+		return &arrow.Int8Type{}, nil
+	case int16:
+		return &arrow.Int16Type{}, nil
+	case int32:
+		return &arrow.Int32Type{}, nil
+	case int64:
+		return &arrow.Int64Type{}, nil
+	case uint8:
+		return &arrow.Uint8Type{}, nil
+	case uint16:
+		return &arrow.Uint16Type{}, nil
+	case uint32:
+		return &arrow.Uint32Type{}, nil
+	case uint64:
+		return &arrow.Uint64Type{}, nil
+	case string:
+		return &arrow.StringType{}, nil
+	case time.Time:
+		return &arrow.Time64Type{Unit: arrow.Nanosecond}, nil
+	}
+	return nil, fmt.Errorf("type %T: %w", value, ErrNotSupported)
+}
+
+func setFieldValue(builder array.Builder, arg interface{}) error {
+	switch b := builder.(type) {
+	case *array.BooleanBuilder:
+		switch v := arg.(type) {
+		case bool:
+			b.Append(v)
+		case []bool:
+			b.AppendValues(v, nil)

Review Comment:
   Well the **schema** specified the `BooleanBuilder` and the user passed in a `[]bool`. Isn't this a valid use?



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

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

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


[GitHub] [arrow] zeroshade commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @jacobmarble I like the idea of having a separate module solely consisting of registering the flightsql driver + a README with the options / how-to-use documentation etc.
   
   I recommend putting it in the ADBC repo solely because in the past I've had difficulties with a submodule of the Arrow Go module due to the `v12` major version in the import path. Could you put up a PR for this on the ADBC repo and myself and @lidavidm will take a look at it as soon as we can to move this forward.
   
   Thanks much!


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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1141852825


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}
+
+func fromArrowType(arr arrow.Array, idx int) (interface{}, error) {
+	switch c := arr.(type) {
+	case *array.Boolean:
+		return c.Value(idx), nil
+	case *array.Float16:
+		return float64(c.Value(idx).Float32()), nil
+	case *array.Float32:
+		return float64(c.Value(idx)), nil
+	case *array.Float64:
+		return c.Value(idx), nil
+	case *array.Int8:
+		return int64(c.Value(idx)), nil
+	case *array.Int16:
+		return int64(c.Value(idx)), nil
+	case *array.Int32:
+		return int64(c.Value(idx)), nil
+	case *array.Int64:
+		return c.Value(idx), nil
+	case *array.String:
+		return c.Value(idx), nil
+	case *array.Time32:
+		dt, ok := arr.DataType().(*arrow.Time32Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time32", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Time64:
+		dt, ok := arr.DataType().(*arrow.Time64Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time64", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Timestamp:
+		dt, ok := arr.DataType().(*arrow.TimestampType)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching timestamp", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	}
+
+	return nil, fmt.Errorf("type %T: %w", arr, ErrNotSupported)
+}
+
+func toArrowDataType(value interface{}) (arrow.DataType, error) {
+	switch value.(type) {
+	case bool:
+		return &arrow.BooleanType{}, nil
+	case float32:
+		return &arrow.Float32Type{}, nil
+	case float64:
+		return &arrow.Float64Type{}, nil
+	case int8:
+		return &arrow.Int8Type{}, nil
+	case int16:
+		return &arrow.Int16Type{}, nil
+	case int32:
+		return &arrow.Int32Type{}, nil
+	case int64:
+		return &arrow.Int64Type{}, nil
+	case uint8:
+		return &arrow.Uint8Type{}, nil
+	case uint16:
+		return &arrow.Uint16Type{}, nil
+	case uint32:
+		return &arrow.Uint32Type{}, nil
+	case uint64:
+		return &arrow.Uint64Type{}, nil
+	case string:
+		return &arrow.StringType{}, nil
+	case time.Time:
+		return &arrow.Time64Type{Unit: arrow.Nanosecond}, nil
+	}
+	return nil, fmt.Errorf("type %T: %w", value, ErrNotSupported)
+}
+
+func setFieldValue(builder array.Builder, arg interface{}) error {
+	switch b := builder.(type) {
+	case *array.BooleanBuilder:
+		switch v := arg.(type) {

Review Comment:
   Ok, will add a comment and only support scalar values as this is most probably the only thing needed for parameters. If it turns out we require more, we can still add it later...



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

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

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


[GitHub] [arrow] srebhan commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @zeroshade thanks again for your thorough review! I've fixed most of your comments (at least IMO ;-)). As I'm not used to your usual habits I left the comments unresolved. Please let me know if I should resolve them by myself or let you go through those.
   
   There are a few point that are open
   1. Unification of backends: I think I should not deal with backend-specific settings at that level. So my idea is to unify those and let the user do the specifics on the `DSN` level. I hope to finish this next Friday...
   2. TLS implementation/customization: My plan is to implement a mechanism similar to the [MySQL driver](https://github.com/go-sql-driver/mysql) to have a driver-global TLS-config registry. I thought about doing this in a separate PR once this one is merged. What do you think?
   3. Moving the driver to another location: You suggested to move this to a separate directory. I'm fine with this, but I'm concerned about testing. The driver, by nature, is closely bound to the API and behavior of the FlightSQL client, therefore it's tests should run whenever something in the client changes. In a separate directory chances are high that the tests are forgotten... Anyway, after unification (see item 1) only `driver.go` and `driver_test.go` will be left. Do you want a separate dir nonetheless?
   
   Thanks again for your support and comments! Much appreciated.


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

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

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


[GitHub] [arrow] lidavidm commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   How does this differ from adbc/sqldriver? https://github.com/apache/arrow-adbc/blob/main/go/adbc/sqldriver/driver.go


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

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

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


[GitHub] [arrow] jorisvandenbossche commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @srebhan sorry for closing this a bit too fast


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

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

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


[GitHub] [arrow] zeroshade commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()

Review Comment:
   It should be pretty easy to create a sample testing server to utilize for the tests by following the examples and have that return an artificial schema to test with.



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

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

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


[GitHub] [arrow] zeroshade commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}
+
+func fromArrowType(arr arrow.Array, idx int) (interface{}, error) {
+	switch c := arr.(type) {
+	case *array.Boolean:
+		return c.Value(idx), nil
+	case *array.Float16:
+		return float64(c.Value(idx).Float32()), nil
+	case *array.Float32:
+		return float64(c.Value(idx)), nil
+	case *array.Float64:
+		return c.Value(idx), nil
+	case *array.Int8:
+		return int64(c.Value(idx)), nil
+	case *array.Int16:
+		return int64(c.Value(idx)), nil
+	case *array.Int32:
+		return int64(c.Value(idx)), nil
+	case *array.Int64:
+		return c.Value(idx), nil
+	case *array.String:
+		return c.Value(idx), nil
+	case *array.Time32:
+		dt, ok := arr.DataType().(*arrow.Time32Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time32", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Time64:
+		dt, ok := arr.DataType().(*arrow.Time64Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time64", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Timestamp:
+		dt, ok := arr.DataType().(*arrow.TimestampType)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching timestamp", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	}
+
+	return nil, fmt.Errorf("type %T: %w", arr, ErrNotSupported)
+}
+
+func toArrowDataType(value interface{}) (arrow.DataType, error) {
+	switch value.(type) {
+	case bool:
+		return &arrow.BooleanType{}, nil
+	case float32:
+		return &arrow.Float32Type{}, nil
+	case float64:
+		return &arrow.Float64Type{}, nil
+	case int8:
+		return &arrow.Int8Type{}, nil
+	case int16:
+		return &arrow.Int16Type{}, nil
+	case int32:
+		return &arrow.Int32Type{}, nil
+	case int64:
+		return &arrow.Int64Type{}, nil
+	case uint8:
+		return &arrow.Uint8Type{}, nil
+	case uint16:
+		return &arrow.Uint16Type{}, nil
+	case uint32:
+		return &arrow.Uint32Type{}, nil
+	case uint64:
+		return &arrow.Uint64Type{}, nil
+	case string:
+		return &arrow.StringType{}, nil
+	case time.Time:
+		return &arrow.Time64Type{Unit: arrow.Nanosecond}, nil
+	}
+	return nil, fmt.Errorf("type %T: %w", value, ErrNotSupported)
+}
+
+func setFieldValue(builder array.Builder, arg interface{}) error {
+	switch b := builder.(type) {
+	case *array.BooleanBuilder:
+		switch v := arg.(type) {
+		case bool:
+			b.Append(v)
+		case []bool:
+			b.AppendValues(v, nil)

Review Comment:
   I don't believe so. Consider the case where the query is something like: `SELECT * FROM <table> WHERE col = ?` and the server returns a parameter schema with a `Boolean` column. If you prepared the above statement, you'd expect a *single* value to be passed, not a slice of values. 
   
   The situation where you have a slice of bools as a single value would be if the expected parameter is supposed to be a *list* of boolean values.



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

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

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


[GitHub] [arrow] zeroshade commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}
+
+func fromArrowType(arr arrow.Array, idx int) (interface{}, error) {
+	switch c := arr.(type) {
+	case *array.Boolean:
+		return c.Value(idx), nil
+	case *array.Float16:
+		return float64(c.Value(idx).Float32()), nil
+	case *array.Float32:
+		return float64(c.Value(idx)), nil
+	case *array.Float64:
+		return c.Value(idx), nil
+	case *array.Int8:
+		return int64(c.Value(idx)), nil
+	case *array.Int16:
+		return int64(c.Value(idx)), nil
+	case *array.Int32:
+		return int64(c.Value(idx)), nil
+	case *array.Int64:
+		return c.Value(idx), nil
+	case *array.String:
+		return c.Value(idx), nil
+	case *array.Time32:
+		dt, ok := arr.DataType().(*arrow.Time32Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time32", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Time64:
+		dt, ok := arr.DataType().(*arrow.Time64Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time64", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Timestamp:
+		dt, ok := arr.DataType().(*arrow.TimestampType)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching timestamp", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	}
+
+	return nil, fmt.Errorf("type %T: %w", arr, ErrNotSupported)
+}
+
+func toArrowDataType(value interface{}) (arrow.DataType, error) {
+	switch value.(type) {
+	case bool:
+		return &arrow.BooleanType{}, nil
+	case float32:
+		return &arrow.Float32Type{}, nil
+	case float64:
+		return &arrow.Float64Type{}, nil
+	case int8:
+		return &arrow.Int8Type{}, nil
+	case int16:
+		return &arrow.Int16Type{}, nil
+	case int32:
+		return &arrow.Int32Type{}, nil
+	case int64:
+		return &arrow.Int64Type{}, nil
+	case uint8:
+		return &arrow.Uint8Type{}, nil
+	case uint16:
+		return &arrow.Uint16Type{}, nil
+	case uint32:
+		return &arrow.Uint32Type{}, nil
+	case uint64:
+		return &arrow.Uint64Type{}, nil
+	case string:
+		return &arrow.StringType{}, nil
+	case time.Time:
+		return &arrow.Time64Type{Unit: arrow.Nanosecond}, nil
+	}
+	return nil, fmt.Errorf("type %T: %w", value, ErrNotSupported)
+}
+
+func setFieldValue(builder array.Builder, arg interface{}) error {
+	switch b := builder.(type) {
+	case *array.BooleanBuilder:
+		switch v := arg.(type) {

Review Comment:
   if you're not going to support *all* value types, then you should at least add a Godoc comment stating that only the types implemented are supported for parameter values.
   
   > Why doesn't arrow provide a helper for doing this? ;-)
   
   It's honestly not a common use case to have an `interface{}` and needing to append it to a `Builder` that isn't already typed. :smile: In the cases where it happens you typically already *know* that the types match so if you type switch on the builder, you can assume the value type or vice versa. The closest case to this would be the way that the `CSV` reader works by having parsing function pointers that it can call which will properly type assert.
   
   As long as what is supported and what is not is explicitly documented for consumers, I'm okay with that.



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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1141849353


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()

Review Comment:
   Sorry that I'm still not getting it. There are 2 paths in `setParameters()` in my understanding:
   
   1. No schema is provided by the server. In this case, the schema is produced from the input data (and it's types) using `toArrowDataType` ([line 267 et seq.](https://github.com/apache/arrow/blob/5029a00865111f865c28c7a5d4f673229505ba62/go/arrow/flight/flightsql/driver/driver.go#L267)). I don't see how this can lead to a panic as the schema _has to_ match the input parameters. It is produced this way and both `toArrowDataType()` as well as `setFieldValue()` (used to fill in the value to the record)  do not panic but handle error cases...
   2.  A schema is provided by the server. In this case I do use `setFieldValue()` to fill the schema. Here the `array.Builder` of each field _from the schema_ is used and the parameter data-type is _checked_ before appending the values... None of these parts can panic in my view as `.Append()` is strongly typed. I also double-checked that all switch statements do have `default`s so you _will_ get an error if there is a mismatch.
   
   IMO `setParameters` cannot panic. Can you please outline an example that would lead to a panic in your opinion so I see the issue? 
   
   Regarding the test, do you expect me to copy (and modify) that part of the ADBC driver? If so I can take a look...



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

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

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


[GitHub] [arrow] github-actions[bot] commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   * Closes: #34332


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

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

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


[GitHub] [arrow] zeroshade commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @srebhan 
   
   > Unification of backends: I think I should not deal with backend-specific settings at the driver level. So my idea is to unify those and let the user do the specifics on the DSN level. I hope to finish this next Friday...
   
   Agreed, unifying the backends would be good and is kind of the point of using flightsql in general i think.
   
   > TLS implementation/customization: My plan is to implement a mechanism similar to the [MySQL driver](https://github.com/go-sql-driver/mysql) to have a driver-global TLS-config registry. I thought about doing this in a separate PR once this one is merged. What do you think?
   
   I haven't looked at that too hard but I think it's fine to do it as a separate PR once this is merged.


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

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

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


[GitHub] [arrow] jacobmarble commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   > @srebhan from the arrow side it's more of a maintenance question. it's hard to justify maintaining two separate implementations. The ADBC FlightSQL driver is basically a thin wrapper around the FlightSQL client already and the `adbc/sqldriver` package shouldn't be adding much in the way of overhead (honestly I'd be surprised if any overhead it adds dwarfs the actual network IO costs....). At a minimum we should document the use case, either through your work here or by documenting how to leverage ADBC's flight sql driver.
   > 
   > Just as a thought, would you be able to create a benchmark that uses `database/sql` against IOx that we can run with both of your solution here and the ADBC driver so we can compare and see if there's a significant difference (thus justifying the separate implementation)?
   
   I'm not sure performance is the concern (@srebhan please correct or confirm). IMO, this driver is valuable because:
   1. [one conventional golang import statement](https://go.dev/doc/database/open-handle#database_driver)
   
   ```golang
   import _ "github.com/apache/arrow/go/v12/arrow/flight/flightsql"
   ...
   sql.Open("flightsql", dsn)
   ```
   vs
   ```golang
   import "github.com/apache/arrow-adbc/go/adbc/driver/flightsql"
   import "github.com/apache/arrow-adbc/go/adbc/sqldriver"
   sql.Register("flightsql", sqldriver.Driver{flightsql.Driver{}})
   ...
   sql.Open("flightsql", dsn)
   ```
   
   2. easy to discover by googling `FlightSQL golang database/sql`
   3. fewer dependencies
   
   All of the above are satisfied by this PR, and not by the ADBC wrapper.
   
   ---
   
   On the other hand, this PR means the Go/Arrow community would maintain two FlightSQL `database/sql` drivers, [which has been discussed before](https://github.com/apache/arrow-adbc/issues/315#issuecomment-1372695586).
   
   Here's an alternative that satisfies 1 and 2, but not 3 (which hardly matters these days), and retains a the implementation:
   
   New module (including go.mod) containing only:
   ```golang
   import (
   	"github.com/apache/arrow-adbc/go/adbc/driver/flightsql"
   	"github.com/apache/arrow-adbc/go/adbc/sqldriver"
   )
   
   func init() {
   	sql.Register("flightsql", sqldriver.Driver{
   		Driver: flightsql.Driver{},
   	})
   }
   ```
   
   The module could live as `github.com/apache/arrow/go/v12/arrow/flight/flightsql/sqldriver`, or it could live in the ADBC repo and get a vanity name like `go.arrow.apache.org/flightsql/sqldriver`.


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

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

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


[GitHub] [arrow] srebhan commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   Thanks @zeroshade, I missed that. Will add the headers...


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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1141850167


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}
+
+func fromArrowType(arr arrow.Array, idx int) (interface{}, error) {
+	switch c := arr.(type) {
+	case *array.Boolean:
+		return c.Value(idx), nil
+	case *array.Float16:
+		return float64(c.Value(idx).Float32()), nil
+	case *array.Float32:
+		return float64(c.Value(idx)), nil
+	case *array.Float64:
+		return c.Value(idx), nil
+	case *array.Int8:
+		return int64(c.Value(idx)), nil
+	case *array.Int16:
+		return int64(c.Value(idx)), nil
+	case *array.Int32:
+		return int64(c.Value(idx)), nil
+	case *array.Int64:
+		return c.Value(idx), nil
+	case *array.String:
+		return c.Value(idx), nil
+	case *array.Time32:
+		dt, ok := arr.DataType().(*arrow.Time32Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time32", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Time64:
+		dt, ok := arr.DataType().(*arrow.Time64Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time64", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Timestamp:
+		dt, ok := arr.DataType().(*arrow.TimestampType)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching timestamp", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	}
+
+	return nil, fmt.Errorf("type %T: %w", arr, ErrNotSupported)
+}
+
+func toArrowDataType(value interface{}) (arrow.DataType, error) {
+	switch value.(type) {
+	case bool:
+		return &arrow.BooleanType{}, nil
+	case float32:
+		return &arrow.Float32Type{}, nil
+	case float64:
+		return &arrow.Float64Type{}, nil
+	case int8:
+		return &arrow.Int8Type{}, nil
+	case int16:
+		return &arrow.Int16Type{}, nil
+	case int32:
+		return &arrow.Int32Type{}, nil
+	case int64:
+		return &arrow.Int64Type{}, nil
+	case uint8:
+		return &arrow.Uint8Type{}, nil
+	case uint16:
+		return &arrow.Uint16Type{}, nil
+	case uint32:
+		return &arrow.Uint32Type{}, nil
+	case uint64:
+		return &arrow.Uint64Type{}, nil
+	case string:
+		return &arrow.StringType{}, nil
+	case time.Time:
+		return &arrow.Time64Type{Unit: arrow.Nanosecond}, nil
+	}
+	return nil, fmt.Errorf("type %T: %w", value, ErrNotSupported)
+}
+
+func setFieldValue(builder array.Builder, arg interface{}) error {
+	switch b := builder.(type) {
+	case *array.BooleanBuilder:
+		switch v := arg.(type) {
+		case bool:
+			b.Append(v)
+		case []bool:
+			b.AppendValues(v, nil)

Review Comment:
   Understood. Will fix.



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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1132239515


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}
+
+func fromArrowType(arr arrow.Array, idx int) (interface{}, error) {
+	switch c := arr.(type) {
+	case *array.Boolean:
+		return c.Value(idx), nil
+	case *array.Float16:
+		return float64(c.Value(idx).Float32()), nil
+	case *array.Float32:
+		return float64(c.Value(idx)), nil
+	case *array.Float64:
+		return c.Value(idx), nil
+	case *array.Int8:
+		return int64(c.Value(idx)), nil
+	case *array.Int16:
+		return int64(c.Value(idx)), nil
+	case *array.Int32:
+		return int64(c.Value(idx)), nil
+	case *array.Int64:
+		return c.Value(idx), nil
+	case *array.String:
+		return c.Value(idx), nil
+	case *array.Time32:
+		dt, ok := arr.DataType().(*arrow.Time32Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time32", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Time64:
+		dt, ok := arr.DataType().(*arrow.Time64Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time64", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Timestamp:
+		dt, ok := arr.DataType().(*arrow.TimestampType)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching timestamp", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	}
+
+	return nil, fmt.Errorf("type %T: %w", arr, ErrNotSupported)
+}
+
+func toArrowDataType(value interface{}) (arrow.DataType, error) {
+	switch value.(type) {
+	case bool:
+		return &arrow.BooleanType{}, nil
+	case float32:
+		return &arrow.Float32Type{}, nil
+	case float64:
+		return &arrow.Float64Type{}, nil
+	case int8:
+		return &arrow.Int8Type{}, nil
+	case int16:
+		return &arrow.Int16Type{}, nil
+	case int32:
+		return &arrow.Int32Type{}, nil
+	case int64:
+		return &arrow.Int64Type{}, nil
+	case uint8:
+		return &arrow.Uint8Type{}, nil
+	case uint16:
+		return &arrow.Uint16Type{}, nil
+	case uint32:
+		return &arrow.Uint32Type{}, nil
+	case uint64:
+		return &arrow.Uint64Type{}, nil
+	case string:
+		return &arrow.StringType{}, nil
+	case time.Time:
+		return &arrow.Time64Type{Unit: arrow.Nanosecond}, nil
+	}
+	return nil, fmt.Errorf("type %T: %w", value, ErrNotSupported)
+}
+
+func setFieldValue(builder array.Builder, arg interface{}) error {
+	switch b := builder.(type) {
+	case *array.BooleanBuilder:
+		switch v := arg.(type) {

Review Comment:
   Is this a requirement from the start? Do I need to support _all_ builders? Why doesn't arrow provide a helper for doing this? ;-)



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

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

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


[GitHub] [arrow] srebhan commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @jorisvandenbossche and @zeroshade sorry for this taking so long but I was busy with other stuff. Regarding the requested changes I think I addressed all review comments. @zeroshade can you please give this another review?


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

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

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


[GitHub] [arrow] github-actions[bot] commented on pull request #34331: [Go][FlightRPC] Add driver for `database/sql` framework

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

   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/main/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on GitHub? https://github.com/apache/arrow/issues/new/choose
   
   Opening GitHub issues ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename the pull request title in the following format?
   
       GH-${GITHUB_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   In the case of PARQUET issues on JIRA the title also supports:
   
       PARQUET-${JIRA_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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

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

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


[GitHub] [arrow] zeroshade commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()

Review Comment:
   For my initial comment, I was referring to the fact that when you attempt to create a new record, if the schema you pass doesn't match the types of the slice of `arrow.Array` you use, then `NewRecord` will panic. So if the conversions you were doing in `toArrowDataType` don't match up with the schema returned from `ParameterSchema` you'd end up getting a panic when you try to create the record due to the mismatch. that's all.
   
   > Regarding creating of a test-server, this is way beyond my current capabilities as it involves parsing the SQL statement itself for adding it to the SQLite mock or writing a complete mockup FlightSQL server...
   
   You don't need a *complete* mockup flightsql server, you only need to implement the necessary methods that would return the ParameterSchema and you can hardcode the schema it returns. For example take a look at the mocked out server used for the ADBC timeout tests: https://github.com/apache/arrow-adbc/blob/main/go/adbc/driver/flightsql/flightsql_adbc_test.go#L709 
   
   it's pretty straightforward I think.



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

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

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


[GitHub] [arrow] zeroshade commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()

Review Comment:
   okay, it can be postponed.



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

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

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


[GitHub] [arrow] zeroshade commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @srebhan I think your plan is fine for going ahead. Once you get the CI tests passing again I'll give this another look over.


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

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

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


[GitHub] [arrow] zeroshade commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})

Review Comment:
   Looks like the artificial naming is still here and should get removed.



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

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

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


[GitHub] [arrow] zeroshade commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}

Review Comment:
   this isn't necessary, all the records in a single stream are currently guaranteed to have the same schema. You should move this above the for loop and just use `reader.Schema()`



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {

Review Comment:
   You're incorrectly mixing `Read` and `Next`.
   
   You should either do 
   
   ```go
   for reader.Next() {
       record := reader.Record()
       record.Retain()
       // do something
   }
   
   if reader.Err() != nil {
       // do something with err
   }
   ```
   
   Or remove the `Next` call at the end of the loop. Calling `reader.Read` advances to the next record on its own.



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}
+
+func fromArrowType(arr arrow.Array, idx int) (interface{}, error) {
+	switch c := arr.(type) {
+	case *array.Boolean:
+		return c.Value(idx), nil
+	case *array.Float16:
+		return float64(c.Value(idx).Float32()), nil
+	case *array.Float32:
+		return float64(c.Value(idx)), nil
+	case *array.Float64:
+		return c.Value(idx), nil
+	case *array.Int8:
+		return int64(c.Value(idx)), nil
+	case *array.Int16:
+		return int64(c.Value(idx)), nil
+	case *array.Int32:
+		return int64(c.Value(idx)), nil
+	case *array.Int64:
+		return c.Value(idx), nil
+	case *array.String:
+		return c.Value(idx), nil
+	case *array.Time32:
+		dt, ok := arr.DataType().(*arrow.Time32Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time32", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Time64:
+		dt, ok := arr.DataType().(*arrow.Time64Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time64", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Timestamp:
+		dt, ok := arr.DataType().(*arrow.TimestampType)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching timestamp", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	}
+
+	return nil, fmt.Errorf("type %T: %w", arr, ErrNotSupported)
+}
+
+func toArrowDataType(value interface{}) (arrow.DataType, error) {
+	switch value.(type) {
+	case bool:
+		return &arrow.BooleanType{}, nil
+	case float32:
+		return &arrow.Float32Type{}, nil
+	case float64:
+		return &arrow.Float64Type{}, nil
+	case int8:
+		return &arrow.Int8Type{}, nil
+	case int16:
+		return &arrow.Int16Type{}, nil
+	case int32:
+		return &arrow.Int32Type{}, nil
+	case int64:
+		return &arrow.Int64Type{}, nil
+	case uint8:
+		return &arrow.Uint8Type{}, nil
+	case uint16:
+		return &arrow.Uint16Type{}, nil
+	case uint32:
+		return &arrow.Uint32Type{}, nil
+	case uint64:
+		return &arrow.Uint64Type{}, nil
+	case string:
+		return &arrow.StringType{}, nil
+	case time.Time:
+		return &arrow.Time64Type{Unit: arrow.Nanosecond}, nil
+	}
+	return nil, fmt.Errorf("type %T: %w", value, ErrNotSupported)
+}
+
+func setFieldValue(builder array.Builder, arg interface{}) error {
+	switch b := builder.(type) {
+	case *array.BooleanBuilder:
+		switch v := arg.(type) {
+		case bool:
+			b.Append(v)
+		case []bool:
+			b.AppendValues(v, nil)

Review Comment:
   shouldn't this case be handled by a list builder instead?



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()

Review Comment:
   Instead of using `memory.DefaultAllocator` we should allow specifying the allocator by embedding it into a context or by some other way.



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}
+
+func fromArrowType(arr arrow.Array, idx int) (interface{}, error) {
+	switch c := arr.(type) {
+	case *array.Boolean:
+		return c.Value(idx), nil
+	case *array.Float16:
+		return float64(c.Value(idx).Float32()), nil
+	case *array.Float32:
+		return float64(c.Value(idx)), nil
+	case *array.Float64:
+		return c.Value(idx), nil
+	case *array.Int8:
+		return int64(c.Value(idx)), nil
+	case *array.Int16:
+		return int64(c.Value(idx)), nil
+	case *array.Int32:
+		return int64(c.Value(idx)), nil
+	case *array.Int64:
+		return c.Value(idx), nil
+	case *array.String:
+		return c.Value(idx), nil
+	case *array.Time32:
+		dt, ok := arr.DataType().(*arrow.Time32Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time32", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Time64:
+		dt, ok := arr.DataType().(*arrow.Time64Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time64", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Timestamp:
+		dt, ok := arr.DataType().(*arrow.TimestampType)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching timestamp", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	}
+
+	return nil, fmt.Errorf("type %T: %w", arr, ErrNotSupported)
+}
+
+func toArrowDataType(value interface{}) (arrow.DataType, error) {
+	switch value.(type) {
+	case bool:
+		return &arrow.BooleanType{}, nil
+	case float32:
+		return &arrow.Float32Type{}, nil
+	case float64:
+		return &arrow.Float64Type{}, nil
+	case int8:
+		return &arrow.Int8Type{}, nil
+	case int16:
+		return &arrow.Int16Type{}, nil
+	case int32:
+		return &arrow.Int32Type{}, nil
+	case int64:
+		return &arrow.Int64Type{}, nil
+	case uint8:
+		return &arrow.Uint8Type{}, nil
+	case uint16:
+		return &arrow.Uint16Type{}, nil
+	case uint32:
+		return &arrow.Uint32Type{}, nil
+	case uint64:
+		return &arrow.Uint64Type{}, nil
+	case string:
+		return &arrow.StringType{}, nil
+	case time.Time:
+		return &arrow.Time64Type{Unit: arrow.Nanosecond}, nil
+	}
+	return nil, fmt.Errorf("type %T: %w", value, ErrNotSupported)
+}
+
+func setFieldValue(builder array.Builder, arg interface{}) error {
+	switch b := builder.(type) {
+	case *array.BooleanBuilder:
+		switch v := arg.(type) {

Review Comment:
   you also need to handle a union builder



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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1132231999


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()

Review Comment:
   Looking at that code again, I'm using the schema if given and otherwise create one from the given values. I changed the code a bit to make this more clear, but I don't see what you mean in your initial comment. Can you elaborate a bit @zeroshade?
   
   Regarding creating of a test-server, this is way beyond my current capabilities as it involves parsing the SQL statement itself for adding it to the SQLite mock _or_ writing a complete mockup FlightSQL server...



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

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

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


[GitHub] [arrow] ursabot commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   Benchmark runs are scheduled for baseline = d21c1c79c0d6377d035bd90da5d5d09e04e49079 and contender = c40e658fbbd6201132c4378eb0fefb746ff5915f. c40e658fbbd6201132c4378eb0fefb746ff5915f is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/80d882993b574bcf898055913a169a6e...1e59dde2091b4b3680241f7fa52d2697/)
   [Failed] [test-mac-arm](https://conbench.ursa.dev/compare/runs/de71aa35c7e640bfab4f9519c86df0c0...97f7819f1d4f421b8f080fa29457af08/)
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/dd74534ccd614c0eaebfe87bc8232d87...f0d9254903584b2ebd202fd8b70bdb12/)
   [Finished :arrow_down:0.12% :arrow_up:0.0%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/3de9ac2e2cf942d99b407bf55a412691...4299173c5460449fb6e8fb76b2981b53/)
   Buildkite builds:
   [Finished] [`c40e658f` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2678)
   [Failed] [`c40e658f` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2711)
   [Finished] [`c40e658f` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2677)
   [Finished] [`c40e658f` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2704)
   [Finished] [`d21c1c79` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2677)
   [Failed] [`d21c1c79` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2710)
   [Finished] [`d21c1c79` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2676)
   [Finished] [`d21c1c79` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2703)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1162478621


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()

Review Comment:
   Added unit-tests do show that you don't get a panic if you use the wrong type or wrong number of params. IMO the only way to panic is if the server does not check the parameters and panics there, but this is the server's fault, isn't it? Can you construct a concrete test-case @zeroshade that panics in the driver? 



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

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

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


[GitHub] [arrow] jacobmarble commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   Here's my alternative proposal:
   https://github.com/apache/arrow-adbc/pull/480
   
   As documented in the PR example, use is:
   ```golang
   import (
   	"database/sql"
   	_ "github.com/apache/arrow-adbc/go/adbc/sqldriver/flightsql"
   )
   
   func Example() {
   	db, err := sql.Open("flightsql", "uri=grpc://localhost:12345")
   	if err != nil {
   		panic(err)
   	}
   
   	_ = db.Ping()
   }
   ```
   
   I still find `github.com/apache/arrow-adbc/go/adbc/sqldriver/flightsql` to be very long-winded, but if documentation consistently points to it, then I think we'll help folks out.


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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1124299303


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)

Review Comment:
   Thanks for the advice and the example. Fixed.



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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1124270433


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})

Review Comment:
   You are right. I was under the impression that the `Name` is required (and has to be unique) which is not the case. Remove artificial naming altogether.



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})

Review Comment:
   You are right. I was under the impression that the `Name` is required (and has to be unique) which is not the case. Removed artificial naming altogether.



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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1124305678


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}
+
+func fromArrowType(arr arrow.Array, idx int) (any, error) {
+	switch c := arr.(type) {
+	case *array.Boolean:
+		return c.Value(idx), nil
+	case *array.Float16:
+		return float64(c.Value(idx).Float32()), nil
+	case *array.Float32:
+		return float64(c.Value(idx)), nil
+	case *array.Float64:
+		return c.Value(idx), nil
+	case *array.Int8:
+		return int64(c.Value(idx)), nil
+	case *array.Int16:
+		return int64(c.Value(idx)), nil
+	case *array.Int32:
+		return int64(c.Value(idx)), nil
+	case *array.Int64:
+		return c.Value(idx), nil
+	case *array.String:
+		return c.Value(idx), nil
+	case *array.Time32:
+		dt, ok := arr.DataType().(*arrow.Time32Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time32", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Time64:
+		dt, ok := arr.DataType().(*arrow.Time64Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time64", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Timestamp:
+		dt, ok := arr.DataType().(*arrow.TimestampType)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching timestamp", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	}
+
+	return nil, fmt.Errorf("type %T: %w", arr, ErrNotSupported)
+}
+
+func toArrowDataType(value any) (arrow.DataType, error) {
+	switch value.(type) {
+	case bool:
+		return &arrow.BooleanType{}, nil
+	case float32:
+		return &arrow.Float32Type{}, nil
+	case float64:
+		return &arrow.Float64Type{}, nil
+	case int8:
+		return &arrow.Int8Type{}, nil
+	case int16:
+		return &arrow.Int16Type{}, nil
+	case int32:
+		return &arrow.Int32Type{}, nil
+	case int64:
+		return &arrow.Int64Type{}, nil
+	case uint8:
+		return &arrow.Uint8Type{}, nil
+	case uint16:
+		return &arrow.Uint16Type{}, nil
+	case uint32:
+		return &arrow.Uint32Type{}, nil
+	case uint64:
+		return &arrow.Uint64Type{}, nil
+	case string:
+		return &arrow.StringType{}, nil
+	case time.Time:
+		return &arrow.Time64Type{Unit: arrow.Nanosecond}, nil
+	}
+	return nil, fmt.Errorf("type %T: %w", value, ErrNotSupported)
+}
+
+func setFieldValue(builder array.Builder, arg any) error {
+	switch b := builder.(type) {
+	case *array.BooleanBuilder:
+		switch v := arg.(type) {
+		case bool:
+			b.Append(v)
+		case []bool:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Float32Builder:
+		switch v := arg.(type) {
+		case float32:
+			b.Append(v)
+		case []float32:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Float64Builder:
+		switch v := arg.(type) {
+		case float64:
+			b.Append(v)
+		case []float64:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Int8Builder:
+		switch v := arg.(type) {
+		case int8:
+			b.Append(v)
+		case []int8:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Int16Builder:
+		switch v := arg.(type) {
+		case int16:
+			b.Append(v)
+		case []int16:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Int32Builder:
+		switch v := arg.(type) {
+		case int32:
+			b.Append(v)
+		case []int32:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Int64Builder:
+		switch v := arg.(type) {
+		case int64:
+			b.Append(v)
+		case []int64:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Uint8Builder:
+		switch v := arg.(type) {
+		case uint8:
+			b.Append(v)
+		case []uint8:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Uint16Builder:
+		switch v := arg.(type) {
+		case uint16:
+			b.Append(v)
+		case []uint16:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Uint32Builder:
+		switch v := arg.(type) {
+		case uint32:
+			b.Append(v)
+		case []uint32:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Uint64Builder:
+		switch v := arg.(type) {
+		case uint64:
+			b.Append(v)
+		case []uint64:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.StringBuilder:
+		switch v := arg.(type) {
+		case string:
+			b.Append(v)
+		case []string:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Time64Builder:
+		switch v := arg.(type) {
+		case int64:
+			b.Append(arrow.Time64(v))
+		case []int64:
+			for _, x := range v {
+				b.Append(arrow.Time64(x))
+			}
+		case uint64:
+			b.Append(arrow.Time64(v))
+		case []uint64:
+			for _, x := range v {
+				b.Append(arrow.Time64(x))
+			}
+		case time.Time:
+			b.Append(arrow.Time64(v.Nanosecond()))
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	}
+	return nil
+}
+
+type DriverConfig struct {
+	Backend  string
+	Address  string
+	Username string
+	Password string
+	Database string
+	Token    string
+	Timeout  time.Duration
+}

Review Comment:
   My plan was to extend the driver in a second PR and add TLS customization. Do you prefer to do it in this PR?



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

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

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


[GitHub] [arrow] zeroshade commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})

Review Comment:
   Looks like the artificial naming is still here and should get removed.



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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1132233429


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,779 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, nil
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	return &Result{affected: n, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+
+		for {
+			record, err := reader.Read()
+			if err != nil {
+				if errors.Is(err, io.EOF) {
+					break
+				}
+				return nil, fmt.Errorf("reading record failed: %w", err)
+			}
+			record.Retain()
+
+			// Check the schemata
+			if rows.schema == nil {
+				rows.schema = record.Schema()
+			}
+			if !rows.schema.Equal(record.Schema()) {
+				return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+			}
+			rows.records = append(rows.records, record)
+
+			reader.Next()
+		}
+		if err := reader.Err(); err != nil {
+			return &rows, err
+		}
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()

Review Comment:
   Can we please also postpone to the point where this is requested by a user? I guess this is not a feature a user of `database/sql` is expecting from the start... :-)



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

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

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


[GitHub] [arrow] zeroshade commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @srebhan If you look at the log:
   
   ```
   apache-rat license violation: go/arrow/flight/flightsql/driver/README.md
   apache-rat license violation: go/arrow/flight/flightsql/driver/config.go
   apache-rat license violation: go/arrow/flight/flightsql/driver/utils.go
   ```
   
   It's complaining that you're missing the apache license from the files you added. Please add the apache license at the top of the files


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

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

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


[GitHub] [arrow] zeroshade merged pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


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

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

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


[GitHub] [arrow] zeroshade commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @srebhan this LGTM for the moment, can you just create issues (referencing this one) for each of the tasks we identified as something that will be done in a followup PR? so that we don't lose track of that work?
   
   I'm good to merge this once those issues are created, potentially creating a top-level umbrella issue with a task list of the separate issues?


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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1124279442


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})

Review Comment:
   There is a subtle difference in the functions, `Query` and `Exec` take a `[]driver.Value` argument while the context variants take a `[]driver.NamedValue`. The difference is that the former is a list of values **that do not have a name**, while the latter are actually named. Therefore, I tried to invent names under the impression that they are required. This doesn't seem to be the case so I removed the artificial naming.



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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1124302042


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()

Review Comment:
   Currently I have nothing to test this against as the example server will not return a parameter schema. Do you have a way to check my (to be written) code will handle the schema correctly?



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

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

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


[GitHub] [arrow] srebhan commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   > The proposed module would mean that the user just imports a single module that re-exports the interface, without explicitly worrying about ADBC.
   
   @lidavidm that's exactly my point. I my view, `database/sql` (think of it as "Golang Database connectivity") and `ADBC` are two independent standards. So **if I do not worry about ADBC** why am I forces to import it? People probably want to use the one or the other. So I do not suggest to have two implementations, I suggest to remove the ADBC wrapper and make it a wrapper around FlightSQL as this is what is done for every other database in Golang! 


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

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

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


[GitHub] [arrow] github-actions[bot] commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   :warning: GitHub issue #34332 **has been automatically assigned in GitHub** to PR creator.


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

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

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


[GitHub] [arrow] zeroshade commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @srebhan from the arrow side it's more of a maintenance question. it's hard to justify maintaining two separate implementations. The ADBC FlightSQL driver is basically a thin wrapper around the FlightSQL client already and the `adbc/sqldriver` package shouldn't be adding much in the way of overhead (honestly I'd be surprised if any overhead it adds dwarfs the actual network IO costs....). At a minimum we should document the use case, either through your work here or by documenting how to leverage ADBC's flight sql driver. 
   
   Just as a thought, would you be able to create a benchmark that uses `database/sql` against IOx that we can run with both of your solution here and the ADBC driver so we can compare and see if there's a significant difference (thus justifying the separate implementation)?


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

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

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


[GitHub] [arrow] srebhan commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @zeroshade I pushed an update containing fixes to most of your comments above, the unification of the drivers and placement of the driver in a `driver` subdir.
   
   I do have one question to the schema usage (see response to your comment). I'll probably be the wrong person to implement a schema mock, but I'm willing to test if someone implements this.
   
   Regarding the other open points I suggest to first get this PR ready and merged. I'm then willing to work on the following items as follow-up PRs:
   1. TLS configuration
   2. parameterization of the memory allocator (if we really want to do this without need)
   3. mock schema generation for SQLite example (not sure if I can find the time for this)
   
   What do you think? 


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

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

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


[GitHub] [arrow] zeroshade commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @srebhan Would it make more sense to add IOx backend testing to the CI for adbc/sqldriver (as pointed to by @lidavidm) along with any necessary changes there?
   
   At the moment we're trying to keep the arrow module/package compatible with go1.17+ so at a minimum you'd need to add `go:build` constraints to not break the CI here. But it would probably be better to consolidate efforts in the ADBC sqldriver (which would provide a `database/sql` wrapper for *anything* that implements ADBC including the FlightSQL driver) rather than duplicating a large part of that work here.
   
   What do you think?


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

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

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


[GitHub] [arrow] zeroshade commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported

Review Comment:
   shouldn't this be `r.affected, nil`?



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})

Review Comment:
   what if the values are named differently in the query? Depending on the underlying sql dialect `arg_0` may not be correct, it may want just `0` or some other format?



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n

Review Comment:
   This is incorrect, the returned value is whatever the *server* replies with. We should be placing `n` as `affected` here and if it's "always returning one" then that's an issue on the server side you were testing with and not the driver/client.



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()

Review Comment:
   if your `toArrowDataType` auto-conversion checks don't match the types in the ParameterSchema, you could end up with a panic or other failure. If `s.stmt.ParameterSchema()` is non-nil, you should use that for converting the arguments rather than inferring the data types.



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)

Review Comment:
   This is only grabbing the first record, you need to either do:
   
   ```go
   for reader.Next() {
       record := reader.Record()
       record.Retain()
       rows.records = append(rows.records, record)
   }
   if reader.Err() {
       // do something with error
   }
   ```
   
   or you need to call `reader.Read()` repeatedly until you get a non-nil `err`. `io.EOF` indicates no more records, and anything else would be an error it reports.



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})

Review Comment:
   same comment as above. Also, why do we rename the arguments for `Query` and for `Exec` but we don't rename the arguments for `ExecContext` and `QueryContext`? what happens if a user calls one of those directly?



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}

Review Comment:
   storing the transaction at the driver level means that this will break if there are multiple connections with their own transactions. You need a separate connection object and to store the transaction at the connection level instead, not the driver level.



##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}
+
+func fromArrowType(arr arrow.Array, idx int) (any, error) {
+	switch c := arr.(type) {
+	case *array.Boolean:
+		return c.Value(idx), nil
+	case *array.Float16:
+		return float64(c.Value(idx).Float32()), nil
+	case *array.Float32:
+		return float64(c.Value(idx)), nil
+	case *array.Float64:
+		return c.Value(idx), nil
+	case *array.Int8:
+		return int64(c.Value(idx)), nil
+	case *array.Int16:
+		return int64(c.Value(idx)), nil
+	case *array.Int32:
+		return int64(c.Value(idx)), nil
+	case *array.Int64:
+		return c.Value(idx), nil
+	case *array.String:
+		return c.Value(idx), nil
+	case *array.Time32:
+		dt, ok := arr.DataType().(*arrow.Time32Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time32", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Time64:
+		dt, ok := arr.DataType().(*arrow.Time64Type)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching time64", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	case *array.Timestamp:
+		dt, ok := arr.DataType().(*arrow.TimestampType)
+		if !ok {
+			return nil, fmt.Errorf("datatype %T not matching timestamp", arr.DataType())
+		}
+		v := c.Value(idx)
+		return v.ToTime(dt.TimeUnit()), nil
+	}
+
+	return nil, fmt.Errorf("type %T: %w", arr, ErrNotSupported)
+}
+
+func toArrowDataType(value any) (arrow.DataType, error) {
+	switch value.(type) {
+	case bool:
+		return &arrow.BooleanType{}, nil
+	case float32:
+		return &arrow.Float32Type{}, nil
+	case float64:
+		return &arrow.Float64Type{}, nil
+	case int8:
+		return &arrow.Int8Type{}, nil
+	case int16:
+		return &arrow.Int16Type{}, nil
+	case int32:
+		return &arrow.Int32Type{}, nil
+	case int64:
+		return &arrow.Int64Type{}, nil
+	case uint8:
+		return &arrow.Uint8Type{}, nil
+	case uint16:
+		return &arrow.Uint16Type{}, nil
+	case uint32:
+		return &arrow.Uint32Type{}, nil
+	case uint64:
+		return &arrow.Uint64Type{}, nil
+	case string:
+		return &arrow.StringType{}, nil
+	case time.Time:
+		return &arrow.Time64Type{Unit: arrow.Nanosecond}, nil
+	}
+	return nil, fmt.Errorf("type %T: %w", value, ErrNotSupported)
+}
+
+func setFieldValue(builder array.Builder, arg any) error {
+	switch b := builder.(type) {
+	case *array.BooleanBuilder:
+		switch v := arg.(type) {
+		case bool:
+			b.Append(v)
+		case []bool:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Float32Builder:
+		switch v := arg.(type) {
+		case float32:
+			b.Append(v)
+		case []float32:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Float64Builder:
+		switch v := arg.(type) {
+		case float64:
+			b.Append(v)
+		case []float64:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Int8Builder:
+		switch v := arg.(type) {
+		case int8:
+			b.Append(v)
+		case []int8:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Int16Builder:
+		switch v := arg.(type) {
+		case int16:
+			b.Append(v)
+		case []int16:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Int32Builder:
+		switch v := arg.(type) {
+		case int32:
+			b.Append(v)
+		case []int32:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Int64Builder:
+		switch v := arg.(type) {
+		case int64:
+			b.Append(v)
+		case []int64:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Uint8Builder:
+		switch v := arg.(type) {
+		case uint8:
+			b.Append(v)
+		case []uint8:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Uint16Builder:
+		switch v := arg.(type) {
+		case uint16:
+			b.Append(v)
+		case []uint16:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Uint32Builder:
+		switch v := arg.(type) {
+		case uint32:
+			b.Append(v)
+		case []uint32:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Uint64Builder:
+		switch v := arg.(type) {
+		case uint64:
+			b.Append(v)
+		case []uint64:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.StringBuilder:
+		switch v := arg.(type) {
+		case string:
+			b.Append(v)
+		case []string:
+			b.AppendValues(v, nil)
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	case *array.Time64Builder:
+		switch v := arg.(type) {
+		case int64:
+			b.Append(arrow.Time64(v))
+		case []int64:
+			for _, x := range v {
+				b.Append(arrow.Time64(x))
+			}
+		case uint64:
+			b.Append(arrow.Time64(v))
+		case []uint64:
+			for _, x := range v {
+				b.Append(arrow.Time64(x))
+			}
+		case time.Time:
+			b.Append(arrow.Time64(v.Nanosecond()))
+		default:
+			return fmt.Errorf("invalid value type %T for builder %T", arg, builder)
+		}
+	}
+	return nil
+}
+
+type DriverConfig struct {
+	Backend  string
+	Address  string
+	Username string
+	Password string
+	Database string
+	Token    string
+	Timeout  time.Duration
+}

Review Comment:
   Also need to add configuration options for TLS and such.



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

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

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


[GitHub] [arrow] zeroshade commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   There is possibly some value in having the separate Flight SQL <--> `database/sql` here, as far as contributing goes, though I'd prefer more reasoning than just an "in principle" argument against the extra layers. That said, it's likely not *too* much maintenance overhead to maintain this implementation assuming we get it right.
   
   @srebhan If we can get this to a better spot with good testing, I'd be okay with getting this merged. But I do want to make it clear that the documentation / examples will likely still be pointing at using the ADBC implementation.
   
   I've already left a bunch of comments above that need to be addressed (above and beyond the failing CI). In addition, I'd prefer if this were put into it's own subdirectory of the `flightsql` directory rather than directly in the `flightsql` package itself.


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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1124268359


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported

Review Comment:
   You are right. Fixed.



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

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

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


[GitHub] [arrow] jacobmarble commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   Whether the technical merits of the "direct driver" are truly meaningful, I support it for other reasons.
   
   ---
   
   > it's likely not too much maintenance overhead to maintain this implementation assuming we get it right.
   
   > If we can get this to a better spot with good testing, I'd be okay with getting this merged. But I do want to make it clear that the documentation / examples will likely still be pointing at using the ADBC implementation.
   
   IMHO, the existence of a "direct driver" is part of "getting it right". Eventually, someone will create such a driver, and it's better that it be a canonical driver live in this repository, rather than `github.com/StartCo/goflightsql`. (I'm thinking about the various Kafka golang clients. Confluent's driver depends on cgo, so it isn't as popular as the "pure Go" drivers published by Segment.io and Shopify.)
   
   Both implementations should behave the same way, as much as possible. Unit tests can help.
   
   > I my view, database/sql (think of it as "Golang Database connectivity") and ADBC are two independent standards. So if I do not worry about ADBC why am I forced to depend on it? People probably want to use the one or the other. So I do not suggest to have two implementations, I suggest to remove the ADBC wrapper and make it a wrapper around FlightSQL as this is what is done for every other database in Golang!
   
   This illustrates a concern I have about the Apache Arrow ecosystem. The relationship between these APIs (Arrow, Flight, FlightSQL, Arrow, database/sql) are not obvious from the outside.
   
   The most common FlightSQL client persona wants a basic SQL driver to access a database that someone else chose. Things like Arrow and ADBC don't matter to this persona, but things like "pure Go" and few dependencies do matter.


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

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

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


[GitHub] [arrow] srebhan commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @jacobmarble it's not only about wrapping the registration, it's about the fundamental addition of (IMO unnecessary) complexity. The ADBC `database/sql` driver is a wrapper around `ADBC` as such it adds another indirection from the database, i.e. `Application -> database/sql -> ADBC -> FlightSQL -> Backend` vs. `Application -> database/sql -> FlightSQL -> Backend`. It's like writing a wrapper around ODBC... What is the benefit, given that `database/sql` is the defacto standard in go? The clear drawback for the user is to deal with ADBC. From the docs it's not yet clear to me which "ADBC drivers" I need. Are they bundled with the driver? If so, how to deal with growing dependencies and code when adding new drivers? What if CGO is not allowed in the application?
   
   I'm certainly willing to work on this PR, but it sounds like there is some resistance to the idea. Please let me know if there is a realistic chance of getting this merged!


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

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

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


[GitHub] [arrow] lidavidm commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   You depend on the Flight SQL driver to use Flight SQL. There is no cgo required. The cgo bits are only if you want to 1) use a non-Go ADBC driver or 2) expose a Go driver to a non-Go application. The proposed module would mean that the user just imports a single module that re-exports the interface, without explicitly worrying about ADBC.
   
   We can add documentation as necessary for searchability (and IMO, having two implementations would make searchability worse, not better). I don't think the answer to "there is no documentation" is "write a second implementation".
   
   Again, the primary concern is maintenance. Are you/InfluxData willing to continue maintaining the implementation here? My other concern is just confusing users by having two implementations that don't functionally differ.
   
   I can see the argument for a separate implementation if there is a specific benefit for users, but I don't think that's been addressed beyond having fewer layers in principle. 
   
   See what was done for Python: instead of maintaining specific Flight SQL<->DBAPI wrappers, we did the same thing where there is just an ADBC<->DBAPI wrapper. It also means we can iterate on those parts separately from/more quickly than the core library.


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

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

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


[GitHub] [arrow] srebhan commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @zeroshade thanks for your comment. I can surely work on making this compatible with Go 1.17 and avoid breaking tests by removing the IOx backend tests (which only supports a subset of the operations anyway). I thought you were running CI tests in "short" mode to strip out integration tests...
   
   Regarding the relation to the ADBC driver: I'm aware of this driver, but I do want the leanest solution possible adhering to the look-and-feel of `database/sql` as close as possible. For _me personally_ there is no value in a wrapper around a wrapper. I even thought of getting rid of the different backend variants by putting in the specific options as parameters to make this a thin wrapper around the FlightSQL client. Does that make sense?


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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1124276170


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n

Review Comment:
   Fixed.



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

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

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


[GitHub] [arrow] zeroshade commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}

Review Comment:
   > From what I read about the Golang code there is only one connection per Driver instance. Golang will then create a pool of drivers which are assigned to an transaction.
   
   That's not quite the case. The `database/sql` package creates one driver instance and maintains a pool of connections. You're guaranteed that a given connection will only be used by one goroutine **at a time**, but you're not guaranteed it won't be used by multiple goroutines in general.
   
   > There is no way to access the "connection" at the BeginTx call... What do you have in mind?
   
   The `Open` function on the Driver should return a connection to the database, not the driver itself. That's where your issue is. `BeginTx` is a method on the `driver.Conn` interface, not `driver.Driver`. You need to fix the implementation to have an actual "connection" type. I highly recommend looking over https://github.com/apache/arrow-adbc/blob/main/go/adbc/sqldriver/driver.go for inspiration.



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

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

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


[GitHub] [arrow] srebhan commented on a diff in pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "srebhan (via GitHub)" <gi...@apache.org>.
srebhan commented on code in PR #34331:
URL: https://github.com/apache/arrow/pull/34331#discussion_r1124304779


##########
go/arrow/flight/flightsql/driver.go:
##########
@@ -0,0 +1,772 @@
+// 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"
+	"database/sql"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"io"
+	"net/url"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/arrow/go/v12/arrow"
+	"github.com/apache/arrow/go/v12/arrow/array"
+	"github.com/apache/arrow/go/v12/arrow/memory"
+
+	"google.golang.org/grpc"
+)
+
+var (
+	ErrNotSupported          = errors.New("not supported")
+	ErrOutOfRange            = errors.New("index out of range")
+	ErrTransactionInProgress = errors.New("transaction still in progress")
+)
+
+type Rows struct {
+	schema        *arrow.Schema
+	records       []arrow.Record
+	currentRecord int
+	currentRow    int
+}
+
+// Columns returns the names of the columns.
+func (r *Rows) Columns() []string {
+	if len(r.records) == 0 {
+		return nil
+	}
+
+	// All records have the same columns
+	var cols []string
+	for _, c := range r.schema.Fields() {
+		cols = append(cols, c.Name)
+	}
+
+	return cols
+}
+
+// Close closes the rows iterator.
+func (r *Rows) Close() error {
+	for _, rec := range r.records {
+		rec.Release()
+	}
+	r.currentRecord = 0
+	r.currentRow = 0
+
+	return nil
+}
+
+// Next is called to populate the next row of data into
+// the provided slice. The provided slice will be the same
+// size as the Columns() are wide.
+//
+// Next should return io.EOF when there are no more rows.
+//
+// The dest should not be written to outside of Next. Care
+// should be taken when closing Rows not to modify
+// a buffer held in dest.
+func (r *Rows) Next(dest []driver.Value) error {
+	if r.currentRecord >= len(r.records) {
+		return io.EOF
+	}
+	record := r.records[r.currentRecord]
+
+	if int64(r.currentRow) >= record.NumRows() {
+		return ErrOutOfRange
+	}
+
+	for i, arr := range record.Columns() {
+		v, err := fromArrowType(arr, r.currentRow)
+		if err != nil {
+			return err
+		}
+		dest[i] = v
+	}
+
+	r.currentRow++
+	if int64(r.currentRow) >= record.NumRows() {
+		r.currentRecord++
+		r.currentRow = 0
+	}
+
+	return nil
+}
+
+type Result struct {
+	affected   int64
+	lastinsert int64
+}
+
+// LastInsertId returns the database's auto-generated ID after, for example,
+// an INSERT into a table with primary key.
+func (r *Result) LastInsertId() (int64, error) {
+	if r.lastinsert < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.lastinsert, nil
+}
+
+// RowsAffected returns the number of rows affected by the query.
+func (r *Result) RowsAffected() (int64, error) {
+	if r.affected < 0 {
+		return -1, ErrNotSupported
+	}
+	return r.affected, ErrNotSupported
+}
+
+type Stmt struct {
+	stmt   *PreparedStatement
+	client *Client
+
+	timeout time.Duration
+}
+
+// Close closes the statement.
+func (s *Stmt) Close() error {
+	ctx := context.Background()
+	if s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	return s.stmt.Close(ctx)
+}
+
+// NumInput returns the number of placeholder parameters.
+func (s *Stmt) NumInput() int {
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		// NumInput may also return -1, if the driver doesn't know its number
+		// of placeholders. In that case, the sql package will not sanity check
+		// Exec or Query argument counts.
+		return -1
+	}
+
+	// If NumInput returns >= 0, the sql package will sanity check argument
+	// counts from callers and return errors to the caller before the
+	// statement's Exec or Query methods are called.
+	return len(schema.Fields())
+}
+
+// Exec executes a query that doesn't return rows, such
+// as an INSERT or UPDATE.
+func (s *Stmt) Exec(args []driver.Value) (driver.Result, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.ExecContext(context.Background(), params)
+}
+
+// ExecContext executes a query that doesn't return rows, such as an INSERT or UPDATE.
+func (s *Stmt) ExecContext(ctx context.Context, args []driver.NamedValue) (driver.Result, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	n, err := s.stmt.ExecuteUpdate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// FIXME: For now we ignore the number of affected records as it seems like
+	// the returned value is always one.
+	_ = n
+
+	return &Result{affected: -1, lastinsert: -1}, nil
+}
+
+// Query executes a query that may return rows, such as a SELECT.
+func (s *Stmt) Query(args []driver.Value) (driver.Rows, error) {
+	var params []driver.NamedValue
+	for i, arg := range args {
+		params = append(params, driver.NamedValue{
+			Name:    fmt.Sprintf("arg_%d", i),
+			Ordinal: i,
+			Value:   arg,
+		})
+	}
+
+	return s.QueryContext(context.Background(), params)
+}
+
+// QueryContext executes a query that may return rows, such as a SELECT.
+func (s *Stmt) QueryContext(ctx context.Context, args []driver.NamedValue) (driver.Rows, error) {
+	if err := s.setParameters(args); err != nil {
+		return nil, err
+	}
+
+	if _, set := ctx.Deadline(); !set && s.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, s.timeout)
+		defer cancel()
+	}
+
+	info, err := s.stmt.Execute(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	rows := Rows{}
+	for _, endpoint := range info.Endpoint {
+		reader, err := s.client.DoGet(ctx, endpoint.GetTicket())
+		if err != nil {
+			return nil, fmt.Errorf("getting ticket failed: %w", err)
+		}
+		record, err := reader.Read()
+		if err != nil {
+			return nil, fmt.Errorf("reading record failed: %w", err)
+		}
+
+		if rows.schema == nil {
+			rows.schema = record.Schema()
+		}
+		if !rows.schema.Equal(record.Schema()) {
+			return nil, fmt.Errorf("mixed schemas %w", ErrNotSupported)
+		}
+		rows.records = append(rows.records, record)
+	}
+
+	return &rows, nil
+}
+
+func (s *Stmt) setParameters(args []driver.NamedValue) error {
+	if len(args) == 0 {
+		s.stmt.SetParameters(nil)
+		return nil
+	}
+
+	var fields []arrow.Field
+	sort.SliceStable(args, func(i, j int) bool {
+		return args[i].Ordinal < args[j].Ordinal
+	})
+
+	for _, arg := range args {
+		dt, err := toArrowDataType(arg.Value)
+		if err != nil {
+			return fmt.Errorf("schema: %w", err)
+		}
+		fields = append(fields, arrow.Field{
+			Name:     arg.Name,
+			Type:     dt,
+			Nullable: true,
+		})
+	}
+
+	schema := s.stmt.ParameterSchema()
+	if schema == nil {
+		schema = arrow.NewSchema(fields, nil)
+	}
+
+	recBuilder := array.NewRecordBuilder(memory.DefaultAllocator, schema)
+	defer recBuilder.Release()
+
+	for i, arg := range args {
+		fieldBuilder := recBuilder.Field(i)
+		if err := setFieldValue(fieldBuilder, arg.Value); err != nil {
+			return err
+		}
+	}
+
+	rec := recBuilder.NewRecord()
+	defer rec.Release()
+
+	s.stmt.SetParameters(rec)
+
+	return nil
+}
+
+type Tx struct {
+	tx      *Txn
+	timeout time.Duration
+}
+
+func (t *Tx) Commit() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Commit(ctx)
+}
+
+func (t *Tx) Rollback() error {
+	ctx := context.Background()
+	if t.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, t.timeout)
+		defer cancel()
+	}
+
+	return t.tx.Rollback(ctx)
+}
+
+type Driver struct {
+	addr    string
+	timeout time.Duration
+	options []grpc.DialOption
+
+	client *Client
+	txn    *Txn
+}
+
+// Open returns a new connection to the database.
+func (d *Driver) Open(name string) (driver.Conn, error) {
+	if _, err := d.OpenConnector(name); err != nil {
+		return nil, err
+	}
+
+	return d.Connect(context.Background())
+}
+
+// OpenConnector must parse the name in the same format that Driver.Open
+// parses the name parameter.
+func (d *Driver) OpenConnector(name string) (driver.Connector, error) {
+	config, err := NewDriverConfigFromDSN(name)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := d.Configure(config); err != nil {
+		return nil, err
+	}
+
+	return d, nil
+}
+
+// Connect returns a connection to the database.
+func (d *Driver) Connect(ctx context.Context) (driver.Conn, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	client, err := NewClientCtx(ctx, d.addr, nil, nil, d.options...)
+	if err != nil {
+		return nil, err
+	}
+	d.client = client
+
+	return d, nil
+}
+
+// Configure the driver with the corresponding config
+func (d *Driver) Configure(config *DriverConfig) error {
+	// Set the driver properties
+	d.addr = config.Address
+	d.timeout = config.Timeout
+
+	// Create GRPC options necessary for the backend
+	var err error
+	switch config.Backend {
+	case "sqlite":
+		d.options, err = newSqliteBackend(config)
+		if err != nil {
+			return err
+		}
+	case "iox", "ioxs":
+		d.options, err = newIOxBackend(config)
+		if err != nil {
+			return err
+		}
+	default:
+		return fmt.Errorf("invalid backend %q", config.Backend)
+	}
+	d.options = append(d.options, grpc.WithBlock())
+
+	return nil
+}
+
+// Driver returns the underlying Driver of the Connector,
+// mainly to maintain compatibility with the Driver method
+// on sql.DB.
+func (d *Driver) Driver() driver.Driver {
+	return d
+}
+
+// Prepare returns a prepared statement, bound to this connection.
+func (d *Driver) Prepare(query string) (driver.Stmt, error) {
+	return d.PrepareContext(context.Background(), query)
+}
+
+// PrepareContext returns a prepared statement, bound to this connection.
+// context is for the preparation of the statement,
+// it must not store the context within the statement itself.
+func (d *Driver) PrepareContext(ctx context.Context, query string) (driver.Stmt, error) {
+	if _, set := ctx.Deadline(); !set && d.timeout > 0 {
+		var cancel context.CancelFunc
+		ctx, cancel = context.WithTimeout(ctx, d.timeout)
+		defer cancel()
+	}
+
+	var err error
+	var stmt *PreparedStatement
+	if d.txn != nil && d.txn.txn.IsValid() {
+		stmt, err = d.txn.Prepare(ctx, query)
+	} else {
+		stmt, err = d.client.Prepare(ctx, query)
+		d.txn = nil
+	}
+	if err != nil {
+		return nil, err
+	}
+
+	return &Stmt{
+		stmt:    stmt,
+		client:  d.client,
+		timeout: d.timeout,
+	}, nil
+}
+
+// Close invalidates and potentially stops any current
+// prepared statements and transactions, marking this
+// connection as no longer in use.
+func (d *Driver) Close() error {
+	d.addr = ""
+	d.options = nil
+	if d.client == nil {
+		return nil
+	}
+
+	if d.txn != nil && d.txn.txn.IsValid() {
+		return ErrTransactionInProgress
+	}
+
+	// Drivers must ensure all network calls made by Close
+	// do not block indefinitely (e.g. apply a timeout).
+	err := d.client.Close()
+	d.client = nil
+
+	return err
+}
+
+// Begin starts and returns a new transaction.
+func (d *Driver) Begin() (driver.Tx, error) {
+	return d.BeginTx(context.Background(), sql.TxOptions{})
+}
+
+func (d *Driver) BeginTx(ctx context.Context, opts sql.TxOptions) (driver.Tx, error) {
+	tx, err := d.client.BeginTransaction(ctx)
+	if err != nil {
+		return nil, err
+	}
+	d.txn = tx
+
+	return &Tx{tx: tx, timeout: d.timeout}, nil
+}

Review Comment:
   From what I read about the Golang code there is only one connection per `Driver` instance. Golang will then create a pool of drivers which are assigned to an transaction. There is no way to access the "connection" at the `BeginTx` call... What do you have in mind?



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

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

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


[GitHub] [arrow] srebhan commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @zeroshade I found the time to fix the CI issue. The remaining failing test can hardly be due to my changes as I didn't touch any of the mentioned parts/languages...


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

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

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


[GitHub] [arrow] amol- commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   Closing because it has been untouched for a while, in case it's still relevant feel free to reopen and move it forward 👍


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

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

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


[GitHub] [arrow] amol- closed pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

Posted by "amol- (via GitHub)" <gi...@apache.org>.
amol- closed pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework
URL: https://github.com/apache/arrow/pull/34331


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

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

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


[GitHub] [arrow] srebhan commented on pull request #34331: GH-34332: [Go][FlightRPC] Add driver for `database/sql` framework

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

   @zeroshade I added the two discussed follow ups. The third one (mock schema generation for SQLite example) is already implemented as a test case and thus I omitted an issue.
   
   Did I forget anything? If not I will start implementing the TLS part (#35042) as soon as this PR is merged.


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

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

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