/
client.go
129 lines (109 loc) · 4.3 KB
/
client.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
// 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 flight
import (
"context"
"encoding/base64"
"io"
"strings"
"golang.org/x/xerrors"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
"google.golang.org/grpc/status"
)
// Client is an interface wrapped around the generated FlightServiceClient which is
// generated by grpc protobuf definitions. This interface provides a useful hiding
// of the authentication handshake via calling Authenticate and using the
// ClientAuthHandler rather than manually having to implement the grpc communication
// and sending of the auth token.
type Client interface {
// Authenticate uses the ClientAuthHandler that was used when creating the client
// in order to use the Handshake endpoints of the service.
Authenticate(context.Context, ...grpc.CallOption) error
AuthenticateBasicToken(ctx context.Context, username string, password string, opts ...grpc.CallOption) (context.Context, error)
Close() error
// join the interface from the FlightServiceClient instead of re-defining all
// the endpoints here.
FlightServiceClient
}
type client struct {
conn *grpc.ClientConn
authHandler ClientAuthHandler
FlightServiceClient
}
// NewFlightClient takes in the address of the grpc server and an auth handler for the
// application-level handshake. If using TLS or other grpc configurations they can still
// be passed via the grpc.DialOption list just as if connecting manually without this
// helper function.
//
// Alternatively, a grpc client can be constructed as normal without this helper as the
// grpc generated client code is still exported. This exists to add utility and helpers
// around the authentication and passing the token with requests.
func NewFlightClient(addr string, auth ClientAuthHandler, opts ...grpc.DialOption) (Client, error) {
if auth != nil {
opts = append([]grpc.DialOption{
grpc.WithChainStreamInterceptor(createClientAuthStreamInterceptor(auth)),
grpc.WithChainUnaryInterceptor(createClientAuthUnaryInterceptor(auth)),
}, opts...)
}
conn, err := grpc.Dial(addr, opts...)
if err != nil {
return nil, err
}
return &client{conn: conn, FlightServiceClient: NewFlightServiceClient(conn), authHandler: auth}, nil
}
func (c *client) AuthenticateBasicToken(ctx context.Context, username, password string, opts ...grpc.CallOption) (context.Context, error) {
authCtx := metadata.AppendToOutgoingContext(ctx, "Authorization", "Basic "+base64.RawStdEncoding.EncodeToString([]byte(strings.Join([]string{username, password}, ":"))))
stream, err := c.FlightServiceClient.Handshake(authCtx, opts...)
if err != nil {
return ctx, err
}
header, err := stream.Header()
if err != nil {
return ctx, err
}
_, err = stream.Recv()
if err != nil && err != io.EOF {
return ctx, err
}
err = stream.CloseSend()
if err != nil {
return ctx, err
}
meta := stream.Trailer()
md := metadata.Join(header, meta)
for _, token := range md.Get("authorization") {
if token != "" {
return metadata.AppendToOutgoingContext(ctx, "Authorization", token), nil
}
}
return ctx, xerrors.Errorf("flight: no authorization header on the response")
}
func (c *client) Authenticate(ctx context.Context, opts ...grpc.CallOption) error {
if c.authHandler == nil {
return status.Error(codes.NotFound, "cannot authenticate without an auth-handler")
}
stream, err := c.FlightServiceClient.Handshake(ctx, opts...)
if err != nil {
return err
}
return c.authHandler.Authenticate(ctx, &clientAuthConn{stream})
}
func (c *client) Close() error {
c.FlightServiceClient = nil
return c.conn.Close()
}