Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
93 changes: 40 additions & 53 deletions go/arrow/flight/flightsql/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -1119,24 +1119,10 @@ func (p *PreparedStatement) Execute(ctx context.Context, opts ...grpc.CallOption
return nil, err
}

if p.hasBindParameters() {
pstream, err := p.client.Client.DoPut(ctx, opts...)
if err != nil {
return nil, err
}
wr, err := p.writeBindParameters(pstream, desc)
if err != nil {
return nil, err
}
if err = wr.Close(); err != nil {
return nil, err
}
pstream.CloseSend()
if err = p.captureDoPutPreparedStatementHandle(pstream); err != nil {
return nil, err
}
desc, err = p.bindParameters(ctx, desc, opts...)
if err != nil {
return nil, err
}

return p.client.getFlightInfo(ctx, desc, opts...)
}

Expand All @@ -1156,23 +1142,9 @@ func (p *PreparedStatement) ExecutePut(ctx context.Context, opts ...grpc.CallOpt
return err
}

if p.hasBindParameters() {
pstream, err := p.client.Client.DoPut(ctx, opts...)
if err != nil {
return err
}

wr, err := p.writeBindParameters(pstream, desc)
if err != nil {
return err
}
if err = wr.Close(); err != nil {
return err
}
pstream.CloseSend()
if err = p.captureDoPutPreparedStatementHandle(pstream); err != nil {
return err
}
_, err = p.bindParameters(ctx, desc, opts...)
if err != nil {
return err
}

return nil
Expand Down Expand Up @@ -1200,23 +1172,9 @@ func (p *PreparedStatement) ExecutePoll(ctx context.Context, retryDescriptor *fl
}

if retryDescriptor == nil {
if p.hasBindParameters() {
pstream, err := p.client.Client.DoPut(ctx, opts...)
if err != nil {
return nil, err
}

wr, err := p.writeBindParameters(pstream, desc)
if err != nil {
return nil, err
}
if err = wr.Close(); err != nil {
return nil, err
}
pstream.CloseSend()
if err = p.captureDoPutPreparedStatementHandle(pstream); err != nil {
return nil, err
}
desc, err = p.bindParameters(ctx, desc, opts...)
if err != nil {
return nil, err
}
}
return p.client.Client.PollFlightInfo(ctx, desc, opts...)
Expand Down Expand Up @@ -1248,7 +1206,7 @@ func (p *PreparedStatement) ExecuteUpdate(ctx context.Context, opts ...grpc.Call
return
}
if p.hasBindParameters() {
wr, err = p.writeBindParameters(pstream, desc)
wr, err = p.writeBindParametersToStream(pstream, desc)
if err != nil {
return
}
Expand Down Expand Up @@ -1283,7 +1241,36 @@ func (p *PreparedStatement) hasBindParameters() bool {
return (p.paramBinding != nil && p.paramBinding.NumRows() > 0) || (p.streamBinding != nil)
}

func (p *PreparedStatement) writeBindParameters(pstream pb.FlightService_DoPutClient, desc *pb.FlightDescriptor) (*flight.Writer, error) {
func (p *PreparedStatement) bindParameters(ctx context.Context, desc *pb.FlightDescriptor, opts ...grpc.CallOption) (*flight.FlightDescriptor, error) {
if p.hasBindParameters() {
pstream, err := p.client.Client.DoPut(ctx, opts...)
if err != nil {
return nil, err
}
wr, err := p.writeBindParametersToStream(pstream, desc)
if err != nil {
return nil, err
}
if err = wr.Close(); err != nil {
return nil, err
}
pstream.CloseSend()
if err = p.captureDoPutPreparedStatementHandle(pstream); err != nil {
return nil, err
}

cmd := pb.CommandPreparedStatementQuery{PreparedStatementHandle: p.handle}
desc, err = descForCommand(&cmd)
if err != nil {
return nil, err
}
return desc, nil
}
return desc, nil
}

// XXX: this does not capture the updated handle. Prefer bindParameters.
func (p *PreparedStatement) writeBindParametersToStream(pstream pb.FlightService_DoPutClient, desc *pb.FlightDescriptor) (*flight.Writer, error) {
if p.paramBinding != nil {
wr := flight.NewRecordWriter(pstream, ipc.WithSchema(p.paramBinding.Schema()))
wr.SetFlightDescriptor(desc)
Expand Down
10 changes: 5 additions & 5 deletions go/arrow/flight/flightsql/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -448,9 +448,9 @@ func (s *FlightSqlClientSuite) TestPreparedStatementExecuteParamBinding() {
expectedDesc := getDesc(&pb.CommandPreparedStatementQuery{PreparedStatementHandle: []byte(handle)})

// mocked DoPut result
doPutPreparedStatementResult := &pb.DoPutPreparedStatementResult{PreparedStatementHandle: []byte(updatedHandle)}
doPutPreparedStatementResult := &pb.DoPutPreparedStatementResult{PreparedStatementHandle: []byte(updatedHandle)}
resdata, _ := proto.Marshal(doPutPreparedStatementResult)
putResult := &pb.PutResult{ AppMetadata: resdata }
putResult := &pb.PutResult{AppMetadata: resdata}

// mocked client stream for DoPut
mockedPut := &mockDoPutClient{}
Expand All @@ -461,7 +461,7 @@ func (s *FlightSqlClientSuite) TestPreparedStatementExecuteParamBinding() {
mockedPut.On("CloseSend").Return(nil)
mockedPut.On("Recv").Return(putResult, nil)

infoCmd := &pb.CommandPreparedStatementQuery{PreparedStatementHandle: []byte(handle)}
infoCmd := &pb.CommandPreparedStatementQuery{PreparedStatementHandle: []byte(updatedHandle)}
Comment thread
zeroshade marked this conversation as resolved.
desc := getDesc(infoCmd)
s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil)

Expand Down Expand Up @@ -525,9 +525,9 @@ func (s *FlightSqlClientSuite) TestPreparedStatementExecuteReaderBinding() {
expectedDesc := getDesc(&pb.CommandPreparedStatementQuery{PreparedStatementHandle: []byte(query)})

// mocked DoPut result
doPutPreparedStatementResult := &pb.DoPutPreparedStatementResult{PreparedStatementHandle: []byte(query)}
doPutPreparedStatementResult := &pb.DoPutPreparedStatementResult{PreparedStatementHandle: []byte(query)}
resdata, _ := proto.Marshal(doPutPreparedStatementResult)
putResult := &pb.PutResult{ AppMetadata: resdata }
putResult := &pb.PutResult{AppMetadata: resdata}

// mocked client stream for DoPut
mockedPut := &mockDoPutClient{}
Expand Down