zeroshade commented on code in PR #36009:
URL: https://github.com/apache/arrow/pull/36009#discussion_r1229979356


##########
go/arrow/flight/client.go:
##########
@@ -348,6 +352,92 @@ func (c *client) Authenticate(ctx context.Context, opts 
...grpc.CallOption) erro
        return c.authHandler.Authenticate(ctx, &clientAuthConn{stream})
 }
 
+func (c *client) CancelFlightInfo(ctx context.Context, info *FlightInfo, opts 
...grpc.CallOption) (result CancelFlightInfoResult, err error) {
+       var action flight.Action
+       action.Type = CancelFlightInfoActionType
+       if action.Body, err = proto.Marshal(info); err != nil {
+               return
+       }
+
+       stream, err := c.DoAction(ctx, &action, opts...)
+       if err != nil {
+               return
+       }
+       res, err := stream.Recv()
+       if err != nil {
+               return
+       }
+       if err = proto.Unmarshal(res.Body, &result); err != nil {
+               return
+       }
+       for {
+               _, err = stream.Recv()
+               if errors.Is(err, io.EOF) {

Review Comment:
   should we add a documentation comment saying that this will return err == 
io.EOF? or should we add `err = nil` before the return here?



##########
go/arrow/flight/server.go:
##########
@@ -49,9 +49,36 @@ type (
        Action                          = flight.Action
        ActionType                      = flight.ActionType
        Result                          = flight.Result
+       CancelFlightInfoResult    = flight.CancelFlightInfoResult
+       CancelResult = flight.CancelResult

Review Comment:
   indentation



##########
go/arrow/flight/flightsql/client_test.go:
##########
@@ -60,6 +60,18 @@ func (m *FlightServiceClientMock) AuthenticateBasicToken(_ 
context.Context, user
        return args.Get(0).(context.Context), args.Error(1)
 }
 
+func (m *FlightServiceClientMock) CancelFlightInfo(ctx context.Context, info 
*FlightInfo, opts ...grpc.CallOption) (CancelFlightInfoResult, error) {
+       panic("not implemented") // TODO: Implement

Review Comment:
   can this just follow the pattern above of `args := m.Called(info, opts)` and 
`return args.Get(0).(CancelFlightInfoResult), args.Error(1)` so we can use the 
mock to create the tests?



##########
go/arrow/flight/client.go:
##########
@@ -348,6 +352,92 @@ func (c *client) Authenticate(ctx context.Context, opts 
...grpc.CallOption) erro
        return c.authHandler.Authenticate(ctx, &clientAuthConn{stream})
 }
 
+func (c *client) CancelFlightInfo(ctx context.Context, info *FlightInfo, opts 
...grpc.CallOption) (result CancelFlightInfoResult, err error) {
+       var action flight.Action
+       action.Type = CancelFlightInfoActionType
+       if action.Body, err = proto.Marshal(info); err != nil {
+               return
+       }
+
+       stream, err := c.DoAction(ctx, &action, opts...)
+       if err != nil {
+               return
+       }
+       res, err := stream.Recv()
+       if err != nil {
+               return
+       }
+       if err = proto.Unmarshal(res.Body, &result); err != nil {
+               return
+       }
+       for {
+               _, err = stream.Recv()
+               if errors.Is(err, io.EOF) {
+                       break
+               }
+               if err != nil {
+                       return
+               }
+       }
+       return
+}
+
+func (c *client) CloseFlightInfo(ctx context.Context, info *FlightInfo, opts 
...grpc.CallOption) (err error) {
+       var action flight.Action
+       action.Type = CloseFlightInfoActionType
+       if action.Body, err = proto.Marshal(info); err != nil {
+               return
+       }
+
+       stream, err := c.DoAction(ctx, &action, opts...)
+       if err != nil {
+               return
+       }
+       for {
+               _, err = stream.Recv()
+               if errors.Is(err, io.EOF) {
+                       break
+               }
+               if err != nil {
+                       return
+               }
+       }
+       return
+}
+
+func (c *client) RefreshFlightEndpoint(ctx context.Context, endpoint 
*FlightEndpoint, opts ...grpc.CallOption) (*FlightEndpoint, error) {
+       var err error
+
+       var action flight.Action
+       action.Type = RefreshFlightEndpointActionType
+       if action.Body, err = proto.Marshal(endpoint); err != nil {
+               return nil, err
+       }
+
+       stream, err := c.DoAction(ctx, &action, opts...)
+       if err != nil {
+               return nil, err
+       }
+       res, err := stream.Recv()
+       if err != nil {
+               return nil, err
+       }
+       var refreshedEndpoint FlightEndpoint
+       if err = proto.Unmarshal(res.Body, &refreshedEndpoint); err != nil {
+               return nil, err
+       }
+       for {
+               _, err = stream.Recv()
+               if errors.Is(err, io.EOF) {
+                       break
+               }
+               if err != nil {
+                       return nil, err
+               }
+       }
+       return &refreshedEndpoint, nil
+}

Review Comment:
   since we're returning nil here, we should have the previous ones also return 
nil when they succeed rather than io.EOF that they return now



##########
go/arrow/flight/client.go:
##########
@@ -348,6 +352,92 @@ func (c *client) Authenticate(ctx context.Context, opts 
...grpc.CallOption) erro
        return c.authHandler.Authenticate(ctx, &clientAuthConn{stream})
 }
 
+func (c *client) CancelFlightInfo(ctx context.Context, info *FlightInfo, opts 
...grpc.CallOption) (result CancelFlightInfoResult, err error) {
+       var action flight.Action
+       action.Type = CancelFlightInfoActionType
+       if action.Body, err = proto.Marshal(info); err != nil {
+               return
+       }
+
+       stream, err := c.DoAction(ctx, &action, opts...)
+       if err != nil {
+               return
+       }
+       res, err := stream.Recv()
+       if err != nil {
+               return
+       }
+       if err = proto.Unmarshal(res.Body, &result); err != nil {
+               return
+       }
+       for {
+               _, err = stream.Recv()
+               if errors.Is(err, io.EOF) {
+                       break
+               }
+               if err != nil {
+                       return
+               }
+       }
+       return
+}
+
+func (c *client) CloseFlightInfo(ctx context.Context, info *FlightInfo, opts 
...grpc.CallOption) (err error) {
+       var action flight.Action
+       action.Type = CloseFlightInfoActionType
+       if action.Body, err = proto.Marshal(info); err != nil {
+               return
+       }
+
+       stream, err := c.DoAction(ctx, &action, opts...)
+       if err != nil {
+               return
+       }
+       for {
+               _, err = stream.Recv()
+               if errors.Is(err, io.EOF) {

Review Comment:
   same comment as above, should we add documentation that this returns 
`io.EOF` on success? or should we return nil?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to