Compare commits
92 Commits
Author | SHA1 | Date | |
---|---|---|---|
2d1e2e8e64 | |||
6412758177 | |||
836c8159f6 | |||
e406e6e8f4 | |||
2fa2c6284e | |||
2862c4fa12 | |||
6f89fbf8b5 | |||
6ae7ec9a3f | |||
4a35b1b20a | |||
c859c97ee2 | |||
a091c629e1 | |||
96de94a584 | |||
e9cd8410d7 | |||
e37ede1d2e | |||
4420a29ac4 | |||
0544d4bfd0 | |||
fe7379f102 | |||
c76df5052b | |||
3299cad1c3 | |||
d9ab018c49 | |||
e853451cd2 | |||
1becf9d2f5 | |||
1a712cf187 | |||
023f335f67 | |||
bf0da78b63 | |||
e8473850a2 | |||
b836d187fd | |||
9b09229c4d | |||
546c0f7ed6 | |||
adbad1c9b5 | |||
273b986751 | |||
5b205729b9 | |||
fe900b09dd | |||
494c012659 | |||
4abc381ebe | |||
73c8fdac53 | |||
ee2717493a | |||
2435eb9ecd | |||
8fb533dabe | |||
2f0f5ac504 | |||
9ab811d478 | |||
e0a99fb4ba | |||
d40982fc91 | |||
fe3a1cc31b | |||
70713706a1 | |||
0054e7e89b | |||
97f718b504 | |||
202da9270e | |||
6e83ec0ed7 | |||
5c44cdfdaa | |||
09a239f040 | |||
3faff8b2e2 | |||
2345fda18e | |||
5695120efc | |||
183293e061 | |||
4b48876f0e | |||
5089bf58fb | |||
480a347179 | |||
59e560c7a7 | |||
0bd9bea2e9 | |||
bd7581ac59 | |||
db378c3d26 | |||
23740162dc | |||
96422a955f | |||
6fd996fdac | |||
9efa00d103 | |||
72d30f4c34 | |||
2e92779777 | |||
404415b1e3 | |||
07e421d245 | |||
a7d6e29275 | |||
1a8b295dab | |||
ffc45cc066 | |||
0db1ba8093 | |||
43f7c94ac8 | |||
93d13fb5b4 | |||
6a1e3e73dd | |||
ec576ee5ac | |||
606d79afc4 | |||
f4d15a430c | |||
4a841459f1 | |||
ee8c577fc0 | |||
8ae0f94cd7 | |||
69a97863a9 | |||
12c7e4a9f8 | |||
23cced240b | |||
e73c928d85 | |||
779ad90f9a | |||
dca1740be5 | |||
487b34d857 | |||
a31283cf51 | |||
b722bedf8a |
@ -427,6 +427,7 @@ Empty field.
|
||||
| ----- | ----------- | ---- |
|
||||
| key | key is the first key to delete in the range. | bytes |
|
||||
| range_end | range_end is the key following the last key to delete for the range [key, range_end). If range_end is not given, the range is defined to contain only the key argument. If range_end is '\0', the range is all keys greater than or equal to the key argument. | bytes |
|
||||
| prev_kv | If prev_kv is set, etcd gets the previous key-value pairs before deleting it. The previous key-value pairs will be returned in the delte response. | bool |
|
||||
|
||||
|
||||
|
||||
@ -436,6 +437,7 @@ Empty field.
|
||||
| ----- | ----------- | ---- |
|
||||
| header | | ResponseHeader |
|
||||
| deleted | deleted is the number of keys deleted by the delete range request. | int64 |
|
||||
| prev_kvs | if prev_kv is set in the request, the previous key-value pairs will be returned. | (slice of) mvccpb.KeyValue |
|
||||
|
||||
|
||||
|
||||
@ -591,6 +593,7 @@ Empty field.
|
||||
| key | key is the key, in bytes, to put into the key-value store. | bytes |
|
||||
| value | value is the value, in bytes, to associate with the key in the key-value store. | bytes |
|
||||
| lease | lease is the lease ID to associate with the key in the key-value store. A lease value of 0 indicates no lease. | int64 |
|
||||
| prev_kv | If prev_kv is set, etcd gets the previous key-value pair before changing it. The previous key-value pair will be returned in the put response. | bool |
|
||||
|
||||
|
||||
|
||||
@ -599,6 +602,7 @@ Empty field.
|
||||
| Field | Description | Type |
|
||||
| ----- | ----------- | ---- |
|
||||
| header | | ResponseHeader |
|
||||
| prev_kv | if prev_kv is set in the request, the previous key-value pair will be returned. | mvccpb.KeyValue |
|
||||
|
||||
|
||||
|
||||
@ -735,6 +739,7 @@ From google paxosdb paper: Our implementation hinges around a powerful primitive
|
||||
| range_end | range_end is the end of the range [key, range_end) to watch. If range_end is not given, only the key argument is watched. If range_end is equal to '\0', all keys greater than or equal to the key argument are watched. | bytes |
|
||||
| start_revision | start_revision is an optional revision to watch from (inclusive). No start_revision is "now". | int64 |
|
||||
| progress_notify | progress_notify is set so that the etcd server will periodically send a WatchResponse with no events to the new watcher if there are no recent events. It is useful when clients wish to recover a disconnected watcher starting from a recent known revision. The etcd server may decide how often it will send notifications based on current load. | bool |
|
||||
| prev_kv | If prev_kv is set, created watcher gets the previous KV before the event happens. If the previous KV is already compacted, nothing will be returned. | bool |
|
||||
|
||||
|
||||
|
||||
@ -767,6 +772,7 @@ From google paxosdb paper: Our implementation hinges around a powerful primitive
|
||||
| ----- | ----------- | ---- |
|
||||
| type | type is the kind of event. If type is a PUT, it indicates new data has been stored to the key. If type is a DELETE, it indicates the key was deleted. | EventType |
|
||||
| kv | kv holds the KeyValue for the event. A PUT event contains current kv pair. A PUT event with kv.Version=1 indicates the creation of a key. A DELETE/EXPIRE event contains the deleted key with its modification revision set to the revision of deletion. | KeyValue |
|
||||
| prev_kv | prev_kv holds the key-value pair before the event happens. | KeyValue |
|
||||
|
||||
|
||||
|
||||
|
@ -1474,6 +1474,11 @@
|
||||
"format": "byte",
|
||||
"description": "key is the first key to delete in the range."
|
||||
},
|
||||
"prev_kv": {
|
||||
"type": "boolean",
|
||||
"format": "boolean",
|
||||
"description": "If prev_kv is set, etcd gets the previous key-value pairs before deleting it.\nThe previous key-value pairs will be returned in the delte response."
|
||||
},
|
||||
"range_end": {
|
||||
"type": "string",
|
||||
"format": "byte",
|
||||
@ -1491,6 +1496,13 @@
|
||||
},
|
||||
"header": {
|
||||
"$ref": "#/definitions/etcdserverpbResponseHeader"
|
||||
},
|
||||
"prev_kvs": {
|
||||
"type": "array",
|
||||
"items": {
|
||||
"$ref": "#/definitions/mvccpbKeyValue"
|
||||
},
|
||||
"description": "if prev_kv is set in the request, the previous key-value pairs will be returned."
|
||||
}
|
||||
}
|
||||
},
|
||||
@ -1724,6 +1736,11 @@
|
||||
"format": "int64",
|
||||
"description": "lease is the lease ID to associate with the key in the key-value store. A lease\nvalue of 0 indicates no lease."
|
||||
},
|
||||
"prev_kv": {
|
||||
"type": "boolean",
|
||||
"format": "boolean",
|
||||
"description": "If prev_kv is set, etcd gets the previous key-value pair before changing it.\nThe previous key-value pair will be returned in the put response."
|
||||
},
|
||||
"value": {
|
||||
"type": "string",
|
||||
"format": "byte",
|
||||
@ -1736,6 +1753,10 @@
|
||||
"properties": {
|
||||
"header": {
|
||||
"$ref": "#/definitions/etcdserverpbResponseHeader"
|
||||
},
|
||||
"prev_kv": {
|
||||
"$ref": "#/definitions/mvccpbKeyValue",
|
||||
"description": "if prev_kv is set in the request, the previous key-value pair will be returned."
|
||||
}
|
||||
}
|
||||
},
|
||||
@ -1988,6 +2009,11 @@
|
||||
"format": "byte",
|
||||
"description": "key is the key to register for watching."
|
||||
},
|
||||
"prev_kv": {
|
||||
"type": "boolean",
|
||||
"format": "boolean",
|
||||
"description": "If prev_kv is set, created watcher gets the previous KV before the event happens.\nIf the previous KV is already compacted, nothing will be returned."
|
||||
},
|
||||
"progress_notify": {
|
||||
"type": "boolean",
|
||||
"format": "boolean",
|
||||
@ -2057,6 +2083,10 @@
|
||||
"$ref": "#/definitions/mvccpbKeyValue",
|
||||
"description": "kv holds the KeyValue for the event.\nA PUT event contains current kv pair.\nA PUT event with kv.Version=1 indicates the creation of a key.\nA DELETE/EXPIRE event contains the deleted key with\nits modification revision set to the revision of deletion."
|
||||
},
|
||||
"prev_kv": {
|
||||
"$ref": "#/definitions/mvccpbKeyValue",
|
||||
"description": "prev_kv holds the key-value pair before the event happens."
|
||||
},
|
||||
"type": {
|
||||
"$ref": "#/definitions/EventEventType",
|
||||
"description": "type is the kind of event. If type is a PUT, it indicates\nnew data has been stored to the key. If type is a DELETE,\nit indicates the key was deleted."
|
||||
|
@ -357,6 +357,8 @@ To help clients discover the etcd cluster, the following DNS SRV records are loo
|
||||
|
||||
If `_etcd-client-ssl._tcp.example.com` is found, clients will attempt to communicate with the etcd cluster over SSL/TLS.
|
||||
|
||||
If etcd is using TLS without a custom certificate authority, the discovery domain (e.g., example.com) must match the SRV record domain (e.g., infra1.example.com). This is to mitigate attacks that forge SRV records to point to a different domain; the domain would have a valid certificate under PKI but be controlled by an unknown third party.
|
||||
|
||||
#### Create DNS SRV records
|
||||
|
||||
```
|
||||
|
@ -21,9 +21,9 @@ import (
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
|
||||
math "math"
|
||||
)
|
||||
|
||||
import io "io"
|
||||
io "io"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
|
@ -22,7 +22,10 @@ import (
|
||||
"github.com/coreos/etcd/mvcc/backend"
|
||||
)
|
||||
|
||||
// isSubset returns true if a is a subset of b
|
||||
// isSubset returns true if a is a subset of b.
|
||||
// If a is a prefix of b, then a is a subset of b.
|
||||
// Given intervals [a1,a2) and [b1,b2), is
|
||||
// the a interval a subset of b?
|
||||
func isSubset(a, b *rangePerm) bool {
|
||||
switch {
|
||||
case len(a.end) == 0 && len(b.end) == 0:
|
||||
@ -32,9 +35,11 @@ func isSubset(a, b *rangePerm) bool {
|
||||
// b is a key, a is a range
|
||||
return false
|
||||
case len(a.end) == 0:
|
||||
return 0 <= bytes.Compare(a.begin, b.begin) && bytes.Compare(a.begin, b.end) <= 0
|
||||
// a is a key, b is a range. need b1 <= a1 and a1 < b2
|
||||
return bytes.Compare(b.begin, a.begin) <= 0 && bytes.Compare(a.begin, b.end) < 0
|
||||
default:
|
||||
return 0 <= bytes.Compare(a.begin, b.begin) && bytes.Compare(a.end, b.end) <= 0
|
||||
// both are ranges. need b1 <= a1 and a2 <= b2
|
||||
return bytes.Compare(b.begin, a.begin) <= 0 && bytes.Compare(a.end, b.end) <= 0
|
||||
}
|
||||
}
|
||||
|
||||
@ -88,12 +93,18 @@ func mergeRangePerms(perms []*rangePerm) []*rangePerm {
|
||||
i := 0
|
||||
for i < len(perms) {
|
||||
begin, next := i, i
|
||||
for next+1 < len(perms) && bytes.Compare(perms[next].end, perms[next+1].begin) != -1 {
|
||||
for next+1 < len(perms) && bytes.Compare(perms[next].end, perms[next+1].begin) >= 0 {
|
||||
next++
|
||||
}
|
||||
|
||||
merged = append(merged, &rangePerm{begin: perms[begin].begin, end: perms[next].end})
|
||||
|
||||
// don't merge ["a", "b") with ["b", ""), because perms[next+1].end is empty.
|
||||
if next != begin && len(perms[next].end) > 0 {
|
||||
merged = append(merged, &rangePerm{begin: perms[begin].begin, end: perms[next].end})
|
||||
} else {
|
||||
merged = append(merged, perms[begin])
|
||||
if next != begin {
|
||||
merged = append(merged, perms[next])
|
||||
}
|
||||
}
|
||||
i = next + 1
|
||||
}
|
||||
|
||||
|
@ -46,6 +46,10 @@ func TestGetMergedPerms(t *testing.T) {
|
||||
[]*rangePerm{{[]byte("a"), []byte("b")}},
|
||||
[]*rangePerm{{[]byte("a"), []byte("b")}},
|
||||
},
|
||||
{
|
||||
[]*rangePerm{{[]byte("a"), []byte("b")}, {[]byte("b"), []byte("")}},
|
||||
[]*rangePerm{{[]byte("a"), []byte("b")}, {[]byte("b"), []byte("")}},
|
||||
},
|
||||
{
|
||||
[]*rangePerm{{[]byte("a"), []byte("b")}, {[]byte("b"), []byte("c")}},
|
||||
[]*rangePerm{{[]byte("a"), []byte("c")}},
|
||||
@ -106,7 +110,7 @@ func TestGetMergedPerms(t *testing.T) {
|
||||
},
|
||||
{
|
||||
[]*rangePerm{{[]byte("a"), []byte("")}, {[]byte("b"), []byte("c")}, {[]byte("b"), []byte("")}, {[]byte("c"), []byte("")}, {[]byte("d"), []byte("")}},
|
||||
[]*rangePerm{{[]byte("a"), []byte("")}, {[]byte("b"), []byte("c")}, {[]byte("d"), []byte("")}},
|
||||
[]*rangePerm{{[]byte("a"), []byte("")}, {[]byte("b"), []byte("c")}, {[]byte("c"), []byte("")}, {[]byte("d"), []byte("")}},
|
||||
},
|
||||
// duplicate ranges
|
||||
{
|
||||
|
@ -115,32 +115,32 @@ func NewAuth(c *Client) Auth {
|
||||
}
|
||||
|
||||
func (auth *auth) AuthEnable(ctx context.Context) (*AuthEnableResponse, error) {
|
||||
resp, err := auth.remote.AuthEnable(ctx, &pb.AuthEnableRequest{}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.AuthEnable(ctx, &pb.AuthEnableRequest{})
|
||||
return (*AuthEnableResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
func (auth *auth) AuthDisable(ctx context.Context) (*AuthDisableResponse, error) {
|
||||
resp, err := auth.remote.AuthDisable(ctx, &pb.AuthDisableRequest{}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.AuthDisable(ctx, &pb.AuthDisableRequest{})
|
||||
return (*AuthDisableResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
func (auth *auth) UserAdd(ctx context.Context, name string, password string) (*AuthUserAddResponse, error) {
|
||||
resp, err := auth.remote.UserAdd(ctx, &pb.AuthUserAddRequest{Name: name, Password: password}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.UserAdd(ctx, &pb.AuthUserAddRequest{Name: name, Password: password})
|
||||
return (*AuthUserAddResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
func (auth *auth) UserDelete(ctx context.Context, name string) (*AuthUserDeleteResponse, error) {
|
||||
resp, err := auth.remote.UserDelete(ctx, &pb.AuthUserDeleteRequest{Name: name}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.UserDelete(ctx, &pb.AuthUserDeleteRequest{Name: name})
|
||||
return (*AuthUserDeleteResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
func (auth *auth) UserChangePassword(ctx context.Context, name string, password string) (*AuthUserChangePasswordResponse, error) {
|
||||
resp, err := auth.remote.UserChangePassword(ctx, &pb.AuthUserChangePasswordRequest{Name: name, Password: password}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.UserChangePassword(ctx, &pb.AuthUserChangePasswordRequest{Name: name, Password: password})
|
||||
return (*AuthUserChangePasswordResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
func (auth *auth) UserGrantRole(ctx context.Context, user string, role string) (*AuthUserGrantRoleResponse, error) {
|
||||
resp, err := auth.remote.UserGrantRole(ctx, &pb.AuthUserGrantRoleRequest{User: user, Role: role}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.UserGrantRole(ctx, &pb.AuthUserGrantRoleRequest{User: user, Role: role})
|
||||
return (*AuthUserGrantRoleResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
@ -155,12 +155,12 @@ func (auth *auth) UserList(ctx context.Context) (*AuthUserListResponse, error) {
|
||||
}
|
||||
|
||||
func (auth *auth) UserRevokeRole(ctx context.Context, name string, role string) (*AuthUserRevokeRoleResponse, error) {
|
||||
resp, err := auth.remote.UserRevokeRole(ctx, &pb.AuthUserRevokeRoleRequest{Name: name, Role: role}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.UserRevokeRole(ctx, &pb.AuthUserRevokeRoleRequest{Name: name, Role: role})
|
||||
return (*AuthUserRevokeRoleResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
func (auth *auth) RoleAdd(ctx context.Context, name string) (*AuthRoleAddResponse, error) {
|
||||
resp, err := auth.remote.RoleAdd(ctx, &pb.AuthRoleAddRequest{Name: name}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.RoleAdd(ctx, &pb.AuthRoleAddRequest{Name: name})
|
||||
return (*AuthRoleAddResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
@ -170,7 +170,7 @@ func (auth *auth) RoleGrantPermission(ctx context.Context, name string, key, ran
|
||||
RangeEnd: []byte(rangeEnd),
|
||||
PermType: authpb.Permission_Type(permType),
|
||||
}
|
||||
resp, err := auth.remote.RoleGrantPermission(ctx, &pb.AuthRoleGrantPermissionRequest{Name: name, Perm: perm}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.RoleGrantPermission(ctx, &pb.AuthRoleGrantPermissionRequest{Name: name, Perm: perm})
|
||||
return (*AuthRoleGrantPermissionResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
@ -185,12 +185,12 @@ func (auth *auth) RoleList(ctx context.Context) (*AuthRoleListResponse, error) {
|
||||
}
|
||||
|
||||
func (auth *auth) RoleRevokePermission(ctx context.Context, role string, key, rangeEnd string) (*AuthRoleRevokePermissionResponse, error) {
|
||||
resp, err := auth.remote.RoleRevokePermission(ctx, &pb.AuthRoleRevokePermissionRequest{Role: role, Key: key, RangeEnd: rangeEnd}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.RoleRevokePermission(ctx, &pb.AuthRoleRevokePermissionRequest{Role: role, Key: key, RangeEnd: rangeEnd})
|
||||
return (*AuthRoleRevokePermissionResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
func (auth *auth) RoleDelete(ctx context.Context, role string) (*AuthRoleDeleteResponse, error) {
|
||||
resp, err := auth.remote.RoleDelete(ctx, &pb.AuthRoleDeleteRequest{Role: role}, grpc.FailFast(false))
|
||||
resp, err := auth.remote.RoleDelete(ctx, &pb.AuthRoleDeleteRequest{Role: role})
|
||||
return (*AuthRoleDeleteResponse)(resp), toErr(ctx, err)
|
||||
}
|
||||
|
||||
|
@ -17,7 +17,7 @@ package clientv3
|
||||
import (
|
||||
"net/url"
|
||||
"strings"
|
||||
"sync/atomic"
|
||||
"sync"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc"
|
||||
@ -26,32 +26,133 @@ import (
|
||||
// simpleBalancer does the bare minimum to expose multiple eps
|
||||
// to the grpc reconnection code path
|
||||
type simpleBalancer struct {
|
||||
// eps are the client's endpoints stripped of any URL scheme
|
||||
eps []string
|
||||
ch chan []grpc.Address
|
||||
numGets uint32
|
||||
// addrs are the client's endpoints for grpc
|
||||
addrs []grpc.Address
|
||||
// notifyCh notifies grpc of the set of addresses for connecting
|
||||
notifyCh chan []grpc.Address
|
||||
|
||||
// readyc closes once the first connection is up
|
||||
readyc chan struct{}
|
||||
readyOnce sync.Once
|
||||
|
||||
// mu protects upEps, pinAddr, and connectingAddr
|
||||
mu sync.RWMutex
|
||||
// upEps holds the current endpoints that have an active connection
|
||||
upEps map[string]struct{}
|
||||
// upc closes when upEps transitions from empty to non-zero or the balancer closes.
|
||||
upc chan struct{}
|
||||
|
||||
// pinAddr is the currently pinned address; set to the empty string on
|
||||
// intialization and shutdown.
|
||||
pinAddr string
|
||||
|
||||
closed bool
|
||||
}
|
||||
|
||||
func newSimpleBalancer(eps []string) grpc.Balancer {
|
||||
ch := make(chan []grpc.Address, 1)
|
||||
func newSimpleBalancer(eps []string) *simpleBalancer {
|
||||
notifyCh := make(chan []grpc.Address, 1)
|
||||
addrs := make([]grpc.Address, len(eps))
|
||||
for i := range eps {
|
||||
addrs[i].Addr = getHost(eps[i])
|
||||
}
|
||||
ch <- addrs
|
||||
return &simpleBalancer{eps: eps, ch: ch}
|
||||
notifyCh <- addrs
|
||||
sb := &simpleBalancer{
|
||||
addrs: addrs,
|
||||
notifyCh: notifyCh,
|
||||
readyc: make(chan struct{}),
|
||||
upEps: make(map[string]struct{}),
|
||||
upc: make(chan struct{}),
|
||||
}
|
||||
return sb
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) Start(target string) error { return nil }
|
||||
func (b *simpleBalancer) Up(addr grpc.Address) func(error) { return func(error) {} }
|
||||
func (b *simpleBalancer) Get(ctx context.Context, opts grpc.BalancerGetOptions) (grpc.Address, func(), error) {
|
||||
v := atomic.AddUint32(&b.numGets, 1)
|
||||
ep := b.eps[v%uint32(len(b.eps))]
|
||||
return grpc.Address{Addr: getHost(ep)}, func() {}, nil
|
||||
func (b *simpleBalancer) Start(target string) error { return nil }
|
||||
|
||||
func (b *simpleBalancer) ConnectNotify() <-chan struct{} {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
return b.upc
|
||||
}
|
||||
func (b *simpleBalancer) Notify() <-chan []grpc.Address { return b.ch }
|
||||
|
||||
func (b *simpleBalancer) Up(addr grpc.Address) func(error) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
|
||||
// gRPC might call Up after it called Close. We add this check
|
||||
// to "fix" it up at application layer. Or our simplerBalancer
|
||||
// might panic since b.upc is closed.
|
||||
if b.closed {
|
||||
return func(err error) {}
|
||||
}
|
||||
|
||||
if len(b.upEps) == 0 {
|
||||
// notify waiting Get()s and pin first connected address
|
||||
close(b.upc)
|
||||
b.pinAddr = addr.Addr
|
||||
}
|
||||
b.upEps[addr.Addr] = struct{}{}
|
||||
|
||||
// notify client that a connection is up
|
||||
b.readyOnce.Do(func() { close(b.readyc) })
|
||||
|
||||
return func(err error) {
|
||||
b.mu.Lock()
|
||||
delete(b.upEps, addr.Addr)
|
||||
if len(b.upEps) == 0 && b.pinAddr != "" {
|
||||
b.upc = make(chan struct{})
|
||||
} else if b.pinAddr == addr.Addr {
|
||||
// choose new random up endpoint
|
||||
for k := range b.upEps {
|
||||
b.pinAddr = k
|
||||
break
|
||||
}
|
||||
}
|
||||
b.mu.Unlock()
|
||||
}
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) Get(ctx context.Context, opts grpc.BalancerGetOptions) (grpc.Address, func(), error) {
|
||||
var addr string
|
||||
for {
|
||||
b.mu.RLock()
|
||||
ch := b.upc
|
||||
b.mu.RUnlock()
|
||||
select {
|
||||
case <-ch:
|
||||
case <-ctx.Done():
|
||||
return grpc.Address{Addr: ""}, nil, ctx.Err()
|
||||
}
|
||||
b.mu.RLock()
|
||||
addr = b.pinAddr
|
||||
upEps := len(b.upEps)
|
||||
b.mu.RUnlock()
|
||||
if addr == "" {
|
||||
return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
|
||||
}
|
||||
if upEps > 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
return grpc.Address{Addr: addr}, func() {}, nil
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) Notify() <-chan []grpc.Address { return b.notifyCh }
|
||||
|
||||
func (b *simpleBalancer) Close() error {
|
||||
close(b.ch)
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
// In case gRPC calls close twice. TODO: remove the checking
|
||||
// when we are sure that gRPC wont call close twice.
|
||||
if b.closed {
|
||||
return nil
|
||||
}
|
||||
b.closed = true
|
||||
close(b.notifyCh)
|
||||
// terminate all waiting Get()s
|
||||
b.pinAddr = ""
|
||||
if len(b.upEps) == 0 {
|
||||
close(b.upc)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -46,9 +46,11 @@ type Client struct {
|
||||
Auth
|
||||
Maintenance
|
||||
|
||||
conn *grpc.ClientConn
|
||||
cfg Config
|
||||
creds *credentials.TransportCredentials
|
||||
conn *grpc.ClientConn
|
||||
cfg Config
|
||||
creds *credentials.TransportCredentials
|
||||
balancer *simpleBalancer
|
||||
retryWrapper retryRpcFunc
|
||||
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
@ -138,11 +140,10 @@ func (c *Client) dialTarget(endpoint string) (proto string, host string, creds *
|
||||
return
|
||||
}
|
||||
|
||||
// dialSetupOpts gives the dial opts prioer to any authentication
|
||||
func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) []grpc.DialOption {
|
||||
opts := []grpc.DialOption{
|
||||
grpc.WithBlock(),
|
||||
grpc.WithTimeout(c.cfg.DialTimeout),
|
||||
// dialSetupOpts gives the dial opts prior to any authentication
|
||||
func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) (opts []grpc.DialOption) {
|
||||
if c.cfg.DialTimeout > 0 {
|
||||
opts = []grpc.DialOption{grpc.WithTimeout(c.cfg.DialTimeout)}
|
||||
}
|
||||
opts = append(opts, dopts...)
|
||||
|
||||
@ -240,12 +241,30 @@ func newClient(cfg *Config) (*Client, error) {
|
||||
client.Password = cfg.Password
|
||||
}
|
||||
|
||||
b := newSimpleBalancer(cfg.Endpoints)
|
||||
conn, err := client.dial(cfg.Endpoints[0], grpc.WithBalancer(b))
|
||||
client.balancer = newSimpleBalancer(cfg.Endpoints)
|
||||
conn, err := client.dial(cfg.Endpoints[0], grpc.WithBalancer(client.balancer))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
client.conn = conn
|
||||
client.retryWrapper = client.newRetryWrapper()
|
||||
|
||||
// wait for a connection
|
||||
if cfg.DialTimeout > 0 {
|
||||
hasConn := false
|
||||
waitc := time.After(cfg.DialTimeout)
|
||||
select {
|
||||
case <-client.balancer.readyc:
|
||||
hasConn = true
|
||||
case <-ctx.Done():
|
||||
case <-waitc:
|
||||
}
|
||||
if !hasConn {
|
||||
client.cancel()
|
||||
conn.Close()
|
||||
return nil, grpc.ErrClientConnTimeout
|
||||
}
|
||||
}
|
||||
|
||||
client.Cluster = NewCluster(client)
|
||||
client.KV = NewKV(client)
|
||||
@ -280,8 +299,12 @@ func isHaltErr(ctx context.Context, err error) bool {
|
||||
return eErr != rpctypes.ErrStopped && eErr != rpctypes.ErrNoLeader
|
||||
}
|
||||
// treat etcdserver errors not recognized by the client as halting
|
||||
return strings.Contains(err.Error(), grpc.ErrClientConnClosing.Error()) ||
|
||||
strings.Contains(err.Error(), "etcdserver:")
|
||||
return isConnClosing(err) || strings.Contains(err.Error(), "etcdserver:")
|
||||
}
|
||||
|
||||
// isConnClosing returns true if the error matches a grpc client closing error
|
||||
func isConnClosing(err error) bool {
|
||||
return strings.Contains(err.Error(), grpc.ErrClientConnClosing.Error())
|
||||
}
|
||||
|
||||
func toErr(ctx context.Context, err error) error {
|
||||
@ -289,9 +312,12 @@ func toErr(ctx context.Context, err error) error {
|
||||
return nil
|
||||
}
|
||||
err = rpctypes.Error(err)
|
||||
if ctx.Err() != nil && strings.Contains(err.Error(), "context") {
|
||||
switch {
|
||||
case ctx.Err() != nil && strings.Contains(err.Error(), "context"):
|
||||
err = ctx.Err()
|
||||
} else if strings.Contains(err.Error(), grpc.ErrClientConnClosing.Error()) {
|
||||
case strings.Contains(err.Error(), ErrNoAvailableEndpoints.Error()):
|
||||
err = ErrNoAvailableEndpoints
|
||||
case strings.Contains(err.Error(), grpc.ErrClientConnClosing.Error()):
|
||||
err = grpc.ErrClientConnClosing
|
||||
}
|
||||
return err
|
||||
|
@ -20,11 +20,14 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
func TestDialTimeout(t *testing.T) {
|
||||
defer testutil.AfterTest(t)
|
||||
|
||||
donec := make(chan error)
|
||||
go func() {
|
||||
// without timeout, grpc keeps redialing if connection refused
|
||||
@ -56,6 +59,15 @@ func TestDialTimeout(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestDialNoTimeout(t *testing.T) {
|
||||
cfg := Config{Endpoints: []string{"127.0.0.1:12345"}}
|
||||
c, err := New(cfg)
|
||||
if c == nil || err != nil {
|
||||
t.Fatalf("new client with DialNoWait should succeed, got %v", err)
|
||||
}
|
||||
c.Close()
|
||||
}
|
||||
|
||||
func TestIsHaltErr(t *testing.T) {
|
||||
if !isHaltErr(nil, fmt.Errorf("etcdserver: some etcdserver error")) {
|
||||
t.Errorf(`error prefixed with "etcdserver: " should be Halted by default`)
|
||||
|
@ -47,12 +47,12 @@ type cluster struct {
|
||||
}
|
||||
|
||||
func NewCluster(c *Client) Cluster {
|
||||
return &cluster{remote: pb.NewClusterClient(c.conn)}
|
||||
return &cluster{remote: RetryClusterClient(c)}
|
||||
}
|
||||
|
||||
func (c *cluster) MemberAdd(ctx context.Context, peerAddrs []string) (*MemberAddResponse, error) {
|
||||
r := &pb.MemberAddRequest{PeerURLs: peerAddrs}
|
||||
resp, err := c.remote.MemberAdd(ctx, r, grpc.FailFast(false))
|
||||
resp, err := c.remote.MemberAdd(ctx, r)
|
||||
if err == nil {
|
||||
return (*MemberAddResponse)(resp), nil
|
||||
}
|
||||
@ -64,7 +64,7 @@ func (c *cluster) MemberAdd(ctx context.Context, peerAddrs []string) (*MemberAdd
|
||||
|
||||
func (c *cluster) MemberRemove(ctx context.Context, id uint64) (*MemberRemoveResponse, error) {
|
||||
r := &pb.MemberRemoveRequest{ID: id}
|
||||
resp, err := c.remote.MemberRemove(ctx, r, grpc.FailFast(false))
|
||||
resp, err := c.remote.MemberRemove(ctx, r)
|
||||
if err == nil {
|
||||
return (*MemberRemoveResponse)(resp), nil
|
||||
}
|
||||
@ -78,7 +78,7 @@ func (c *cluster) MemberUpdate(ctx context.Context, id uint64, peerAddrs []strin
|
||||
// it is safe to retry on update.
|
||||
for {
|
||||
r := &pb.MemberUpdateRequest{ID: id, PeerURLs: peerAddrs}
|
||||
resp, err := c.remote.MemberUpdate(ctx, r, grpc.FailFast(false))
|
||||
resp, err := c.remote.MemberUpdate(ctx, r)
|
||||
if err == nil {
|
||||
return (*MemberUpdateResponse)(resp), nil
|
||||
}
|
||||
|
@ -40,7 +40,7 @@ type Election struct {
|
||||
|
||||
// NewElection returns a new election on a given key prefix.
|
||||
func NewElection(client *v3.Client, pfx string) *Election {
|
||||
return &Election{client: client, keyPrefix: pfx}
|
||||
return &Election{client: client, keyPrefix: pfx + "/"}
|
||||
}
|
||||
|
||||
// Campaign puts a value as eligible for the election. It blocks until
|
||||
@ -59,7 +59,6 @@ func (e *Election) Campaign(ctx context.Context, val string) error {
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
e.leaderKey, e.leaderRev, e.leaderSession = k, resp.Header.Revision, s
|
||||
if !resp.Succeeded {
|
||||
kv := resp.Responses[0].GetResponseRange().Kvs[0]
|
||||
|
@ -32,7 +32,7 @@ type Mutex struct {
|
||||
}
|
||||
|
||||
func NewMutex(client *v3.Client, pfx string) *Mutex {
|
||||
return &Mutex{client, pfx, "", -1}
|
||||
return &Mutex{client, pfx + "/", "", -1}
|
||||
}
|
||||
|
||||
// Lock locks the mutex with a cancellable context. If the context is cancelled
|
||||
@ -43,7 +43,7 @@ func (m *Mutex) Lock(ctx context.Context) error {
|
||||
return serr
|
||||
}
|
||||
|
||||
m.myKey = fmt.Sprintf("%s/%x", m.pfx, s.Lease())
|
||||
m.myKey = fmt.Sprintf("%s%x", m.pfx, s.Lease())
|
||||
cmp := v3.Compare(v3.CreateRevision(m.myKey), "=", 0)
|
||||
// put self in lock waiters via myKey; oldest waiter holds lock
|
||||
put := v3.OpPut(m.myKey, "", v3.WithLease(s.Lease()))
|
||||
|
@ -16,6 +16,7 @@ package integration
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"math/rand"
|
||||
"reflect"
|
||||
"strings"
|
||||
"testing"
|
||||
@ -662,3 +663,75 @@ func TestKVPutStoppedServerAndClose(t *testing.T) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestKVGetOneEndpointDown ensures a client can connect and get if one endpoint is down
|
||||
func TestKVPutOneEndpointDown(t *testing.T) {
|
||||
defer testutil.AfterTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// get endpoint list
|
||||
eps := make([]string, 3)
|
||||
for i := range eps {
|
||||
eps[i] = clus.Members[i].GRPCAddr()
|
||||
}
|
||||
|
||||
// make a dead node
|
||||
clus.Members[rand.Intn(len(eps))].Stop(t)
|
||||
|
||||
// try to connect with dead node in the endpoint list
|
||||
cfg := clientv3.Config{Endpoints: eps, DialTimeout: 1 * time.Second}
|
||||
cli, err := clientv3.New(cfg)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer cli.Close()
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), 3*time.Second)
|
||||
if _, err := cli.Get(ctx, "abc", clientv3.WithSerializable()); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
cancel()
|
||||
}
|
||||
|
||||
// TestKVGetResetLoneEndpoint ensures that if an endpoint resets and all other
|
||||
// endpoints are down, then it will reconnect.
|
||||
func TestKVGetResetLoneEndpoint(t *testing.T) {
|
||||
defer testutil.AfterTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// get endpoint list
|
||||
eps := make([]string, 2)
|
||||
for i := range eps {
|
||||
eps[i] = clus.Members[i].GRPCAddr()
|
||||
}
|
||||
|
||||
cfg := clientv3.Config{Endpoints: eps, DialTimeout: 500 * time.Millisecond}
|
||||
cli, err := clientv3.New(cfg)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer cli.Close()
|
||||
|
||||
// disconnect everything
|
||||
clus.Members[0].Stop(t)
|
||||
clus.Members[1].Stop(t)
|
||||
|
||||
// have Get try to reconnect
|
||||
donec := make(chan struct{})
|
||||
go func() {
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), 5*time.Second)
|
||||
if _, err := cli.Get(ctx, "abc", clientv3.WithSerializable()); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
cancel()
|
||||
close(donec)
|
||||
}()
|
||||
time.Sleep(500 * time.Millisecond)
|
||||
clus.Members[0].Restart(t)
|
||||
select {
|
||||
case <-time.After(10 * time.Second):
|
||||
t.Fatalf("timed out waiting for Get")
|
||||
case <-donec:
|
||||
}
|
||||
}
|
||||
|
@ -673,3 +673,112 @@ func TestWatchWithRequireLeader(t *testing.T) {
|
||||
t.Fatalf("expected response, got closed channel")
|
||||
}
|
||||
}
|
||||
|
||||
// TestWatchOverlapContextCancel stresses the watcher stream teardown path by
|
||||
// creating/canceling watchers to ensure that new watchers are not taken down
|
||||
// by a torn down watch stream. The sort of race that's being detected:
|
||||
// 1. create w1 using a cancelable ctx with %v as "ctx"
|
||||
// 2. cancel ctx
|
||||
// 3. watcher client begins tearing down watcher grpc stream since no more watchers
|
||||
// 3. start creating watcher w2 using a new "ctx" (not canceled), attaches to old grpc stream
|
||||
// 4. watcher client finishes tearing down stream on "ctx"
|
||||
// 5. w2 comes back canceled
|
||||
func TestWatchOverlapContextCancel(t *testing.T) {
|
||||
f := func(clus *integration.ClusterV3) {}
|
||||
testWatchOverlapContextCancel(t, f)
|
||||
}
|
||||
|
||||
func TestWatchOverlapDropConnContextCancel(t *testing.T) {
|
||||
f := func(clus *integration.ClusterV3) {
|
||||
clus.Members[0].DropConnections()
|
||||
}
|
||||
testWatchOverlapContextCancel(t, f)
|
||||
}
|
||||
|
||||
func testWatchOverlapContextCancel(t *testing.T, f func(*integration.ClusterV3)) {
|
||||
defer testutil.AfterTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// each unique context "%v" has a unique grpc stream
|
||||
n := 100
|
||||
ctxs, ctxc := make([]context.Context, 5), make([]chan struct{}, 5)
|
||||
for i := range ctxs {
|
||||
// make "%v" unique
|
||||
ctxs[i] = context.WithValue(context.TODO(), "key", i)
|
||||
// limits the maximum number of outstanding watchers per stream
|
||||
ctxc[i] = make(chan struct{}, 2)
|
||||
}
|
||||
|
||||
// issue concurrent watches on "abc" with cancel
|
||||
cli := clus.RandClient()
|
||||
if _, err := cli.Put(context.TODO(), "abc", "def"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
ch := make(chan struct{}, n)
|
||||
for i := 0; i < n; i++ {
|
||||
go func() {
|
||||
defer func() { ch <- struct{}{} }()
|
||||
idx := rand.Intn(len(ctxs))
|
||||
ctx, cancel := context.WithCancel(ctxs[idx])
|
||||
ctxc[idx] <- struct{}{}
|
||||
wch := cli.Watch(ctx, "abc", clientv3.WithRev(1))
|
||||
f(clus)
|
||||
select {
|
||||
case _, ok := <-wch:
|
||||
if !ok {
|
||||
t.Fatalf("unexpected closed channel %p", wch)
|
||||
}
|
||||
// may take a second or two to reestablish a watcher because of
|
||||
// grpc backoff policies for disconnects
|
||||
case <-time.After(5 * time.Second):
|
||||
t.Errorf("timed out waiting for watch on %p", wch)
|
||||
}
|
||||
// randomize how cancel overlaps with watch creation
|
||||
if rand.Intn(2) == 0 {
|
||||
<-ctxc[idx]
|
||||
cancel()
|
||||
} else {
|
||||
cancel()
|
||||
<-ctxc[idx]
|
||||
}
|
||||
}()
|
||||
}
|
||||
// join on watches
|
||||
for i := 0; i < n; i++ {
|
||||
select {
|
||||
case <-ch:
|
||||
case <-time.After(5 * time.Second):
|
||||
t.Fatalf("timed out waiting for completed watch")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TestWatchCanelAndCloseClient ensures that canceling a watcher then immediately
|
||||
// closing the client does not return a client closing error.
|
||||
func TestWatchCancelAndCloseClient(t *testing.T) {
|
||||
defer testutil.AfterTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.Client(0)
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
wch := cli.Watch(ctx, "abc")
|
||||
donec := make(chan struct{})
|
||||
go func() {
|
||||
defer close(donec)
|
||||
select {
|
||||
case wr, ok := <-wch:
|
||||
if ok {
|
||||
t.Fatalf("expected closed watch after cancel(), got resp=%+v err=%v", wr, wr.Err())
|
||||
}
|
||||
case <-time.After(5 * time.Second):
|
||||
t.Fatal("timed out waiting for closed channel")
|
||||
}
|
||||
}()
|
||||
cancel()
|
||||
if err := cli.Close(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
<-donec
|
||||
clus.TakeClient(0)
|
||||
}
|
||||
|
@ -82,7 +82,7 @@ type kv struct {
|
||||
}
|
||||
|
||||
func NewKV(c *Client) KV {
|
||||
return &kv{remote: pb.NewKVClient(c.conn)}
|
||||
return &kv{remote: RetryKVClient(c)}
|
||||
}
|
||||
|
||||
func (kv *kv) Put(ctx context.Context, key, val string, opts ...OpOption) (*PutResponse, error) {
|
||||
@ -157,15 +157,15 @@ func (kv *kv) do(ctx context.Context, op Op) (OpResponse, error) {
|
||||
}
|
||||
case tPut:
|
||||
var resp *pb.PutResponse
|
||||
r := &pb.PutRequest{Key: op.key, Value: op.val, Lease: int64(op.leaseID)}
|
||||
resp, err = kv.remote.Put(ctx, r, grpc.FailFast(false))
|
||||
r := &pb.PutRequest{Key: op.key, Value: op.val, Lease: int64(op.leaseID), PrevKv: op.prevKV}
|
||||
resp, err = kv.remote.Put(ctx, r)
|
||||
if err == nil {
|
||||
return OpResponse{put: (*PutResponse)(resp)}, nil
|
||||
}
|
||||
case tDeleteRange:
|
||||
var resp *pb.DeleteRangeResponse
|
||||
r := &pb.DeleteRangeRequest{Key: op.key, RangeEnd: op.end}
|
||||
resp, err = kv.remote.DeleteRange(ctx, r, grpc.FailFast(false))
|
||||
r := &pb.DeleteRangeRequest{Key: op.key, RangeEnd: op.end, PrevKv: op.prevKV}
|
||||
resp, err = kv.remote.DeleteRange(ctx, r)
|
||||
if err == nil {
|
||||
return OpResponse{del: (*DeleteResponse)(resp)}, nil
|
||||
}
|
||||
|
@ -110,7 +110,7 @@ func NewLease(c *Client) Lease {
|
||||
l := &lessor{
|
||||
donec: make(chan struct{}),
|
||||
keepAlives: make(map[LeaseID]*keepAlive),
|
||||
remote: pb.NewLeaseClient(c.conn),
|
||||
remote: RetryLeaseClient(c),
|
||||
firstKeepAliveTimeout: c.cfg.DialTimeout + time.Second,
|
||||
}
|
||||
if l.firstKeepAliveTimeout == time.Second {
|
||||
@ -130,7 +130,7 @@ func (l *lessor) Grant(ctx context.Context, ttl int64) (*LeaseGrantResponse, err
|
||||
|
||||
for {
|
||||
r := &pb.LeaseGrantRequest{TTL: ttl}
|
||||
resp, err := l.remote.LeaseGrant(cctx, r, grpc.FailFast(false))
|
||||
resp, err := l.remote.LeaseGrant(cctx, r)
|
||||
if err == nil {
|
||||
gresp := &LeaseGrantResponse{
|
||||
ResponseHeader: resp.GetHeader(),
|
||||
@ -156,7 +156,7 @@ func (l *lessor) Revoke(ctx context.Context, id LeaseID) (*LeaseRevokeResponse,
|
||||
|
||||
for {
|
||||
r := &pb.LeaseRevokeRequest{ID: int64(id)}
|
||||
resp, err := l.remote.LeaseRevoke(cctx, r, grpc.FailFast(false))
|
||||
resp, err := l.remote.LeaseRevoke(cctx, r)
|
||||
|
||||
if err == nil {
|
||||
return (*LeaseRevokeResponse)(resp), nil
|
||||
|
@ -47,6 +47,9 @@ type Op struct {
|
||||
// for range, watch
|
||||
rev int64
|
||||
|
||||
// for watch, put, delete
|
||||
prevKV bool
|
||||
|
||||
// progressNotify is for progress updates.
|
||||
progressNotify bool
|
||||
|
||||
@ -73,10 +76,10 @@ func (op Op) toRequestOp() *pb.RequestOp {
|
||||
}
|
||||
return &pb.RequestOp{Request: &pb.RequestOp_RequestRange{RequestRange: r}}
|
||||
case tPut:
|
||||
r := &pb.PutRequest{Key: op.key, Value: op.val, Lease: int64(op.leaseID)}
|
||||
r := &pb.PutRequest{Key: op.key, Value: op.val, Lease: int64(op.leaseID), PrevKv: op.prevKV}
|
||||
return &pb.RequestOp{Request: &pb.RequestOp_RequestPut{RequestPut: r}}
|
||||
case tDeleteRange:
|
||||
r := &pb.DeleteRangeRequest{Key: op.key, RangeEnd: op.end}
|
||||
r := &pb.DeleteRangeRequest{Key: op.key, RangeEnd: op.end, PrevKv: op.prevKV}
|
||||
return &pb.RequestOp{Request: &pb.RequestOp_RequestDeleteRange{RequestDeleteRange: r}}
|
||||
default:
|
||||
panic("Unknown Op")
|
||||
@ -271,3 +274,11 @@ func WithProgressNotify() OpOption {
|
||||
op.progressNotify = true
|
||||
}
|
||||
}
|
||||
|
||||
// WithPrevKV gets the previous key-value pair before the event happens. If the previous KV is already compacted,
|
||||
// nothing will be returned.
|
||||
func WithPrevKV() OpOption {
|
||||
return func(op *Op) {
|
||||
op.prevKV = true
|
||||
}
|
||||
}
|
||||
|
243
clientv3/retry.go
Normal file
243
clientv3/retry.go
Normal file
@ -0,0 +1,243 @@
|
||||
// Copyright 2016 The etcd Authors
|
||||
//
|
||||
// Licensed 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 clientv3
|
||||
|
||||
import (
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
type rpcFunc func(ctx context.Context) error
|
||||
type retryRpcFunc func(context.Context, rpcFunc)
|
||||
|
||||
func (c *Client) newRetryWrapper() retryRpcFunc {
|
||||
return func(rpcCtx context.Context, f rpcFunc) {
|
||||
for {
|
||||
err := f(rpcCtx)
|
||||
// ignore grpc conn closing on fail-fast calls; they are transient errors
|
||||
if err == nil || !isConnClosing(err) {
|
||||
return
|
||||
}
|
||||
select {
|
||||
case <-c.balancer.ConnectNotify():
|
||||
case <-rpcCtx.Done():
|
||||
case <-c.ctx.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
type retryKVClient struct {
|
||||
pb.KVClient
|
||||
retryf retryRpcFunc
|
||||
}
|
||||
|
||||
// RetryKVClient implements a KVClient that uses the client's FailFast retry policy.
|
||||
func RetryKVClient(c *Client) pb.KVClient {
|
||||
return &retryKVClient{pb.NewKVClient(c.conn), c.retryWrapper}
|
||||
}
|
||||
|
||||
func (rkv *retryKVClient) Put(ctx context.Context, in *pb.PutRequest, opts ...grpc.CallOption) (resp *pb.PutResponse, err error) {
|
||||
rkv.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rkv.KVClient.Put(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rkv *retryKVClient) DeleteRange(ctx context.Context, in *pb.DeleteRangeRequest, opts ...grpc.CallOption) (resp *pb.DeleteRangeResponse, err error) {
|
||||
rkv.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rkv.KVClient.DeleteRange(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rkv *retryKVClient) Txn(ctx context.Context, in *pb.TxnRequest, opts ...grpc.CallOption) (resp *pb.TxnResponse, err error) {
|
||||
rkv.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rkv.KVClient.Txn(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rkv *retryKVClient) Compact(ctx context.Context, in *pb.CompactionRequest, opts ...grpc.CallOption) (resp *pb.CompactionResponse, err error) {
|
||||
rkv.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rkv.KVClient.Compact(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
type retryLeaseClient struct {
|
||||
pb.LeaseClient
|
||||
retryf retryRpcFunc
|
||||
}
|
||||
|
||||
// RetryLeaseClient implements a LeaseClient that uses the client's FailFast retry policy.
|
||||
func RetryLeaseClient(c *Client) pb.LeaseClient {
|
||||
return &retryLeaseClient{pb.NewLeaseClient(c.conn), c.retryWrapper}
|
||||
}
|
||||
|
||||
func (rlc *retryLeaseClient) LeaseGrant(ctx context.Context, in *pb.LeaseGrantRequest, opts ...grpc.CallOption) (resp *pb.LeaseGrantResponse, err error) {
|
||||
rlc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rlc.LeaseClient.LeaseGrant(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
|
||||
}
|
||||
|
||||
func (rlc *retryLeaseClient) LeaseRevoke(ctx context.Context, in *pb.LeaseRevokeRequest, opts ...grpc.CallOption) (resp *pb.LeaseRevokeResponse, err error) {
|
||||
rlc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rlc.LeaseClient.LeaseRevoke(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
type retryClusterClient struct {
|
||||
pb.ClusterClient
|
||||
retryf retryRpcFunc
|
||||
}
|
||||
|
||||
// RetryClusterClient implements a ClusterClient that uses the client's FailFast retry policy.
|
||||
func RetryClusterClient(c *Client) pb.ClusterClient {
|
||||
return &retryClusterClient{pb.NewClusterClient(c.conn), c.retryWrapper}
|
||||
}
|
||||
|
||||
func (rcc *retryClusterClient) MemberAdd(ctx context.Context, in *pb.MemberAddRequest, opts ...grpc.CallOption) (resp *pb.MemberAddResponse, err error) {
|
||||
rcc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rcc.ClusterClient.MemberAdd(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rcc *retryClusterClient) MemberRemove(ctx context.Context, in *pb.MemberRemoveRequest, opts ...grpc.CallOption) (resp *pb.MemberRemoveResponse, err error) {
|
||||
rcc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rcc.ClusterClient.MemberRemove(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rcc *retryClusterClient) MemberUpdate(ctx context.Context, in *pb.MemberUpdateRequest, opts ...grpc.CallOption) (resp *pb.MemberUpdateResponse, err error) {
|
||||
rcc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rcc.ClusterClient.MemberUpdate(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
type retryAuthClient struct {
|
||||
pb.AuthClient
|
||||
retryf retryRpcFunc
|
||||
}
|
||||
|
||||
// RetryAuthClient implements a AuthClient that uses the client's FailFast retry policy.
|
||||
func RetryAuthClient(c *Client) pb.AuthClient {
|
||||
return &retryAuthClient{pb.NewAuthClient(c.conn), c.retryWrapper}
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) AuthEnable(ctx context.Context, in *pb.AuthEnableRequest, opts ...grpc.CallOption) (resp *pb.AuthEnableResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.AuthEnable(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) AuthDisable(ctx context.Context, in *pb.AuthDisableRequest, opts ...grpc.CallOption) (resp *pb.AuthDisableResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.AuthDisable(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserAdd(ctx context.Context, in *pb.AuthUserAddRequest, opts ...grpc.CallOption) (resp *pb.AuthUserAddResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.UserAdd(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserDelete(ctx context.Context, in *pb.AuthUserDeleteRequest, opts ...grpc.CallOption) (resp *pb.AuthUserDeleteResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.UserDelete(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserChangePassword(ctx context.Context, in *pb.AuthUserChangePasswordRequest, opts ...grpc.CallOption) (resp *pb.AuthUserChangePasswordResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.UserChangePassword(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserGrantRole(ctx context.Context, in *pb.AuthUserGrantRoleRequest, opts ...grpc.CallOption) (resp *pb.AuthUserGrantRoleResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.UserGrantRole(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserRevokeRole(ctx context.Context, in *pb.AuthUserRevokeRoleRequest, opts ...grpc.CallOption) (resp *pb.AuthUserRevokeRoleResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.UserRevokeRole(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) RoleAdd(ctx context.Context, in *pb.AuthRoleAddRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleAddResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.RoleAdd(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) RoleDelete(ctx context.Context, in *pb.AuthRoleDeleteRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleDeleteResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.RoleDelete(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) RoleGrantPermission(ctx context.Context, in *pb.AuthRoleGrantPermissionRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleGrantPermissionResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.RoleGrantPermission(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) RoleRevokePermission(ctx context.Context, in *pb.AuthRoleRevokePermissionRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleRevokePermissionResponse, err error) {
|
||||
rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.AuthClient.RoleRevokePermission(rctx, in, opts...)
|
||||
return err
|
||||
})
|
||||
return resp, err
|
||||
}
|
@ -19,7 +19,6 @@ import (
|
||||
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
// Txn is the interface that wraps mini-transactions.
|
||||
@ -153,7 +152,7 @@ func (txn *txn) Commit() (*TxnResponse, error) {
|
||||
|
||||
func (txn *txn) commit() (*TxnResponse, error) {
|
||||
r := &pb.TxnRequest{Compare: txn.cmps, Success: txn.sus, Failure: txn.fas}
|
||||
resp, err := txn.kv.remote.Txn(txn.ctx, r, grpc.FailFast(false))
|
||||
resp, err := txn.kv.remote.Txn(txn.ctx, r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -61,6 +61,9 @@ type WatchResponse struct {
|
||||
// the channel sends a final response that has Canceled set to true with a non-nil Err().
|
||||
Canceled bool
|
||||
|
||||
// created is used to indicate the creation of the watcher.
|
||||
created bool
|
||||
|
||||
closeErr error
|
||||
}
|
||||
|
||||
@ -89,7 +92,7 @@ func (wr *WatchResponse) Err() error {
|
||||
|
||||
// IsProgressNotify returns true if the WatchResponse is progress notification.
|
||||
func (wr *WatchResponse) IsProgressNotify() bool {
|
||||
return len(wr.Events) == 0 && !wr.Canceled
|
||||
return len(wr.Events) == 0 && !wr.Canceled && !wr.created && wr.CompactRevision == 0 && wr.Header.Revision != 0
|
||||
}
|
||||
|
||||
// watcher implements the Watcher interface
|
||||
@ -102,6 +105,7 @@ type watcher struct {
|
||||
streams map[string]*watchGrpcStream
|
||||
}
|
||||
|
||||
// watchGrpcStream tracks all watch resources attached to a single grpc stream.
|
||||
type watchGrpcStream struct {
|
||||
owner *watcher
|
||||
remote pb.WatchClient
|
||||
@ -112,10 +116,10 @@ type watchGrpcStream struct {
|
||||
ctxKey string
|
||||
cancel context.CancelFunc
|
||||
|
||||
// mu protects the streams map
|
||||
mu sync.RWMutex
|
||||
// streams holds all active watchers
|
||||
streams map[int64]*watcherStream
|
||||
// substreams holds all active watchers on this grpc stream
|
||||
substreams map[int64]*watcherStream
|
||||
// resuming holds all resuming watchers on this grpc stream
|
||||
resuming []*watcherStream
|
||||
|
||||
// reqc sends a watch request from Watch() to the main goroutine
|
||||
reqc chan *watchRequest
|
||||
@ -127,8 +131,12 @@ type watchGrpcStream struct {
|
||||
donec chan struct{}
|
||||
// errc transmits errors from grpc Recv to the watch stream reconn logic
|
||||
errc chan error
|
||||
// closingc gets the watcherStream of closing watchers
|
||||
closingc chan *watcherStream
|
||||
|
||||
// the error that closed the watch stream
|
||||
// resumec closes to signal that all substreams should begin resuming
|
||||
resumec chan struct{}
|
||||
// closeErr is the error that closed the watch stream
|
||||
closeErr error
|
||||
}
|
||||
|
||||
@ -140,6 +148,8 @@ type watchRequest struct {
|
||||
rev int64
|
||||
// progressNotify is for progress updates.
|
||||
progressNotify bool
|
||||
// get the previous key-value pair before the event happens
|
||||
prevKV bool
|
||||
// retc receives a chan WatchResponse once the watcher is established
|
||||
retc chan chan WatchResponse
|
||||
}
|
||||
@ -150,15 +160,18 @@ type watcherStream struct {
|
||||
initReq watchRequest
|
||||
|
||||
// outc publishes watch responses to subscriber
|
||||
outc chan<- WatchResponse
|
||||
outc chan WatchResponse
|
||||
// recvc buffers watch responses before publishing
|
||||
recvc chan *WatchResponse
|
||||
id int64
|
||||
// donec closes when the watcherStream goroutine stops.
|
||||
donec chan struct{}
|
||||
// closing is set to true when stream should be scheduled to shutdown.
|
||||
closing bool
|
||||
// id is the registered watch id on the grpc stream
|
||||
id int64
|
||||
|
||||
// lastRev is revision last successfully sent over outc
|
||||
lastRev int64
|
||||
// resumec indicates the stream must recover at a given revision
|
||||
resumec chan int64
|
||||
// buf holds all events received from etcd but not yet consumed by the client
|
||||
buf []*WatchResponse
|
||||
}
|
||||
|
||||
func NewWatcher(c *Client) Watcher {
|
||||
@ -182,18 +195,20 @@ func (vc *valCtx) Err() error { return nil }
|
||||
func (w *watcher) newWatcherGrpcStream(inctx context.Context) *watchGrpcStream {
|
||||
ctx, cancel := context.WithCancel(&valCtx{inctx})
|
||||
wgs := &watchGrpcStream{
|
||||
owner: w,
|
||||
remote: w.remote,
|
||||
ctx: ctx,
|
||||
ctxKey: fmt.Sprintf("%v", inctx),
|
||||
cancel: cancel,
|
||||
streams: make(map[int64]*watcherStream),
|
||||
owner: w,
|
||||
remote: w.remote,
|
||||
ctx: ctx,
|
||||
ctxKey: fmt.Sprintf("%v", inctx),
|
||||
cancel: cancel,
|
||||
substreams: make(map[int64]*watcherStream),
|
||||
|
||||
respc: make(chan *pb.WatchResponse),
|
||||
reqc: make(chan *watchRequest),
|
||||
stopc: make(chan struct{}),
|
||||
donec: make(chan struct{}),
|
||||
errc: make(chan error, 1),
|
||||
respc: make(chan *pb.WatchResponse),
|
||||
reqc: make(chan *watchRequest),
|
||||
stopc: make(chan struct{}),
|
||||
donec: make(chan struct{}),
|
||||
errc: make(chan error, 1),
|
||||
closingc: make(chan *watcherStream),
|
||||
resumec: make(chan struct{}),
|
||||
}
|
||||
go wgs.run()
|
||||
return wgs
|
||||
@ -203,14 +218,14 @@ func (w *watcher) newWatcherGrpcStream(inctx context.Context) *watchGrpcStream {
|
||||
func (w *watcher) Watch(ctx context.Context, key string, opts ...OpOption) WatchChan {
|
||||
ow := opWatch(key, opts...)
|
||||
|
||||
retc := make(chan chan WatchResponse, 1)
|
||||
wr := &watchRequest{
|
||||
ctx: ctx,
|
||||
key: string(ow.key),
|
||||
end: string(ow.end),
|
||||
rev: ow.rev,
|
||||
progressNotify: ow.progressNotify,
|
||||
retc: retc,
|
||||
prevKV: ow.prevKV,
|
||||
retc: make(chan chan WatchResponse, 1),
|
||||
}
|
||||
|
||||
ok := false
|
||||
@ -254,7 +269,7 @@ func (w *watcher) Watch(ctx context.Context, key string, opts ...OpOption) Watch
|
||||
// receive channel
|
||||
if ok {
|
||||
select {
|
||||
case ret := <-retc:
|
||||
case ret := <-wr.retc:
|
||||
return ret
|
||||
case <-ctx.Done():
|
||||
case <-donec:
|
||||
@ -294,65 +309,57 @@ func (w *watchGrpcStream) Close() (err error) {
|
||||
return toErr(w.ctx, err)
|
||||
}
|
||||
|
||||
func (w *watchGrpcStream) addStream(resp *pb.WatchResponse, pendingReq *watchRequest) {
|
||||
if pendingReq == nil {
|
||||
// no pending request; ignore
|
||||
return
|
||||
}
|
||||
if resp.Canceled || resp.CompactRevision != 0 {
|
||||
// a cancel at id creation time means the start revision has
|
||||
// been compacted out of the store
|
||||
ret := make(chan WatchResponse, 1)
|
||||
ret <- WatchResponse{
|
||||
Header: *resp.Header,
|
||||
CompactRevision: resp.CompactRevision,
|
||||
Canceled: true}
|
||||
close(ret)
|
||||
pendingReq.retc <- ret
|
||||
return
|
||||
}
|
||||
|
||||
ret := make(chan WatchResponse)
|
||||
if resp.WatchId == -1 {
|
||||
// failed; no channel
|
||||
close(ret)
|
||||
pendingReq.retc <- ret
|
||||
return
|
||||
}
|
||||
|
||||
ws := &watcherStream{
|
||||
initReq: *pendingReq,
|
||||
id: resp.WatchId,
|
||||
outc: ret,
|
||||
// buffered so unlikely to block on sending while holding mu
|
||||
recvc: make(chan *WatchResponse, 4),
|
||||
resumec: make(chan int64),
|
||||
}
|
||||
|
||||
if pendingReq.rev == 0 {
|
||||
// note the header revision so that a put following a current watcher
|
||||
// disconnect will arrive on the watcher channel after reconnect
|
||||
ws.initReq.rev = resp.Header.Revision
|
||||
}
|
||||
|
||||
func (w *watcher) closeStream(wgs *watchGrpcStream) {
|
||||
w.mu.Lock()
|
||||
w.streams[ws.id] = ws
|
||||
close(wgs.donec)
|
||||
wgs.cancel()
|
||||
if w.streams != nil {
|
||||
delete(w.streams, wgs.ctxKey)
|
||||
}
|
||||
w.mu.Unlock()
|
||||
|
||||
// pass back the subscriber channel for the watcher
|
||||
pendingReq.retc <- ret
|
||||
|
||||
// send messages to subscriber
|
||||
go w.serveStream(ws)
|
||||
}
|
||||
|
||||
// closeStream closes the watcher resources and removes it
|
||||
func (w *watchGrpcStream) closeStream(ws *watcherStream) {
|
||||
// cancels request stream; subscriber receives nil channel
|
||||
close(ws.initReq.retc)
|
||||
// close subscriber's channel
|
||||
func (w *watchGrpcStream) addSubstream(resp *pb.WatchResponse, ws *watcherStream) {
|
||||
if resp.WatchId == -1 {
|
||||
// failed; no channel
|
||||
close(ws.recvc)
|
||||
return
|
||||
}
|
||||
ws.id = resp.WatchId
|
||||
w.substreams[ws.id] = ws
|
||||
}
|
||||
|
||||
func (w *watchGrpcStream) sendCloseSubstream(ws *watcherStream, resp *WatchResponse) {
|
||||
select {
|
||||
case ws.outc <- *resp:
|
||||
case <-ws.initReq.ctx.Done():
|
||||
case <-time.After(closeSendErrTimeout):
|
||||
}
|
||||
close(ws.outc)
|
||||
delete(w.streams, ws.id)
|
||||
}
|
||||
|
||||
func (w *watchGrpcStream) closeSubstream(ws *watcherStream) {
|
||||
// send channel response in case stream was never established
|
||||
select {
|
||||
case ws.initReq.retc <- ws.outc:
|
||||
default:
|
||||
}
|
||||
// close subscriber's channel
|
||||
if closeErr := w.closeErr; closeErr != nil && ws.initReq.ctx.Err() == nil {
|
||||
go w.sendCloseSubstream(ws, &WatchResponse{closeErr: w.closeErr})
|
||||
} else {
|
||||
close(ws.outc)
|
||||
}
|
||||
if ws.id != -1 {
|
||||
delete(w.substreams, ws.id)
|
||||
return
|
||||
}
|
||||
for i := range w.resuming {
|
||||
if w.resuming[i] == ws {
|
||||
w.resuming[i] = nil
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// run is the root of the goroutines for managing a watcher client
|
||||
@ -360,15 +367,29 @@ func (w *watchGrpcStream) run() {
|
||||
var wc pb.Watch_WatchClient
|
||||
var closeErr error
|
||||
|
||||
// substreams marked to close but goroutine still running; needed for
|
||||
// avoiding double-closing recvc on grpc stream teardown
|
||||
closing := make(map[*watcherStream]struct{})
|
||||
|
||||
defer func() {
|
||||
w.owner.mu.Lock()
|
||||
w.closeErr = closeErr
|
||||
if w.owner.streams != nil {
|
||||
delete(w.owner.streams, w.ctxKey)
|
||||
// shutdown substreams and resuming substreams
|
||||
for _, ws := range w.substreams {
|
||||
if _, ok := closing[ws]; !ok {
|
||||
close(ws.recvc)
|
||||
}
|
||||
}
|
||||
close(w.donec)
|
||||
w.owner.mu.Unlock()
|
||||
w.cancel()
|
||||
for _, ws := range w.resuming {
|
||||
if _, ok := closing[ws]; ws != nil && !ok {
|
||||
close(ws.recvc)
|
||||
}
|
||||
}
|
||||
w.joinSubstreams()
|
||||
for toClose := len(w.substreams) + len(w.resuming); toClose > 0; toClose-- {
|
||||
w.closeSubstream(<-w.closingc)
|
||||
}
|
||||
|
||||
w.owner.closeStream(w)
|
||||
}()
|
||||
|
||||
// start a stream with the etcd grpc server
|
||||
@ -376,42 +397,49 @@ func (w *watchGrpcStream) run() {
|
||||
return
|
||||
}
|
||||
|
||||
var pendingReq, failedReq *watchRequest
|
||||
curReqC := w.reqc
|
||||
cancelSet := make(map[int64]struct{})
|
||||
|
||||
for {
|
||||
select {
|
||||
// Watch() requested
|
||||
case pendingReq = <-curReqC:
|
||||
// no more watch requests until there's a response
|
||||
curReqC = nil
|
||||
if err := wc.Send(pendingReq.toPB()); err == nil {
|
||||
// pendingReq now waits on w.respc
|
||||
break
|
||||
case wreq := <-w.reqc:
|
||||
outc := make(chan WatchResponse, 1)
|
||||
ws := &watcherStream{
|
||||
initReq: *wreq,
|
||||
id: -1,
|
||||
outc: outc,
|
||||
// unbufffered so resumes won't cause repeat events
|
||||
recvc: make(chan *WatchResponse),
|
||||
}
|
||||
|
||||
ws.donec = make(chan struct{})
|
||||
go w.serveSubstream(ws, w.resumec)
|
||||
|
||||
// queue up for watcher creation/resume
|
||||
w.resuming = append(w.resuming, ws)
|
||||
if len(w.resuming) == 1 {
|
||||
// head of resume queue, can register a new watcher
|
||||
wc.Send(ws.initReq.toPB())
|
||||
}
|
||||
failedReq = pendingReq
|
||||
// New events from the watch client
|
||||
case pbresp := <-w.respc:
|
||||
switch {
|
||||
case pbresp.Created:
|
||||
// response to pending req, try to add
|
||||
w.addStream(pbresp, pendingReq)
|
||||
pendingReq = nil
|
||||
curReqC = w.reqc
|
||||
// response to head of queue creation
|
||||
if ws := w.resuming[0]; ws != nil {
|
||||
w.addSubstream(pbresp, ws)
|
||||
w.dispatchEvent(pbresp)
|
||||
w.resuming[0] = nil
|
||||
}
|
||||
if ws := w.nextResume(); ws != nil {
|
||||
wc.Send(ws.initReq.toPB())
|
||||
}
|
||||
case pbresp.Canceled:
|
||||
delete(cancelSet, pbresp.WatchId)
|
||||
// shutdown serveStream, if any
|
||||
w.mu.Lock()
|
||||
if ws, ok := w.streams[pbresp.WatchId]; ok {
|
||||
if ws, ok := w.substreams[pbresp.WatchId]; ok {
|
||||
// signal to stream goroutine to update closingc
|
||||
close(ws.recvc)
|
||||
delete(w.streams, ws.id)
|
||||
}
|
||||
numStreams := len(w.streams)
|
||||
w.mu.Unlock()
|
||||
if numStreams == 0 {
|
||||
// don't leak watcher streams
|
||||
return
|
||||
closing[ws] = struct{}{}
|
||||
}
|
||||
default:
|
||||
// dispatch to appropriate watch stream
|
||||
@ -432,7 +460,6 @@ func (w *watchGrpcStream) run() {
|
||||
wc.Send(req)
|
||||
}
|
||||
// watch client failed to recv; spawn another if possible
|
||||
// TODO report watch client errors from errc?
|
||||
case err := <-w.errc:
|
||||
if toErr(w.ctx, err) == v3rpc.ErrNoLeader {
|
||||
closeErr = err
|
||||
@ -441,48 +468,58 @@ func (w *watchGrpcStream) run() {
|
||||
if wc, closeErr = w.newWatchClient(); closeErr != nil {
|
||||
return
|
||||
}
|
||||
curReqC = w.reqc
|
||||
if pendingReq != nil {
|
||||
failedReq = pendingReq
|
||||
if ws := w.nextResume(); ws != nil {
|
||||
wc.Send(ws.initReq.toPB())
|
||||
}
|
||||
cancelSet = make(map[int64]struct{})
|
||||
case <-w.stopc:
|
||||
return
|
||||
}
|
||||
|
||||
// send failed; queue for retry
|
||||
if failedReq != nil {
|
||||
go func(wr *watchRequest) {
|
||||
select {
|
||||
case w.reqc <- wr:
|
||||
case <-wr.ctx.Done():
|
||||
case <-w.donec:
|
||||
}
|
||||
}(pendingReq)
|
||||
failedReq = nil
|
||||
pendingReq = nil
|
||||
case ws := <-w.closingc:
|
||||
w.closeSubstream(ws)
|
||||
delete(closing, ws)
|
||||
if len(w.substreams)+len(w.resuming) == 0 {
|
||||
// no more watchers on this stream, shutdown
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// nextResume chooses the next resuming to register with the grpc stream. Abandoned
|
||||
// streams are marked as nil in the queue since the head must wait for its inflight registration.
|
||||
func (w *watchGrpcStream) nextResume() *watcherStream {
|
||||
for len(w.resuming) != 0 {
|
||||
if w.resuming[0] != nil {
|
||||
return w.resuming[0]
|
||||
}
|
||||
w.resuming = w.resuming[1:len(w.resuming)]
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// dispatchEvent sends a WatchResponse to the appropriate watcher stream
|
||||
func (w *watchGrpcStream) dispatchEvent(pbresp *pb.WatchResponse) bool {
|
||||
w.mu.RLock()
|
||||
defer w.mu.RUnlock()
|
||||
ws, ok := w.streams[pbresp.WatchId]
|
||||
ws, ok := w.substreams[pbresp.WatchId]
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
events := make([]*Event, len(pbresp.Events))
|
||||
for i, ev := range pbresp.Events {
|
||||
events[i] = (*Event)(ev)
|
||||
}
|
||||
if ok {
|
||||
wr := &WatchResponse{
|
||||
Header: *pbresp.Header,
|
||||
Events: events,
|
||||
CompactRevision: pbresp.CompactRevision,
|
||||
Canceled: pbresp.Canceled}
|
||||
ws.recvc <- wr
|
||||
wr := &WatchResponse{
|
||||
Header: *pbresp.Header,
|
||||
Events: events,
|
||||
CompactRevision: pbresp.CompactRevision,
|
||||
created: pbresp.Created,
|
||||
Canceled: pbresp.Canceled,
|
||||
}
|
||||
return ok
|
||||
select {
|
||||
case ws.recvc <- wr:
|
||||
case <-ws.donec:
|
||||
return false
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
// serveWatchClient forwards messages from the grpc stream to run()
|
||||
@ -504,113 +541,111 @@ func (w *watchGrpcStream) serveWatchClient(wc pb.Watch_WatchClient) {
|
||||
}
|
||||
}
|
||||
|
||||
// serveStream forwards watch responses from run() to the subscriber
|
||||
func (w *watchGrpcStream) serveStream(ws *watcherStream) {
|
||||
var closeErr error
|
||||
emptyWr := &WatchResponse{}
|
||||
wrs := []*WatchResponse{}
|
||||
// serveSubstream forwards watch responses from run() to the subscriber
|
||||
func (w *watchGrpcStream) serveSubstream(ws *watcherStream, resumec chan struct{}) {
|
||||
if ws.closing {
|
||||
panic("created substream goroutine but substream is closing")
|
||||
}
|
||||
|
||||
// nextRev is the minimum expected next revision
|
||||
nextRev := ws.initReq.rev
|
||||
resuming := false
|
||||
closing := false
|
||||
for !closing {
|
||||
defer func() {
|
||||
if !resuming {
|
||||
ws.closing = true
|
||||
}
|
||||
close(ws.donec)
|
||||
if !resuming {
|
||||
w.closingc <- ws
|
||||
}
|
||||
}()
|
||||
|
||||
emptyWr := &WatchResponse{}
|
||||
for {
|
||||
curWr := emptyWr
|
||||
outc := ws.outc
|
||||
if len(wrs) > 0 {
|
||||
curWr = wrs[0]
|
||||
|
||||
if len(ws.buf) > 0 && ws.buf[0].created {
|
||||
select {
|
||||
case ws.initReq.retc <- ws.outc:
|
||||
default:
|
||||
}
|
||||
ws.buf = ws.buf[1:]
|
||||
}
|
||||
|
||||
if len(ws.buf) > 0 {
|
||||
curWr = ws.buf[0]
|
||||
} else {
|
||||
outc = nil
|
||||
}
|
||||
select {
|
||||
case outc <- *curWr:
|
||||
if wrs[0].Err() != nil {
|
||||
closing = true
|
||||
break
|
||||
}
|
||||
var newRev int64
|
||||
if len(wrs[0].Events) > 0 {
|
||||
newRev = wrs[0].Events[len(wrs[0].Events)-1].Kv.ModRevision
|
||||
} else {
|
||||
newRev = wrs[0].Header.Revision
|
||||
}
|
||||
if newRev != ws.lastRev {
|
||||
ws.lastRev = newRev
|
||||
}
|
||||
wrs[0] = nil
|
||||
wrs = wrs[1:]
|
||||
case wr, ok := <-ws.recvc:
|
||||
if !ok {
|
||||
// shutdown from closeStream
|
||||
if ws.buf[0].Err() != nil {
|
||||
return
|
||||
}
|
||||
// resume up to last seen event if disconnected
|
||||
if resuming && wr.Err() == nil {
|
||||
resuming = false
|
||||
// trim events already seen
|
||||
for i := 0; i < len(wr.Events); i++ {
|
||||
if wr.Events[i].Kv.ModRevision > ws.lastRev {
|
||||
wr.Events = wr.Events[i:]
|
||||
break
|
||||
}
|
||||
}
|
||||
// only forward new events
|
||||
if wr.Events[0].Kv.ModRevision == ws.lastRev {
|
||||
break
|
||||
}
|
||||
ws.buf[0] = nil
|
||||
ws.buf = ws.buf[1:]
|
||||
case wr, ok := <-ws.recvc:
|
||||
if !ok {
|
||||
// shutdown from closeSubstream
|
||||
return
|
||||
}
|
||||
resuming = false
|
||||
// TODO don't keep buffering if subscriber stops reading
|
||||
wrs = append(wrs, wr)
|
||||
case resumeRev := <-ws.resumec:
|
||||
wrs = nil
|
||||
resuming = true
|
||||
if resumeRev == -1 {
|
||||
// pause serving stream while resume gets set up
|
||||
break
|
||||
// TODO pause channel if buffer gets too large
|
||||
ws.buf = append(ws.buf, wr)
|
||||
nextRev = wr.Header.Revision
|
||||
if len(wr.Events) > 0 {
|
||||
nextRev = wr.Events[len(wr.Events)-1].Kv.ModRevision + 1
|
||||
}
|
||||
if resumeRev != ws.lastRev {
|
||||
panic("unexpected resume revision")
|
||||
}
|
||||
case <-w.donec:
|
||||
closing = true
|
||||
closeErr = w.closeErr
|
||||
ws.initReq.rev = nextRev
|
||||
case <-ws.initReq.ctx.Done():
|
||||
closing = true
|
||||
return
|
||||
case <-resumec:
|
||||
resuming = true
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// try to send off close error
|
||||
if closeErr != nil {
|
||||
select {
|
||||
case ws.outc <- WatchResponse{closeErr: w.closeErr}:
|
||||
case <-w.donec:
|
||||
case <-time.After(closeSendErrTimeout):
|
||||
}
|
||||
}
|
||||
|
||||
w.mu.Lock()
|
||||
w.closeStream(ws)
|
||||
w.mu.Unlock()
|
||||
// lazily send cancel message if events on missing id
|
||||
}
|
||||
|
||||
func (w *watchGrpcStream) newWatchClient() (pb.Watch_WatchClient, error) {
|
||||
ws, rerr := w.resume()
|
||||
if rerr != nil {
|
||||
return nil, rerr
|
||||
// connect to grpc stream
|
||||
wc, err := w.openWatchClient()
|
||||
if err != nil {
|
||||
return nil, v3rpc.Error(err)
|
||||
}
|
||||
go w.serveWatchClient(ws)
|
||||
return ws, nil
|
||||
}
|
||||
|
||||
// resume creates a new WatchClient with all current watchers reestablished
|
||||
func (w *watchGrpcStream) resume() (ws pb.Watch_WatchClient, err error) {
|
||||
for {
|
||||
if ws, err = w.openWatchClient(); err != nil {
|
||||
break
|
||||
} else if err = w.resumeWatchers(ws); err == nil {
|
||||
break
|
||||
// mark all substreams as resuming
|
||||
if len(w.substreams)+len(w.resuming) > 0 {
|
||||
close(w.resumec)
|
||||
w.resumec = make(chan struct{})
|
||||
w.joinSubstreams()
|
||||
for _, ws := range w.substreams {
|
||||
ws.id = -1
|
||||
w.resuming = append(w.resuming, ws)
|
||||
}
|
||||
for _, ws := range w.resuming {
|
||||
if ws == nil || ws.closing {
|
||||
continue
|
||||
}
|
||||
ws.donec = make(chan struct{})
|
||||
go w.serveSubstream(ws, w.resumec)
|
||||
}
|
||||
}
|
||||
w.substreams = make(map[int64]*watcherStream)
|
||||
// receive data from new grpc stream
|
||||
go w.serveWatchClient(wc)
|
||||
return wc, nil
|
||||
}
|
||||
|
||||
// joinSubstream waits for all substream goroutines to complete
|
||||
func (w *watchGrpcStream) joinSubstreams() {
|
||||
for _, ws := range w.substreams {
|
||||
<-ws.donec
|
||||
}
|
||||
for _, ws := range w.resuming {
|
||||
if ws != nil {
|
||||
<-ws.donec
|
||||
}
|
||||
}
|
||||
return ws, v3rpc.Error(err)
|
||||
}
|
||||
|
||||
// openWatchClient retries opening a watchclient until retryConnection fails
|
||||
@ -619,7 +654,7 @@ func (w *watchGrpcStream) openWatchClient() (ws pb.Watch_WatchClient, err error)
|
||||
select {
|
||||
case <-w.stopc:
|
||||
if err == nil {
|
||||
err = context.Canceled
|
||||
return nil, context.Canceled
|
||||
}
|
||||
return nil, err
|
||||
default:
|
||||
@ -634,48 +669,6 @@ func (w *watchGrpcStream) openWatchClient() (ws pb.Watch_WatchClient, err error)
|
||||
return ws, nil
|
||||
}
|
||||
|
||||
// resumeWatchers rebuilds every registered watcher on a new client
|
||||
func (w *watchGrpcStream) resumeWatchers(wc pb.Watch_WatchClient) error {
|
||||
w.mu.RLock()
|
||||
streams := make([]*watcherStream, 0, len(w.streams))
|
||||
for _, ws := range w.streams {
|
||||
streams = append(streams, ws)
|
||||
}
|
||||
w.mu.RUnlock()
|
||||
|
||||
for _, ws := range streams {
|
||||
// pause serveStream
|
||||
ws.resumec <- -1
|
||||
|
||||
// reconstruct watcher from initial request
|
||||
if ws.lastRev != 0 {
|
||||
ws.initReq.rev = ws.lastRev
|
||||
}
|
||||
if err := wc.Send(ws.initReq.toPB()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// wait for request ack
|
||||
resp, err := wc.Recv()
|
||||
if err != nil {
|
||||
return err
|
||||
} else if len(resp.Events) != 0 || !resp.Created {
|
||||
return fmt.Errorf("watcher: unexpected response (%+v)", resp)
|
||||
}
|
||||
|
||||
// id may be different since new remote watcher; update map
|
||||
w.mu.Lock()
|
||||
delete(w.streams, ws.id)
|
||||
ws.id = resp.WatchId
|
||||
w.streams[ws.id] = ws
|
||||
w.mu.Unlock()
|
||||
|
||||
// unpause serveStream
|
||||
ws.resumec <- ws.lastRev
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// toPB converts an internal watch request structure to its protobuf messagefunc (wr *watchRequest)
|
||||
func (wr *watchRequest) toPB() *pb.WatchRequest {
|
||||
req := &pb.WatchCreateRequest{
|
||||
@ -683,6 +676,7 @@ func (wr *watchRequest) toPB() *pb.WatchRequest {
|
||||
Key: []byte(wr.key),
|
||||
RangeEnd: []byte(wr.end),
|
||||
ProgressNotify: wr.progressNotify,
|
||||
PrevKv: wr.prevKV,
|
||||
}
|
||||
cr := &pb.WatchRequest_CreateRequest{CreateRequest: req}
|
||||
return &pb.WatchRequest{RequestUnion: cr}
|
||||
|
42
cmd/Godeps/Godeps.json
generated
42
cmd/Godeps/Godeps.json
generated
@ -1,6 +1,6 @@
|
||||
{
|
||||
"ImportPath": "github.com/coreos/etcd",
|
||||
"GoVersion": "devel-a6dbfc1",
|
||||
"GoVersion": "go1.7",
|
||||
"GodepVersion": "v74",
|
||||
"Packages": [
|
||||
"./..."
|
||||
@ -25,8 +25,8 @@
|
||||
},
|
||||
{
|
||||
"ImportPath": "github.com/boltdb/bolt",
|
||||
"Comment": "v1.2.1",
|
||||
"Rev": "dfb21201d9270c1082d5fb0f07f500311ff72f18"
|
||||
"Comment": "v1.3.0",
|
||||
"Rev": "583e8937c61f1af6513608ccc75c97b6abdf4ff9"
|
||||
},
|
||||
{
|
||||
"ImportPath": "github.com/cockroachdb/cmux",
|
||||
@ -237,48 +237,48 @@
|
||||
},
|
||||
{
|
||||
"ImportPath": "google.golang.org/grpc",
|
||||
"Comment": "v1.0.0-6-g02fca89",
|
||||
"Rev": "02fca896ff5f50c6bbbee0860345a49344b37a03"
|
||||
"Comment": "v1.0.0-183-g231b4cf",
|
||||
"Rev": "231b4cfea0e79843053a33f5fe90bd4d84b23cd3"
|
||||
},
|
||||
{
|
||||
"ImportPath": "google.golang.org/grpc/codes",
|
||||
"Comment": "v1.0.0-6-g02fca89",
|
||||
"Rev": "02fca896ff5f50c6bbbee0860345a49344b37a03"
|
||||
"Comment": "v1.0.0-183-g231b4cf",
|
||||
"Rev": "231b4cfea0e79843053a33f5fe90bd4d84b23cd3"
|
||||
},
|
||||
{
|
||||
"ImportPath": "google.golang.org/grpc/credentials",
|
||||
"Comment": "v1.0.0-6-g02fca89",
|
||||
"Rev": "02fca896ff5f50c6bbbee0860345a49344b37a03"
|
||||
"Comment": "v1.0.0-183-g231b4cf",
|
||||
"Rev": "231b4cfea0e79843053a33f5fe90bd4d84b23cd3"
|
||||
},
|
||||
{
|
||||
"ImportPath": "google.golang.org/grpc/grpclog",
|
||||
"Comment": "v1.0.0-6-g02fca89",
|
||||
"Rev": "02fca896ff5f50c6bbbee0860345a49344b37a03"
|
||||
"Comment": "v1.0.0-183-g231b4cf",
|
||||
"Rev": "231b4cfea0e79843053a33f5fe90bd4d84b23cd3"
|
||||
},
|
||||
{
|
||||
"ImportPath": "google.golang.org/grpc/internal",
|
||||
"Comment": "v1.0.0-6-g02fca89",
|
||||
"Rev": "02fca896ff5f50c6bbbee0860345a49344b37a03"
|
||||
"Comment": "v1.0.0-183-g231b4cf",
|
||||
"Rev": "231b4cfea0e79843053a33f5fe90bd4d84b23cd3"
|
||||
},
|
||||
{
|
||||
"ImportPath": "google.golang.org/grpc/metadata",
|
||||
"Comment": "v1.0.0-6-g02fca89",
|
||||
"Rev": "02fca896ff5f50c6bbbee0860345a49344b37a03"
|
||||
"Comment": "v1.0.0-183-g231b4cf",
|
||||
"Rev": "231b4cfea0e79843053a33f5fe90bd4d84b23cd3"
|
||||
},
|
||||
{
|
||||
"ImportPath": "google.golang.org/grpc/naming",
|
||||
"Comment": "v1.0.0-6-g02fca89",
|
||||
"Rev": "02fca896ff5f50c6bbbee0860345a49344b37a03"
|
||||
"Comment": "v1.0.0-183-g231b4cf",
|
||||
"Rev": "231b4cfea0e79843053a33f5fe90bd4d84b23cd3"
|
||||
},
|
||||
{
|
||||
"ImportPath": "google.golang.org/grpc/peer",
|
||||
"Comment": "v1.0.0-6-g02fca89",
|
||||
"Rev": "02fca896ff5f50c6bbbee0860345a49344b37a03"
|
||||
"Comment": "v1.0.0-183-g231b4cf",
|
||||
"Rev": "231b4cfea0e79843053a33f5fe90bd4d84b23cd3"
|
||||
},
|
||||
{
|
||||
"ImportPath": "google.golang.org/grpc/transport",
|
||||
"Comment": "v1.0.0-6-g02fca89",
|
||||
"Rev": "02fca896ff5f50c6bbbee0860345a49344b37a03"
|
||||
"Comment": "v1.0.0-183-g231b4cf",
|
||||
"Rev": "231b4cfea0e79843053a33f5fe90bd4d84b23cd3"
|
||||
},
|
||||
{
|
||||
"ImportPath": "gopkg.in/cheggaaa/pb.v1",
|
||||
|
12
cmd/vendor/github.com/boltdb/bolt/README.md
generated
vendored
12
cmd/vendor/github.com/boltdb/bolt/README.md
generated
vendored
@ -1,4 +1,4 @@
|
||||
Bolt [](https://coveralls.io/r/boltdb/bolt?branch=master) [](https://godoc.org/github.com/boltdb/bolt) 
|
||||
Bolt [](https://coveralls.io/r/boltdb/bolt?branch=master) [](https://godoc.org/github.com/boltdb/bolt) 
|
||||
====
|
||||
|
||||
Bolt is a pure Go key/value store inspired by [Howard Chu's][hyc_symas]
|
||||
@ -313,7 +313,7 @@ func (s *Store) CreateUser(u *User) error {
|
||||
// Generate ID for the user.
|
||||
// This returns an error only if the Tx is closed or not writeable.
|
||||
// That can't happen in an Update() call so I ignore the error check.
|
||||
id, _ = b.NextSequence()
|
||||
id, _ := b.NextSequence()
|
||||
u.ID = int(id)
|
||||
|
||||
// Marshal user data into bytes.
|
||||
@ -557,7 +557,7 @@ if err != nil {
|
||||
Bolt is able to run on mobile devices by leveraging the binding feature of the
|
||||
[gomobile](https://github.com/golang/mobile) tool. Create a struct that will
|
||||
contain your database logic and a reference to a `*bolt.DB` with a initializing
|
||||
contstructor that takes in a filepath where the database file will be stored.
|
||||
constructor that takes in a filepath where the database file will be stored.
|
||||
Neither Android nor iOS require extra permissions or cleanup from using this method.
|
||||
|
||||
```go
|
||||
@ -807,6 +807,7 @@ them via pull request.
|
||||
|
||||
Below is a list of public, open source projects that use Bolt:
|
||||
|
||||
* [BoltDbWeb](https://github.com/evnix/boltdbweb) - A web based GUI for BoltDB files.
|
||||
* [Operation Go: A Routine Mission](http://gocode.io) - An online programming game for Golang using Bolt for user accounts and a leaderboard.
|
||||
* [Bazil](https://bazil.org/) - A file system that lets your data reside where it is most convenient for it to reside.
|
||||
* [DVID](https://github.com/janelia-flyem/dvid) - Added Bolt as optional storage engine and testing it against Basho-tuned leveldb.
|
||||
@ -825,7 +826,6 @@ Below is a list of public, open source projects that use Bolt:
|
||||
* [cayley](https://github.com/google/cayley) - Cayley is an open-source graph database using Bolt as optional backend.
|
||||
* [bleve](http://www.blevesearch.com/) - A pure Go search engine similar to ElasticSearch that uses Bolt as the default storage backend.
|
||||
* [tentacool](https://github.com/optiflows/tentacool) - REST api server to manage system stuff (IP, DNS, Gateway...) on a linux server.
|
||||
* [SkyDB](https://github.com/skydb/sky) - Behavioral analytics database.
|
||||
* [Seaweed File System](https://github.com/chrislusf/seaweedfs) - Highly scalable distributed key~file system with O(1) disk read.
|
||||
* [InfluxDB](https://influxdata.com) - Scalable datastore for metrics, events, and real-time analytics.
|
||||
* [Freehold](http://tshannon.bitbucket.org/freehold/) - An open, secure, and lightweight platform for your files and data.
|
||||
@ -842,9 +842,11 @@ Below is a list of public, open source projects that use Bolt:
|
||||
* [Go Report Card](https://goreportcard.com/) - Go code quality report cards as a (free and open source) service.
|
||||
* [Boltdb Boilerplate](https://github.com/bobintornado/boltdb-boilerplate) - Boilerplate wrapper around bolt aiming to make simple calls one-liners.
|
||||
* [lru](https://github.com/crowdriff/lru) - Easy to use Bolt-backed Least-Recently-Used (LRU) read-through cache with chainable remote stores.
|
||||
* [Storm](https://github.com/asdine/storm) - A simple ORM around BoltDB.
|
||||
* [Storm](https://github.com/asdine/storm) - Simple and powerful ORM for BoltDB.
|
||||
* [GoWebApp](https://github.com/josephspurrier/gowebapp) - A basic MVC web application in Go using BoltDB.
|
||||
* [SimpleBolt](https://github.com/xyproto/simplebolt) - A simple way to use BoltDB. Deals mainly with strings.
|
||||
* [Algernon](https://github.com/xyproto/algernon) - A HTTP/2 web server with built-in support for Lua. Uses BoltDB as the default database backend.
|
||||
* [MuLiFS](https://github.com/dankomiocevic/mulifs) - Music Library Filesystem creates a filesystem to organise your music files.
|
||||
* [GoShort](https://github.com/pankajkhairnar/goShort) - GoShort is a URL shortener written in Golang and BoltDB for persistent key/value storage and for routing it's using high performent HTTPRouter.
|
||||
|
||||
If you are using Bolt in a project please send a pull request to add it to the list.
|
||||
|
18
cmd/vendor/github.com/boltdb/bolt/freelist.go
generated
vendored
18
cmd/vendor/github.com/boltdb/bolt/freelist.go
generated
vendored
@ -166,12 +166,16 @@ func (f *freelist) read(p *page) {
|
||||
}
|
||||
|
||||
// Copy the list of page ids from the freelist.
|
||||
ids := ((*[maxAllocSize]pgid)(unsafe.Pointer(&p.ptr)))[idx:count]
|
||||
f.ids = make([]pgid, len(ids))
|
||||
copy(f.ids, ids)
|
||||
if count == 0 {
|
||||
f.ids = nil
|
||||
} else {
|
||||
ids := ((*[maxAllocSize]pgid)(unsafe.Pointer(&p.ptr)))[idx:count]
|
||||
f.ids = make([]pgid, len(ids))
|
||||
copy(f.ids, ids)
|
||||
|
||||
// Make sure they're sorted.
|
||||
sort.Sort(pgids(f.ids))
|
||||
// Make sure they're sorted.
|
||||
sort.Sort(pgids(f.ids))
|
||||
}
|
||||
|
||||
// Rebuild the page cache.
|
||||
f.reindex()
|
||||
@ -189,7 +193,9 @@ func (f *freelist) write(p *page) error {
|
||||
|
||||
// The page.count can only hold up to 64k elements so if we overflow that
|
||||
// number then we handle it by putting the size in the first element.
|
||||
if len(ids) < 0xFFFF {
|
||||
if len(ids) == 0 {
|
||||
p.count = uint16(len(ids))
|
||||
} else if len(ids) < 0xFFFF {
|
||||
p.count = uint16(len(ids))
|
||||
copy(((*[maxAllocSize]pgid)(unsafe.Pointer(&p.ptr)))[:], ids)
|
||||
} else {
|
||||
|
5
cmd/vendor/github.com/boltdb/bolt/node.go
generated
vendored
5
cmd/vendor/github.com/boltdb/bolt/node.go
generated
vendored
@ -201,6 +201,11 @@ func (n *node) write(p *page) {
|
||||
}
|
||||
p.count = uint16(len(n.inodes))
|
||||
|
||||
// Stop here if there are no items to write.
|
||||
if p.count == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
// Loop over each item and write it to the page.
|
||||
b := (*[maxAllocSize]byte)(unsafe.Pointer(&p.ptr))[n.pageElementSize()*len(n.inodes):]
|
||||
for i, item := range n.inodes {
|
||||
|
6
cmd/vendor/github.com/boltdb/bolt/page.go
generated
vendored
6
cmd/vendor/github.com/boltdb/bolt/page.go
generated
vendored
@ -62,6 +62,9 @@ func (p *page) leafPageElement(index uint16) *leafPageElement {
|
||||
|
||||
// leafPageElements retrieves a list of leaf nodes.
|
||||
func (p *page) leafPageElements() []leafPageElement {
|
||||
if p.count == 0 {
|
||||
return nil
|
||||
}
|
||||
return ((*[0x7FFFFFF]leafPageElement)(unsafe.Pointer(&p.ptr)))[:]
|
||||
}
|
||||
|
||||
@ -72,6 +75,9 @@ func (p *page) branchPageElement(index uint16) *branchPageElement {
|
||||
|
||||
// branchPageElements retrieves a list of branch nodes.
|
||||
func (p *page) branchPageElements() []branchPageElement {
|
||||
if p.count == 0 {
|
||||
return nil
|
||||
}
|
||||
return ((*[0x7FFFFFF]branchPageElement)(unsafe.Pointer(&p.ptr)))[:]
|
||||
}
|
||||
|
||||
|
16
cmd/vendor/google.golang.org/grpc/.travis.yml
generated
vendored
16
cmd/vendor/google.golang.org/grpc/.travis.yml
generated
vendored
@ -1,17 +1,21 @@
|
||||
language: go
|
||||
|
||||
go:
|
||||
- 1.5.3
|
||||
- 1.6
|
||||
- 1.5.4
|
||||
- 1.6.3
|
||||
|
||||
go_import_path: google.golang.org/grpc
|
||||
|
||||
before_install:
|
||||
- go get golang.org/x/tools/cmd/goimports
|
||||
- go get github.com/golang/lint/golint
|
||||
- go get github.com/axw/gocov/gocov
|
||||
- go get github.com/mattn/goveralls
|
||||
- go get golang.org/x/tools/cmd/cover
|
||||
|
||||
install:
|
||||
- mkdir -p "$GOPATH/src/google.golang.org"
|
||||
- mv "$TRAVIS_BUILD_DIR" "$GOPATH/src/google.golang.org/grpc"
|
||||
|
||||
script:
|
||||
- '! gofmt -s -d -l . 2>&1 | read'
|
||||
- '! goimports -l . | read'
|
||||
- '! golint ./... | grep -vE "(_string|\.pb)\.go:"'
|
||||
- '! go tool vet -all . 2>&1 | grep -vE "constant [0-9]+ not a string in call to Errorf"'
|
||||
- make test testrace
|
||||
|
29
cmd/vendor/google.golang.org/grpc/call.go
generated
vendored
29
cmd/vendor/google.golang.org/grpc/call.go
generated
vendored
@ -36,6 +36,7 @@ package grpc
|
||||
import (
|
||||
"bytes"
|
||||
"io"
|
||||
"math"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
@ -51,13 +52,20 @@ import (
|
||||
func recvResponse(dopts dialOptions, t transport.ClientTransport, c *callInfo, stream *transport.Stream, reply interface{}) error {
|
||||
// Try to acquire header metadata from the server if there is any.
|
||||
var err error
|
||||
defer func() {
|
||||
if err != nil {
|
||||
if _, ok := err.(transport.ConnectionError); !ok {
|
||||
t.CloseStream(stream, err)
|
||||
}
|
||||
}
|
||||
}()
|
||||
c.headerMD, err = stream.Header()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
p := &parser{r: stream}
|
||||
for {
|
||||
if err = recv(p, dopts.codec, stream, dopts.dc, reply); err != nil {
|
||||
if err = recv(p, dopts.codec, stream, dopts.dc, reply, math.MaxInt32); err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
@ -76,6 +84,7 @@ func sendRequest(ctx context.Context, codec Codec, compressor Compressor, callHd
|
||||
}
|
||||
defer func() {
|
||||
if err != nil {
|
||||
// If err is connection error, t will be closed, no need to close stream here.
|
||||
if _, ok := err.(transport.ConnectionError); !ok {
|
||||
t.CloseStream(stream, err)
|
||||
}
|
||||
@ -90,7 +99,10 @@ func sendRequest(ctx context.Context, codec Codec, compressor Compressor, callHd
|
||||
return nil, transport.StreamErrorf(codes.Internal, "grpc: %v", err)
|
||||
}
|
||||
err = t.Write(stream, outBuf, opts)
|
||||
if err != nil {
|
||||
// t.NewStream(...) could lead to an early rejection of the RPC (e.g., the service/method
|
||||
// does not exist.) so that t.Write could get io.EOF from wait(...). Leave the following
|
||||
// recvResponse to get the final status.
|
||||
if err != nil && err != io.EOF {
|
||||
return nil, err
|
||||
}
|
||||
// Sent successfully.
|
||||
@ -158,9 +170,9 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
if _, ok := err.(*rpcError); ok {
|
||||
return err
|
||||
}
|
||||
if err == errConnClosing {
|
||||
if err == errConnClosing || err == errConnUnavailable {
|
||||
if c.failFast {
|
||||
return Errorf(codes.Unavailable, "%v", errConnClosing)
|
||||
return Errorf(codes.Unavailable, "%v", err)
|
||||
}
|
||||
continue
|
||||
}
|
||||
@ -176,7 +188,10 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
put()
|
||||
put = nil
|
||||
}
|
||||
if _, ok := err.(transport.ConnectionError); ok {
|
||||
// Retry a non-failfast RPC when
|
||||
// i) there is a connection error; or
|
||||
// ii) the server started to drain before this RPC was initiated.
|
||||
if _, ok := err.(transport.ConnectionError); ok || err == transport.ErrStreamDrain {
|
||||
if c.failFast {
|
||||
return toRPCErr(err)
|
||||
}
|
||||
@ -184,20 +199,18 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
}
|
||||
return toRPCErr(err)
|
||||
}
|
||||
// Receive the response
|
||||
err = recvResponse(cc.dopts, t, &c, stream, reply)
|
||||
if err != nil {
|
||||
if put != nil {
|
||||
put()
|
||||
put = nil
|
||||
}
|
||||
if _, ok := err.(transport.ConnectionError); ok {
|
||||
if _, ok := err.(transport.ConnectionError); ok || err == transport.ErrStreamDrain {
|
||||
if c.failFast {
|
||||
return toRPCErr(err)
|
||||
}
|
||||
continue
|
||||
}
|
||||
t.CloseStream(stream, err)
|
||||
return toRPCErr(err)
|
||||
}
|
||||
if c.traceInfo.tr != nil {
|
||||
|
343
cmd/vendor/google.golang.org/grpc/clientconn.go
generated
vendored
343
cmd/vendor/google.golang.org/grpc/clientconn.go
generated
vendored
@ -43,7 +43,6 @@ import (
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"golang.org/x/net/trace"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/credentials"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/transport"
|
||||
@ -68,13 +67,15 @@ var (
|
||||
// errCredentialsConflict indicates that grpc.WithTransportCredentials()
|
||||
// and grpc.WithInsecure() are both called for a connection.
|
||||
errCredentialsConflict = errors.New("grpc: transport credentials are set for an insecure connection (grpc.WithTransportCredentials() and grpc.WithInsecure() are both called)")
|
||||
// errNetworkIP indicates that the connection is down due to some network I/O error.
|
||||
// errNetworkIO indicates that the connection is down due to some network I/O error.
|
||||
errNetworkIO = errors.New("grpc: failed with network I/O error")
|
||||
// errConnDrain indicates that the connection starts to be drained and does not accept any new RPCs.
|
||||
errConnDrain = errors.New("grpc: the connection is drained")
|
||||
// errConnClosing indicates that the connection is closing.
|
||||
errConnClosing = errors.New("grpc: the connection is closing")
|
||||
errNoAddr = errors.New("grpc: there is no address available to dial")
|
||||
// errConnUnavailable indicates that the connection is unavailable.
|
||||
errConnUnavailable = errors.New("grpc: the connection is unavailable")
|
||||
errNoAddr = errors.New("grpc: there is no address available to dial")
|
||||
// minimum time to give a connection to complete
|
||||
minConnectTimeout = 20 * time.Second
|
||||
)
|
||||
@ -196,9 +197,14 @@ func WithTimeout(d time.Duration) DialOption {
|
||||
}
|
||||
|
||||
// WithDialer returns a DialOption that specifies a function to use for dialing network addresses.
|
||||
func WithDialer(f func(addr string, timeout time.Duration) (net.Conn, error)) DialOption {
|
||||
func WithDialer(f func(string, time.Duration) (net.Conn, error)) DialOption {
|
||||
return func(o *dialOptions) {
|
||||
o.copts.Dialer = f
|
||||
o.copts.Dialer = func(ctx context.Context, addr string) (net.Conn, error) {
|
||||
if deadline, ok := ctx.Deadline(); ok {
|
||||
return f(addr, deadline.Sub(time.Now()))
|
||||
}
|
||||
return f(addr, 0)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -209,12 +215,19 @@ func WithUserAgent(s string) DialOption {
|
||||
}
|
||||
}
|
||||
|
||||
// Dial creates a client connection the given target.
|
||||
// Dial creates a client connection to the given target.
|
||||
func Dial(target string, opts ...DialOption) (*ClientConn, error) {
|
||||
return DialContext(context.Background(), target, opts...)
|
||||
}
|
||||
|
||||
// DialContext creates a client connection to the given target
|
||||
// using the supplied context.
|
||||
func DialContext(ctx context.Context, target string, opts ...DialOption) (*ClientConn, error) {
|
||||
cc := &ClientConn{
|
||||
target: target,
|
||||
conns: make(map[Address]*addrConn),
|
||||
}
|
||||
cc.ctx, cc.cancel = context.WithCancel(ctx)
|
||||
for _, opt := range opts {
|
||||
opt(&cc.dopts)
|
||||
}
|
||||
@ -226,31 +239,33 @@ func Dial(target string, opts ...DialOption) (*ClientConn, error) {
|
||||
if cc.dopts.bs == nil {
|
||||
cc.dopts.bs = DefaultBackoffConfig
|
||||
}
|
||||
if cc.dopts.balancer == nil {
|
||||
cc.dopts.balancer = RoundRobin(nil)
|
||||
}
|
||||
|
||||
if err := cc.dopts.balancer.Start(target); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var (
|
||||
ok bool
|
||||
addrs []Address
|
||||
)
|
||||
ch := cc.dopts.balancer.Notify()
|
||||
if ch == nil {
|
||||
// There is no name resolver installed.
|
||||
if cc.dopts.balancer == nil {
|
||||
// Connect to target directly if balancer is nil.
|
||||
addrs = append(addrs, Address{Addr: target})
|
||||
} else {
|
||||
addrs, ok = <-ch
|
||||
if !ok || len(addrs) == 0 {
|
||||
return nil, errNoAddr
|
||||
if err := cc.dopts.balancer.Start(target); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
ch := cc.dopts.balancer.Notify()
|
||||
if ch == nil {
|
||||
// There is no name resolver installed.
|
||||
addrs = append(addrs, Address{Addr: target})
|
||||
} else {
|
||||
addrs, ok = <-ch
|
||||
if !ok || len(addrs) == 0 {
|
||||
return nil, errNoAddr
|
||||
}
|
||||
}
|
||||
}
|
||||
waitC := make(chan error, 1)
|
||||
go func() {
|
||||
for _, a := range addrs {
|
||||
if err := cc.newAddrConn(a, false); err != nil {
|
||||
if err := cc.resetAddrConn(a, false, nil); err != nil {
|
||||
waitC <- err
|
||||
return
|
||||
}
|
||||
@ -267,10 +282,15 @@ func Dial(target string, opts ...DialOption) (*ClientConn, error) {
|
||||
cc.Close()
|
||||
return nil, err
|
||||
}
|
||||
case <-cc.ctx.Done():
|
||||
cc.Close()
|
||||
return nil, cc.ctx.Err()
|
||||
case <-timeoutCh:
|
||||
cc.Close()
|
||||
return nil, ErrClientConnTimeout
|
||||
}
|
||||
// If balancer is nil or balancer.Notify() is nil, ok will be false here.
|
||||
// The lbWatcher goroutine will not be created.
|
||||
if ok {
|
||||
go cc.lbWatcher()
|
||||
}
|
||||
@ -317,6 +337,9 @@ func (s ConnectivityState) String() string {
|
||||
|
||||
// ClientConn represents a client connection to an RPC server.
|
||||
type ClientConn struct {
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
|
||||
target string
|
||||
authority string
|
||||
dopts dialOptions
|
||||
@ -347,11 +370,12 @@ func (cc *ClientConn) lbWatcher() {
|
||||
}
|
||||
if !keep {
|
||||
del = append(del, c)
|
||||
delete(cc.conns, c.addr)
|
||||
}
|
||||
}
|
||||
cc.mu.Unlock()
|
||||
for _, a := range add {
|
||||
cc.newAddrConn(a, true)
|
||||
cc.resetAddrConn(a, true, nil)
|
||||
}
|
||||
for _, c := range del {
|
||||
c.tearDown(errConnDrain)
|
||||
@ -359,13 +383,17 @@ func (cc *ClientConn) lbWatcher() {
|
||||
}
|
||||
}
|
||||
|
||||
func (cc *ClientConn) newAddrConn(addr Address, skipWait bool) error {
|
||||
// resetAddrConn creates an addrConn for addr and adds it to cc.conns.
|
||||
// If there is an old addrConn for addr, it will be torn down, using tearDownErr as the reason.
|
||||
// If tearDownErr is nil, errConnDrain will be used instead.
|
||||
func (cc *ClientConn) resetAddrConn(addr Address, skipWait bool, tearDownErr error) error {
|
||||
ac := &addrConn{
|
||||
cc: cc,
|
||||
addr: addr,
|
||||
dopts: cc.dopts,
|
||||
shutdownChan: make(chan struct{}),
|
||||
cc: cc,
|
||||
addr: addr,
|
||||
dopts: cc.dopts,
|
||||
}
|
||||
ac.ctx, ac.cancel = context.WithCancel(cc.ctx)
|
||||
ac.stateCV = sync.NewCond(&ac.mu)
|
||||
if EnableTracing {
|
||||
ac.events = trace.NewEventLog("grpc.ClientConn", ac.addr.Addr)
|
||||
}
|
||||
@ -383,26 +411,44 @@ func (cc *ClientConn) newAddrConn(addr Address, skipWait bool) error {
|
||||
}
|
||||
}
|
||||
}
|
||||
// Insert ac into ac.cc.conns. This needs to be done before any getTransport(...) is called.
|
||||
ac.cc.mu.Lock()
|
||||
if ac.cc.conns == nil {
|
||||
ac.cc.mu.Unlock()
|
||||
// Track ac in cc. This needs to be done before any getTransport(...) is called.
|
||||
cc.mu.Lock()
|
||||
if cc.conns == nil {
|
||||
cc.mu.Unlock()
|
||||
return ErrClientConnClosing
|
||||
}
|
||||
stale := ac.cc.conns[ac.addr]
|
||||
ac.cc.conns[ac.addr] = ac
|
||||
ac.cc.mu.Unlock()
|
||||
stale := cc.conns[ac.addr]
|
||||
cc.conns[ac.addr] = ac
|
||||
cc.mu.Unlock()
|
||||
if stale != nil {
|
||||
// There is an addrConn alive on ac.addr already. This could be due to
|
||||
// i) stale's Close is undergoing;
|
||||
// ii) a buggy Balancer notifies duplicated Addresses.
|
||||
stale.tearDown(errConnDrain)
|
||||
// 1) a buggy Balancer notifies duplicated Addresses;
|
||||
// 2) goaway was received, a new ac will replace the old ac.
|
||||
// The old ac should be deleted from cc.conns, but the
|
||||
// underlying transport should drain rather than close.
|
||||
if tearDownErr == nil {
|
||||
// tearDownErr is nil if resetAddrConn is called by
|
||||
// 1) Dial
|
||||
// 2) lbWatcher
|
||||
// In both cases, the stale ac should drain, not close.
|
||||
stale.tearDown(errConnDrain)
|
||||
} else {
|
||||
stale.tearDown(tearDownErr)
|
||||
}
|
||||
}
|
||||
ac.stateCV = sync.NewCond(&ac.mu)
|
||||
// skipWait may overwrite the decision in ac.dopts.block.
|
||||
if ac.dopts.block && !skipWait {
|
||||
if err := ac.resetTransport(false); err != nil {
|
||||
ac.tearDown(err)
|
||||
if err != errConnClosing {
|
||||
// Tear down ac and delete it from cc.conns.
|
||||
cc.mu.Lock()
|
||||
delete(cc.conns, ac.addr)
|
||||
cc.mu.Unlock()
|
||||
ac.tearDown(err)
|
||||
}
|
||||
if e, ok := err.(transport.ConnectionError); ok && !e.Temporary() {
|
||||
return e.Origin()
|
||||
}
|
||||
return err
|
||||
}
|
||||
// Start to monitor the error status of transport.
|
||||
@ -412,7 +458,10 @@ func (cc *ClientConn) newAddrConn(addr Address, skipWait bool) error {
|
||||
go func() {
|
||||
if err := ac.resetTransport(false); err != nil {
|
||||
grpclog.Printf("Failed to dial %s: %v; please retry.", ac.addr.Addr, err)
|
||||
ac.tearDown(err)
|
||||
if err != errConnClosing {
|
||||
// Keep this ac in cc.conns, to get the reason it's torn down.
|
||||
ac.tearDown(err)
|
||||
}
|
||||
return
|
||||
}
|
||||
ac.transportMonitor()
|
||||
@ -422,24 +471,48 @@ func (cc *ClientConn) newAddrConn(addr Address, skipWait bool) error {
|
||||
}
|
||||
|
||||
func (cc *ClientConn) getTransport(ctx context.Context, opts BalancerGetOptions) (transport.ClientTransport, func(), error) {
|
||||
addr, put, err := cc.dopts.balancer.Get(ctx, opts)
|
||||
if err != nil {
|
||||
return nil, nil, toRPCErr(err)
|
||||
}
|
||||
cc.mu.RLock()
|
||||
if cc.conns == nil {
|
||||
var (
|
||||
ac *addrConn
|
||||
ok bool
|
||||
put func()
|
||||
)
|
||||
if cc.dopts.balancer == nil {
|
||||
// If balancer is nil, there should be only one addrConn available.
|
||||
cc.mu.RLock()
|
||||
if cc.conns == nil {
|
||||
cc.mu.RUnlock()
|
||||
return nil, nil, toRPCErr(ErrClientConnClosing)
|
||||
}
|
||||
for _, ac = range cc.conns {
|
||||
// Break after the first iteration to get the first addrConn.
|
||||
ok = true
|
||||
break
|
||||
}
|
||||
cc.mu.RUnlock()
|
||||
} else {
|
||||
var (
|
||||
addr Address
|
||||
err error
|
||||
)
|
||||
addr, put, err = cc.dopts.balancer.Get(ctx, opts)
|
||||
if err != nil {
|
||||
return nil, nil, toRPCErr(err)
|
||||
}
|
||||
cc.mu.RLock()
|
||||
if cc.conns == nil {
|
||||
cc.mu.RUnlock()
|
||||
return nil, nil, toRPCErr(ErrClientConnClosing)
|
||||
}
|
||||
ac, ok = cc.conns[addr]
|
||||
cc.mu.RUnlock()
|
||||
return nil, nil, toRPCErr(ErrClientConnClosing)
|
||||
}
|
||||
ac, ok := cc.conns[addr]
|
||||
cc.mu.RUnlock()
|
||||
if !ok {
|
||||
if put != nil {
|
||||
put()
|
||||
}
|
||||
return nil, nil, Errorf(codes.Internal, "grpc: failed to find the transport to send the rpc")
|
||||
return nil, nil, errConnClosing
|
||||
}
|
||||
t, err := ac.wait(ctx, !opts.BlockingWait)
|
||||
t, err := ac.wait(ctx, cc.dopts.balancer != nil, !opts.BlockingWait)
|
||||
if err != nil {
|
||||
if put != nil {
|
||||
put()
|
||||
@ -451,6 +524,8 @@ func (cc *ClientConn) getTransport(ctx context.Context, opts BalancerGetOptions)
|
||||
|
||||
// Close tears down the ClientConn and all underlying connections.
|
||||
func (cc *ClientConn) Close() error {
|
||||
cc.cancel()
|
||||
|
||||
cc.mu.Lock()
|
||||
if cc.conns == nil {
|
||||
cc.mu.Unlock()
|
||||
@ -459,7 +534,9 @@ func (cc *ClientConn) Close() error {
|
||||
conns := cc.conns
|
||||
cc.conns = nil
|
||||
cc.mu.Unlock()
|
||||
cc.dopts.balancer.Close()
|
||||
if cc.dopts.balancer != nil {
|
||||
cc.dopts.balancer.Close()
|
||||
}
|
||||
for _, ac := range conns {
|
||||
ac.tearDown(ErrClientConnClosing)
|
||||
}
|
||||
@ -468,11 +545,13 @@ func (cc *ClientConn) Close() error {
|
||||
|
||||
// addrConn is a network connection to a given address.
|
||||
type addrConn struct {
|
||||
cc *ClientConn
|
||||
addr Address
|
||||
dopts dialOptions
|
||||
shutdownChan chan struct{}
|
||||
events trace.EventLog
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
|
||||
cc *ClientConn
|
||||
addr Address
|
||||
dopts dialOptions
|
||||
events trace.EventLog
|
||||
|
||||
mu sync.Mutex
|
||||
state ConnectivityState
|
||||
@ -482,6 +561,9 @@ type addrConn struct {
|
||||
// due to timeout.
|
||||
ready chan struct{}
|
||||
transport transport.ClientTransport
|
||||
|
||||
// The reason this addrConn is torn down.
|
||||
tearDownErr error
|
||||
}
|
||||
|
||||
// printf records an event in ac's event log, unless ac has been closed.
|
||||
@ -537,8 +619,7 @@ func (ac *addrConn) waitForStateChange(ctx context.Context, sourceState Connecti
|
||||
}
|
||||
|
||||
func (ac *addrConn) resetTransport(closeTransport bool) error {
|
||||
var retries int
|
||||
for {
|
||||
for retries := 0; ; retries++ {
|
||||
ac.mu.Lock()
|
||||
ac.printf("connecting")
|
||||
if ac.state == Shutdown {
|
||||
@ -558,13 +639,20 @@ func (ac *addrConn) resetTransport(closeTransport bool) error {
|
||||
t.Close()
|
||||
}
|
||||
sleepTime := ac.dopts.bs.backoff(retries)
|
||||
ac.dopts.copts.Timeout = sleepTime
|
||||
if sleepTime < minConnectTimeout {
|
||||
ac.dopts.copts.Timeout = minConnectTimeout
|
||||
timeout := minConnectTimeout
|
||||
if timeout < sleepTime {
|
||||
timeout = sleepTime
|
||||
}
|
||||
ctx, cancel := context.WithTimeout(ac.ctx, timeout)
|
||||
connectTime := time.Now()
|
||||
newTransport, err := transport.NewClientTransport(ac.addr.Addr, &ac.dopts.copts)
|
||||
newTransport, err := transport.NewClientTransport(ctx, ac.addr.Addr, ac.dopts.copts)
|
||||
if err != nil {
|
||||
cancel()
|
||||
|
||||
if e, ok := err.(transport.ConnectionError); ok && !e.Temporary() {
|
||||
return err
|
||||
}
|
||||
grpclog.Printf("grpc: addrConn.resetTransport failed to create client transport: %v; Reconnecting to %q", err, ac.addr)
|
||||
ac.mu.Lock()
|
||||
if ac.state == Shutdown {
|
||||
// ac.tearDown(...) has been invoked.
|
||||
@ -579,17 +667,12 @@ func (ac *addrConn) resetTransport(closeTransport bool) error {
|
||||
ac.ready = nil
|
||||
}
|
||||
ac.mu.Unlock()
|
||||
sleepTime -= time.Since(connectTime)
|
||||
if sleepTime < 0 {
|
||||
sleepTime = 0
|
||||
}
|
||||
closeTransport = false
|
||||
select {
|
||||
case <-time.After(sleepTime):
|
||||
case <-ac.shutdownChan:
|
||||
case <-time.After(sleepTime - time.Since(connectTime)):
|
||||
case <-ac.ctx.Done():
|
||||
return ac.ctx.Err()
|
||||
}
|
||||
retries++
|
||||
grpclog.Printf("grpc: addrConn.resetTransport failed to create client transport: %v; Reconnecting to %q", err, ac.addr)
|
||||
continue
|
||||
}
|
||||
ac.mu.Lock()
|
||||
@ -607,7 +690,9 @@ func (ac *addrConn) resetTransport(closeTransport bool) error {
|
||||
close(ac.ready)
|
||||
ac.ready = nil
|
||||
}
|
||||
ac.down = ac.cc.dopts.balancer.Up(ac.addr)
|
||||
if ac.cc.dopts.balancer != nil {
|
||||
ac.down = ac.cc.dopts.balancer.Up(ac.addr)
|
||||
}
|
||||
ac.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
@ -621,14 +706,42 @@ func (ac *addrConn) transportMonitor() {
|
||||
t := ac.transport
|
||||
ac.mu.Unlock()
|
||||
select {
|
||||
// shutdownChan is needed to detect the teardown when
|
||||
// This is needed to detect the teardown when
|
||||
// the addrConn is idle (i.e., no RPC in flight).
|
||||
case <-ac.shutdownChan:
|
||||
case <-ac.ctx.Done():
|
||||
select {
|
||||
case <-t.Error():
|
||||
t.Close()
|
||||
default:
|
||||
}
|
||||
return
|
||||
case <-t.GoAway():
|
||||
// If GoAway happens without any network I/O error, ac is closed without shutting down the
|
||||
// underlying transport (the transport will be closed when all the pending RPCs finished or
|
||||
// failed.).
|
||||
// If GoAway and some network I/O error happen concurrently, ac and its underlying transport
|
||||
// are closed.
|
||||
// In both cases, a new ac is created.
|
||||
select {
|
||||
case <-t.Error():
|
||||
ac.cc.resetAddrConn(ac.addr, true, errNetworkIO)
|
||||
default:
|
||||
ac.cc.resetAddrConn(ac.addr, true, errConnDrain)
|
||||
}
|
||||
return
|
||||
case <-t.Error():
|
||||
select {
|
||||
case <-ac.ctx.Done():
|
||||
t.Close()
|
||||
return
|
||||
case <-t.GoAway():
|
||||
ac.cc.resetAddrConn(ac.addr, true, errNetworkIO)
|
||||
return
|
||||
default:
|
||||
}
|
||||
ac.mu.Lock()
|
||||
if ac.state == Shutdown {
|
||||
// ac.tearDown(...) has been invoked.
|
||||
// ac has been shutdown.
|
||||
ac.mu.Unlock()
|
||||
return
|
||||
}
|
||||
@ -640,6 +753,10 @@ func (ac *addrConn) transportMonitor() {
|
||||
ac.printf("transport exiting: %v", err)
|
||||
ac.mu.Unlock()
|
||||
grpclog.Printf("grpc: addrConn.transportMonitor exits due to: %v", err)
|
||||
if err != errConnClosing {
|
||||
// Keep this ac in cc.conns, to get the reason it's torn down.
|
||||
ac.tearDown(err)
|
||||
}
|
||||
return
|
||||
}
|
||||
}
|
||||
@ -647,35 +764,42 @@ func (ac *addrConn) transportMonitor() {
|
||||
}
|
||||
|
||||
// wait blocks until i) the new transport is up or ii) ctx is done or iii) ac is closed or
|
||||
// iv) transport is in TransientFailure and the RPC is fail-fast.
|
||||
func (ac *addrConn) wait(ctx context.Context, failFast bool) (transport.ClientTransport, error) {
|
||||
// iv) transport is in TransientFailure and there's no balancer/failfast is true.
|
||||
func (ac *addrConn) wait(ctx context.Context, hasBalancer, failfast bool) (transport.ClientTransport, error) {
|
||||
for {
|
||||
ac.mu.Lock()
|
||||
switch {
|
||||
case ac.state == Shutdown:
|
||||
if failfast || !hasBalancer {
|
||||
// RPC is failfast or balancer is nil. This RPC should fail with ac.tearDownErr.
|
||||
err := ac.tearDownErr
|
||||
ac.mu.Unlock()
|
||||
return nil, err
|
||||
}
|
||||
ac.mu.Unlock()
|
||||
return nil, errConnClosing
|
||||
case ac.state == Ready:
|
||||
ct := ac.transport
|
||||
ac.mu.Unlock()
|
||||
return ct, nil
|
||||
case ac.state == TransientFailure && failFast:
|
||||
ac.mu.Unlock()
|
||||
return nil, Errorf(codes.Unavailable, "grpc: RPC failed fast due to transport failure")
|
||||
default:
|
||||
ready := ac.ready
|
||||
if ready == nil {
|
||||
ready = make(chan struct{})
|
||||
ac.ready = ready
|
||||
}
|
||||
ac.mu.Unlock()
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, toRPCErr(ctx.Err())
|
||||
// Wait until the new transport is ready or failed.
|
||||
case <-ready:
|
||||
case ac.state == TransientFailure:
|
||||
if failfast || hasBalancer {
|
||||
ac.mu.Unlock()
|
||||
return nil, errConnUnavailable
|
||||
}
|
||||
}
|
||||
ready := ac.ready
|
||||
if ready == nil {
|
||||
ready = make(chan struct{})
|
||||
ac.ready = ready
|
||||
}
|
||||
ac.mu.Unlock()
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, toRPCErr(ctx.Err())
|
||||
// Wait until the new transport is ready or failed.
|
||||
case <-ready:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -683,24 +807,28 @@ func (ac *addrConn) wait(ctx context.Context, failFast bool) (transport.ClientTr
|
||||
// TODO(zhaoq): Make this synchronous to avoid unbounded memory consumption in
|
||||
// some edge cases (e.g., the caller opens and closes many addrConn's in a
|
||||
// tight loop.
|
||||
// tearDown doesn't remove ac from ac.cc.conns.
|
||||
func (ac *addrConn) tearDown(err error) {
|
||||
ac.cancel()
|
||||
|
||||
ac.mu.Lock()
|
||||
defer func() {
|
||||
ac.mu.Unlock()
|
||||
ac.cc.mu.Lock()
|
||||
if ac.cc.conns != nil {
|
||||
delete(ac.cc.conns, ac.addr)
|
||||
}
|
||||
ac.cc.mu.Unlock()
|
||||
}()
|
||||
if ac.state == Shutdown {
|
||||
return
|
||||
}
|
||||
ac.state = Shutdown
|
||||
defer ac.mu.Unlock()
|
||||
if ac.down != nil {
|
||||
ac.down(downErrorf(false, false, "%v", err))
|
||||
ac.down = nil
|
||||
}
|
||||
if err == errConnDrain && ac.transport != nil {
|
||||
// GracefulClose(...) may be executed multiple times when
|
||||
// i) receiving multiple GoAway frames from the server; or
|
||||
// ii) there are concurrent name resolver/Balancer triggered
|
||||
// address removal and GoAway.
|
||||
ac.transport.GracefulClose()
|
||||
}
|
||||
if ac.state == Shutdown {
|
||||
return
|
||||
}
|
||||
ac.state = Shutdown
|
||||
ac.tearDownErr = err
|
||||
ac.stateCV.Broadcast()
|
||||
if ac.events != nil {
|
||||
ac.events.Finish()
|
||||
@ -710,15 +838,8 @@ func (ac *addrConn) tearDown(err error) {
|
||||
close(ac.ready)
|
||||
ac.ready = nil
|
||||
}
|
||||
if ac.transport != nil {
|
||||
if err == errConnDrain {
|
||||
ac.transport.GracefulClose()
|
||||
} else {
|
||||
ac.transport.Close()
|
||||
}
|
||||
}
|
||||
if ac.shutdownChan != nil {
|
||||
close(ac.shutdownChan)
|
||||
if ac.transport != nil && err != errConnDrain {
|
||||
ac.transport.Close()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
53
cmd/vendor/google.golang.org/grpc/credentials/credentials.go
generated
vendored
53
cmd/vendor/google.golang.org/grpc/credentials/credentials.go
generated
vendored
@ -44,7 +44,6 @@ import (
|
||||
"io/ioutil"
|
||||
"net"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
@ -93,11 +92,12 @@ type TransportCredentials interface {
|
||||
// ClientHandshake does the authentication handshake specified by the corresponding
|
||||
// authentication protocol on rawConn for clients. It returns the authenticated
|
||||
// connection and the corresponding auth information about the connection.
|
||||
ClientHandshake(addr string, rawConn net.Conn, timeout time.Duration) (net.Conn, AuthInfo, error)
|
||||
// Implementations must use the provided context to implement timely cancellation.
|
||||
ClientHandshake(context.Context, string, net.Conn) (net.Conn, AuthInfo, error)
|
||||
// ServerHandshake does the authentication handshake for servers. It returns
|
||||
// the authenticated connection and the corresponding auth information about
|
||||
// the connection.
|
||||
ServerHandshake(rawConn net.Conn) (net.Conn, AuthInfo, error)
|
||||
ServerHandshake(net.Conn) (net.Conn, AuthInfo, error)
|
||||
// Info provides the ProtocolInfo of this TransportCredentials.
|
||||
Info() ProtocolInfo
|
||||
}
|
||||
@ -136,42 +136,28 @@ func (c *tlsCreds) RequireTransportSecurity() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
type timeoutError struct{}
|
||||
|
||||
func (timeoutError) Error() string { return "credentials: Dial timed out" }
|
||||
func (timeoutError) Timeout() bool { return true }
|
||||
func (timeoutError) Temporary() bool { return true }
|
||||
|
||||
func (c *tlsCreds) ClientHandshake(addr string, rawConn net.Conn, timeout time.Duration) (_ net.Conn, _ AuthInfo, err error) {
|
||||
// borrow some code from tls.DialWithDialer
|
||||
var errChannel chan error
|
||||
if timeout != 0 {
|
||||
errChannel = make(chan error, 2)
|
||||
time.AfterFunc(timeout, func() {
|
||||
errChannel <- timeoutError{}
|
||||
})
|
||||
}
|
||||
func (c *tlsCreds) ClientHandshake(ctx context.Context, addr string, rawConn net.Conn) (_ net.Conn, _ AuthInfo, err error) {
|
||||
// use local cfg to avoid clobbering ServerName if using multiple endpoints
|
||||
cfg := *c.config
|
||||
if c.config.ServerName == "" {
|
||||
cfg := cloneTLSConfig(c.config)
|
||||
if cfg.ServerName == "" {
|
||||
colonPos := strings.LastIndex(addr, ":")
|
||||
if colonPos == -1 {
|
||||
colonPos = len(addr)
|
||||
}
|
||||
cfg.ServerName = addr[:colonPos]
|
||||
}
|
||||
conn := tls.Client(rawConn, &cfg)
|
||||
if timeout == 0 {
|
||||
err = conn.Handshake()
|
||||
} else {
|
||||
go func() {
|
||||
errChannel <- conn.Handshake()
|
||||
}()
|
||||
err = <-errChannel
|
||||
}
|
||||
if err != nil {
|
||||
rawConn.Close()
|
||||
return nil, nil, err
|
||||
conn := tls.Client(rawConn, cfg)
|
||||
errChannel := make(chan error, 1)
|
||||
go func() {
|
||||
errChannel <- conn.Handshake()
|
||||
}()
|
||||
select {
|
||||
case err := <-errChannel:
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
case <-ctx.Done():
|
||||
return nil, nil, ctx.Err()
|
||||
}
|
||||
// TODO(zhaoq): Omit the auth info for client now. It is more for
|
||||
// information than anything else.
|
||||
@ -181,7 +167,6 @@ func (c *tlsCreds) ClientHandshake(addr string, rawConn net.Conn, timeout time.D
|
||||
func (c *tlsCreds) ServerHandshake(rawConn net.Conn) (net.Conn, AuthInfo, error) {
|
||||
conn := tls.Server(rawConn, c.config)
|
||||
if err := conn.Handshake(); err != nil {
|
||||
rawConn.Close()
|
||||
return nil, nil, err
|
||||
}
|
||||
return conn, TLSInfo{conn.ConnectionState()}, nil
|
||||
@ -189,7 +174,7 @@ func (c *tlsCreds) ServerHandshake(rawConn net.Conn) (net.Conn, AuthInfo, error)
|
||||
|
||||
// NewTLS uses c to construct a TransportCredentials based on TLS.
|
||||
func NewTLS(c *tls.Config) TransportCredentials {
|
||||
tc := &tlsCreds{c}
|
||||
tc := &tlsCreds{cloneTLSConfig(c)}
|
||||
tc.config.NextProtos = alpnProtoStr
|
||||
return tc
|
||||
}
|
||||
|
76
cmd/vendor/google.golang.org/grpc/credentials/credentials_util_go17.go
generated
vendored
Normal file
76
cmd/vendor/google.golang.org/grpc/credentials/credentials_util_go17.go
generated
vendored
Normal file
@ -0,0 +1,76 @@
|
||||
// +build go1.7
|
||||
|
||||
/*
|
||||
*
|
||||
* Copyright 2016, Google Inc.
|
||||
* All rights reserved.
|
||||
*
|
||||
* Redistribution and use in source and binary forms, with or without
|
||||
* modification, are permitted provided that the following conditions are
|
||||
* met:
|
||||
*
|
||||
* * Redistributions of source code must retain the above copyright
|
||||
* notice, this list of conditions and the following disclaimer.
|
||||
* * Redistributions in binary form must reproduce the above
|
||||
* copyright notice, this list of conditions and the following disclaimer
|
||||
* in the documentation and/or other materials provided with the
|
||||
* distribution.
|
||||
* * Neither the name of Google Inc. nor the names of its
|
||||
* contributors may be used to endorse or promote products derived from
|
||||
* this software without specific prior written permission.
|
||||
*
|
||||
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
*
|
||||
*/
|
||||
|
||||
package credentials
|
||||
|
||||
import (
|
||||
"crypto/tls"
|
||||
)
|
||||
|
||||
// cloneTLSConfig returns a shallow clone of the exported
|
||||
// fields of cfg, ignoring the unexported sync.Once, which
|
||||
// contains a mutex and must not be copied.
|
||||
//
|
||||
// If cfg is nil, a new zero tls.Config is returned.
|
||||
//
|
||||
// TODO replace this function with official clone function.
|
||||
func cloneTLSConfig(cfg *tls.Config) *tls.Config {
|
||||
if cfg == nil {
|
||||
return &tls.Config{}
|
||||
}
|
||||
return &tls.Config{
|
||||
Rand: cfg.Rand,
|
||||
Time: cfg.Time,
|
||||
Certificates: cfg.Certificates,
|
||||
NameToCertificate: cfg.NameToCertificate,
|
||||
GetCertificate: cfg.GetCertificate,
|
||||
RootCAs: cfg.RootCAs,
|
||||
NextProtos: cfg.NextProtos,
|
||||
ServerName: cfg.ServerName,
|
||||
ClientAuth: cfg.ClientAuth,
|
||||
ClientCAs: cfg.ClientCAs,
|
||||
InsecureSkipVerify: cfg.InsecureSkipVerify,
|
||||
CipherSuites: cfg.CipherSuites,
|
||||
PreferServerCipherSuites: cfg.PreferServerCipherSuites,
|
||||
SessionTicketsDisabled: cfg.SessionTicketsDisabled,
|
||||
SessionTicketKey: cfg.SessionTicketKey,
|
||||
ClientSessionCache: cfg.ClientSessionCache,
|
||||
MinVersion: cfg.MinVersion,
|
||||
MaxVersion: cfg.MaxVersion,
|
||||
CurvePreferences: cfg.CurvePreferences,
|
||||
DynamicRecordSizingDisabled: cfg.DynamicRecordSizingDisabled,
|
||||
Renegotiation: cfg.Renegotiation,
|
||||
}
|
||||
}
|
74
cmd/vendor/google.golang.org/grpc/credentials/credentials_util_pre_go17.go
generated
vendored
Normal file
74
cmd/vendor/google.golang.org/grpc/credentials/credentials_util_pre_go17.go
generated
vendored
Normal file
@ -0,0 +1,74 @@
|
||||
// +build !go1.7
|
||||
|
||||
/*
|
||||
*
|
||||
* Copyright 2016, Google Inc.
|
||||
* All rights reserved.
|
||||
*
|
||||
* Redistribution and use in source and binary forms, with or without
|
||||
* modification, are permitted provided that the following conditions are
|
||||
* met:
|
||||
*
|
||||
* * Redistributions of source code must retain the above copyright
|
||||
* notice, this list of conditions and the following disclaimer.
|
||||
* * Redistributions in binary form must reproduce the above
|
||||
* copyright notice, this list of conditions and the following disclaimer
|
||||
* in the documentation and/or other materials provided with the
|
||||
* distribution.
|
||||
* * Neither the name of Google Inc. nor the names of its
|
||||
* contributors may be used to endorse or promote products derived from
|
||||
* this software without specific prior written permission.
|
||||
*
|
||||
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
*
|
||||
*/
|
||||
|
||||
package credentials
|
||||
|
||||
import (
|
||||
"crypto/tls"
|
||||
)
|
||||
|
||||
// cloneTLSConfig returns a shallow clone of the exported
|
||||
// fields of cfg, ignoring the unexported sync.Once, which
|
||||
// contains a mutex and must not be copied.
|
||||
//
|
||||
// If cfg is nil, a new zero tls.Config is returned.
|
||||
//
|
||||
// TODO replace this function with official clone function.
|
||||
func cloneTLSConfig(cfg *tls.Config) *tls.Config {
|
||||
if cfg == nil {
|
||||
return &tls.Config{}
|
||||
}
|
||||
return &tls.Config{
|
||||
Rand: cfg.Rand,
|
||||
Time: cfg.Time,
|
||||
Certificates: cfg.Certificates,
|
||||
NameToCertificate: cfg.NameToCertificate,
|
||||
GetCertificate: cfg.GetCertificate,
|
||||
RootCAs: cfg.RootCAs,
|
||||
NextProtos: cfg.NextProtos,
|
||||
ServerName: cfg.ServerName,
|
||||
ClientAuth: cfg.ClientAuth,
|
||||
ClientCAs: cfg.ClientCAs,
|
||||
InsecureSkipVerify: cfg.InsecureSkipVerify,
|
||||
CipherSuites: cfg.CipherSuites,
|
||||
PreferServerCipherSuites: cfg.PreferServerCipherSuites,
|
||||
SessionTicketsDisabled: cfg.SessionTicketsDisabled,
|
||||
SessionTicketKey: cfg.SessionTicketKey,
|
||||
ClientSessionCache: cfg.ClientSessionCache,
|
||||
MinVersion: cfg.MinVersion,
|
||||
MaxVersion: cfg.MaxVersion,
|
||||
CurvePreferences: cfg.CurvePreferences,
|
||||
}
|
||||
}
|
14
cmd/vendor/google.golang.org/grpc/metadata/metadata.go
generated
vendored
14
cmd/vendor/google.golang.org/grpc/metadata/metadata.go
generated
vendored
@ -60,15 +60,21 @@ func encodeKeyValue(k, v string) (string, string) {
|
||||
|
||||
// DecodeKeyValue returns the original key and value corresponding to the
|
||||
// encoded data in k, v.
|
||||
// If k is a binary header and v contains comma, v is split on comma before decoded,
|
||||
// and the decoded v will be joined with comma before returned.
|
||||
func DecodeKeyValue(k, v string) (string, string, error) {
|
||||
if !strings.HasSuffix(k, binHdrSuffix) {
|
||||
return k, v, nil
|
||||
}
|
||||
val, err := base64.StdEncoding.DecodeString(v)
|
||||
if err != nil {
|
||||
return "", "", err
|
||||
vvs := strings.Split(v, ",")
|
||||
for i, vv := range vvs {
|
||||
val, err := base64.StdEncoding.DecodeString(vv)
|
||||
if err != nil {
|
||||
return "", "", err
|
||||
}
|
||||
vvs[i] = string(val)
|
||||
}
|
||||
return k, string(val), nil
|
||||
return k, strings.Join(vvs, ","), nil
|
||||
}
|
||||
|
||||
// MD is a mapping from metadata keys to values. Users should use the following
|
||||
|
18
cmd/vendor/google.golang.org/grpc/rpc_util.go
generated
vendored
18
cmd/vendor/google.golang.org/grpc/rpc_util.go
generated
vendored
@ -227,7 +227,7 @@ type parser struct {
|
||||
// No other error values or types must be returned, which also means
|
||||
// that the underlying io.Reader must not return an incompatible
|
||||
// error.
|
||||
func (p *parser) recvMsg() (pf payloadFormat, msg []byte, err error) {
|
||||
func (p *parser) recvMsg(maxMsgSize int) (pf payloadFormat, msg []byte, err error) {
|
||||
if _, err := io.ReadFull(p.r, p.header[:]); err != nil {
|
||||
return 0, nil, err
|
||||
}
|
||||
@ -238,6 +238,9 @@ func (p *parser) recvMsg() (pf payloadFormat, msg []byte, err error) {
|
||||
if length == 0 {
|
||||
return pf, nil, nil
|
||||
}
|
||||
if length > uint32(maxMsgSize) {
|
||||
return 0, nil, Errorf(codes.Internal, "grpc: received message length %d exceeding the max size %d", length, maxMsgSize)
|
||||
}
|
||||
// TODO(bradfitz,zhaoq): garbage. reuse buffer after proto decoding instead
|
||||
// of making it for each message:
|
||||
msg = make([]byte, int(length))
|
||||
@ -308,8 +311,8 @@ func checkRecvPayload(pf payloadFormat, recvCompress string, dc Decompressor) er
|
||||
return nil
|
||||
}
|
||||
|
||||
func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{}) error {
|
||||
pf, d, err := p.recvMsg()
|
||||
func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{}, maxMsgSize int) error {
|
||||
pf, d, err := p.recvMsg(maxMsgSize)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -319,11 +322,16 @@ func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{
|
||||
if pf == compressionMade {
|
||||
d, err = dc.Do(bytes.NewReader(d))
|
||||
if err != nil {
|
||||
return transport.StreamErrorf(codes.Internal, "grpc: failed to decompress the received message %v", err)
|
||||
return Errorf(codes.Internal, "grpc: failed to decompress the received message %v", err)
|
||||
}
|
||||
}
|
||||
if len(d) > maxMsgSize {
|
||||
// TODO: Revisit the error code. Currently keep it consistent with java
|
||||
// implementation.
|
||||
return Errorf(codes.Internal, "grpc: received a message of %d bytes exceeding %d limit", len(d), maxMsgSize)
|
||||
}
|
||||
if err := c.Unmarshal(d, m); err != nil {
|
||||
return transport.StreamErrorf(codes.Internal, "grpc: failed to unmarshal the received message %v", err)
|
||||
return Errorf(codes.Internal, "grpc: failed to unmarshal the received message %v", err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
95
cmd/vendor/google.golang.org/grpc/server.go
generated
vendored
95
cmd/vendor/google.golang.org/grpc/server.go
generated
vendored
@ -89,9 +89,13 @@ type service struct {
|
||||
type Server struct {
|
||||
opts options
|
||||
|
||||
mu sync.Mutex // guards following
|
||||
lis map[net.Listener]bool
|
||||
conns map[io.Closer]bool
|
||||
mu sync.Mutex // guards following
|
||||
lis map[net.Listener]bool
|
||||
conns map[io.Closer]bool
|
||||
drain bool
|
||||
// A CondVar to let GracefulStop() blocks until all the pending RPCs are finished
|
||||
// and all the transport goes away.
|
||||
cv *sync.Cond
|
||||
m map[string]*service // service name -> service info
|
||||
events trace.EventLog
|
||||
}
|
||||
@ -101,12 +105,15 @@ type options struct {
|
||||
codec Codec
|
||||
cp Compressor
|
||||
dc Decompressor
|
||||
maxMsgSize int
|
||||
unaryInt UnaryServerInterceptor
|
||||
streamInt StreamServerInterceptor
|
||||
maxConcurrentStreams uint32
|
||||
useHandlerImpl bool // use http.Handler-based server
|
||||
}
|
||||
|
||||
var defaultMaxMsgSize = 1024 * 1024 * 4 // use 4MB as the default message size limit
|
||||
|
||||
// A ServerOption sets options.
|
||||
type ServerOption func(*options)
|
||||
|
||||
@ -117,20 +124,28 @@ func CustomCodec(codec Codec) ServerOption {
|
||||
}
|
||||
}
|
||||
|
||||
// RPCCompressor returns a ServerOption that sets a compressor for outbound message.
|
||||
// RPCCompressor returns a ServerOption that sets a compressor for outbound messages.
|
||||
func RPCCompressor(cp Compressor) ServerOption {
|
||||
return func(o *options) {
|
||||
o.cp = cp
|
||||
}
|
||||
}
|
||||
|
||||
// RPCDecompressor returns a ServerOption that sets a decompressor for inbound message.
|
||||
// RPCDecompressor returns a ServerOption that sets a decompressor for inbound messages.
|
||||
func RPCDecompressor(dc Decompressor) ServerOption {
|
||||
return func(o *options) {
|
||||
o.dc = dc
|
||||
}
|
||||
}
|
||||
|
||||
// MaxMsgSize returns a ServerOption to set the max message size in bytes for inbound mesages.
|
||||
// If this is not set, gRPC uses the default 4MB.
|
||||
func MaxMsgSize(m int) ServerOption {
|
||||
return func(o *options) {
|
||||
o.maxMsgSize = m
|
||||
}
|
||||
}
|
||||
|
||||
// MaxConcurrentStreams returns a ServerOption that will apply a limit on the number
|
||||
// of concurrent streams to each ServerTransport.
|
||||
func MaxConcurrentStreams(n uint32) ServerOption {
|
||||
@ -173,6 +188,7 @@ func StreamInterceptor(i StreamServerInterceptor) ServerOption {
|
||||
// started to accept requests yet.
|
||||
func NewServer(opt ...ServerOption) *Server {
|
||||
var opts options
|
||||
opts.maxMsgSize = defaultMaxMsgSize
|
||||
for _, o := range opt {
|
||||
o(&opts)
|
||||
}
|
||||
@ -186,6 +202,7 @@ func NewServer(opt ...ServerOption) *Server {
|
||||
conns: make(map[io.Closer]bool),
|
||||
m: make(map[string]*service),
|
||||
}
|
||||
s.cv = sync.NewCond(&s.mu)
|
||||
if EnableTracing {
|
||||
_, file, line, _ := runtime.Caller(1)
|
||||
s.events = trace.NewEventLog("grpc.Server", fmt.Sprintf("%s:%d", file, line))
|
||||
@ -264,8 +281,8 @@ type ServiceInfo struct {
|
||||
|
||||
// GetServiceInfo returns a map from service names to ServiceInfo.
|
||||
// Service names include the package names, in the form of <package>.<service>.
|
||||
func (s *Server) GetServiceInfo() map[string]*ServiceInfo {
|
||||
ret := make(map[string]*ServiceInfo)
|
||||
func (s *Server) GetServiceInfo() map[string]ServiceInfo {
|
||||
ret := make(map[string]ServiceInfo)
|
||||
for n, srv := range s.m {
|
||||
methods := make([]MethodInfo, 0, len(srv.md)+len(srv.sd))
|
||||
for m := range srv.md {
|
||||
@ -283,7 +300,7 @@ func (s *Server) GetServiceInfo() map[string]*ServiceInfo {
|
||||
})
|
||||
}
|
||||
|
||||
ret[n] = &ServiceInfo{
|
||||
ret[n] = ServiceInfo{
|
||||
Methods: methods,
|
||||
Metadata: srv.mdata,
|
||||
}
|
||||
@ -468,7 +485,7 @@ func (s *Server) traceInfo(st transport.ServerTransport, stream *transport.Strea
|
||||
func (s *Server) addConn(c io.Closer) bool {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
if s.conns == nil {
|
||||
if s.conns == nil || s.drain {
|
||||
return false
|
||||
}
|
||||
s.conns[c] = true
|
||||
@ -480,6 +497,7 @@ func (s *Server) removeConn(c io.Closer) {
|
||||
defer s.mu.Unlock()
|
||||
if s.conns != nil {
|
||||
delete(s.conns, c)
|
||||
s.cv.Signal()
|
||||
}
|
||||
}
|
||||
|
||||
@ -520,7 +538,7 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport.
|
||||
}
|
||||
p := &parser{r: stream}
|
||||
for {
|
||||
pf, req, err := p.recvMsg()
|
||||
pf, req, err := p.recvMsg(s.opts.maxMsgSize)
|
||||
if err == io.EOF {
|
||||
// The entire stream is done (for unary RPC only).
|
||||
return err
|
||||
@ -530,6 +548,10 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport.
|
||||
}
|
||||
if err != nil {
|
||||
switch err := err.(type) {
|
||||
case *rpcError:
|
||||
if err := t.WriteStatus(stream, err.code, err.desc); err != nil {
|
||||
grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", err)
|
||||
}
|
||||
case transport.ConnectionError:
|
||||
// Nothing to do here.
|
||||
case transport.StreamError:
|
||||
@ -569,6 +591,12 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport.
|
||||
return err
|
||||
}
|
||||
}
|
||||
if len(req) > s.opts.maxMsgSize {
|
||||
// TODO: Revisit the error code. Currently keep it consistent with
|
||||
// java implementation.
|
||||
statusCode = codes.Internal
|
||||
statusDesc = fmt.Sprintf("grpc: server received a message of %d bytes exceeding %d limit", len(req), s.opts.maxMsgSize)
|
||||
}
|
||||
if err := s.opts.codec.Unmarshal(req, v); err != nil {
|
||||
return err
|
||||
}
|
||||
@ -628,13 +656,14 @@ func (s *Server) processStreamingRPC(t transport.ServerTransport, stream *transp
|
||||
stream.SetSendCompress(s.opts.cp.Type())
|
||||
}
|
||||
ss := &serverStream{
|
||||
t: t,
|
||||
s: stream,
|
||||
p: &parser{r: stream},
|
||||
codec: s.opts.codec,
|
||||
cp: s.opts.cp,
|
||||
dc: s.opts.dc,
|
||||
trInfo: trInfo,
|
||||
t: t,
|
||||
s: stream,
|
||||
p: &parser{r: stream},
|
||||
codec: s.opts.codec,
|
||||
cp: s.opts.cp,
|
||||
dc: s.opts.dc,
|
||||
maxMsgSize: s.opts.maxMsgSize,
|
||||
trInfo: trInfo,
|
||||
}
|
||||
if ss.cp != nil {
|
||||
ss.cbuf = new(bytes.Buffer)
|
||||
@ -766,14 +795,16 @@ func (s *Server) Stop() {
|
||||
s.mu.Lock()
|
||||
listeners := s.lis
|
||||
s.lis = nil
|
||||
cs := s.conns
|
||||
st := s.conns
|
||||
s.conns = nil
|
||||
// interrupt GracefulStop if Stop and GracefulStop are called concurrently.
|
||||
s.cv.Signal()
|
||||
s.mu.Unlock()
|
||||
|
||||
for lis := range listeners {
|
||||
lis.Close()
|
||||
}
|
||||
for c := range cs {
|
||||
for c := range st {
|
||||
c.Close()
|
||||
}
|
||||
|
||||
@ -785,6 +816,32 @@ func (s *Server) Stop() {
|
||||
s.mu.Unlock()
|
||||
}
|
||||
|
||||
// GracefulStop stops the gRPC server gracefully. It stops the server to accept new
|
||||
// connections and RPCs and blocks until all the pending RPCs are finished.
|
||||
func (s *Server) GracefulStop() {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
if s.drain == true || s.conns == nil {
|
||||
return
|
||||
}
|
||||
s.drain = true
|
||||
for lis := range s.lis {
|
||||
lis.Close()
|
||||
}
|
||||
s.lis = nil
|
||||
for c := range s.conns {
|
||||
c.(transport.ServerTransport).Drain()
|
||||
}
|
||||
for len(s.conns) != 0 {
|
||||
s.cv.Wait()
|
||||
}
|
||||
s.conns = nil
|
||||
if s.events != nil {
|
||||
s.events.Finish()
|
||||
s.events = nil
|
||||
}
|
||||
}
|
||||
|
||||
func init() {
|
||||
internal.TestingCloseConns = func(arg interface{}) {
|
||||
arg.(*Server).testingCloseConns()
|
||||
|
115
cmd/vendor/google.golang.org/grpc/stream.go
generated
vendored
115
cmd/vendor/google.golang.org/grpc/stream.go
generated
vendored
@ -37,6 +37,7 @@ import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"io"
|
||||
"math"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -84,12 +85,9 @@ type ClientStream interface {
|
||||
// Header returns the header metadata received from the server if there
|
||||
// is any. It blocks if the metadata is not ready to read.
|
||||
Header() (metadata.MD, error)
|
||||
// Trailer returns the trailer metadata from the server. It must be called
|
||||
// after stream.Recv() returns non-nil error (including io.EOF) for
|
||||
// bi-directional streaming and server streaming or stream.CloseAndRecv()
|
||||
// returns for client streaming in order to receive trailer metadata if
|
||||
// present. Otherwise, it could returns an empty MD even though trailer
|
||||
// is present.
|
||||
// Trailer returns the trailer metadata from the server, if there is any.
|
||||
// It must only be called after stream.CloseAndRecv has returned, or
|
||||
// stream.Recv has returned a non-nil error (including io.EOF).
|
||||
Trailer() metadata.MD
|
||||
// CloseSend closes the send direction of the stream. It closes the stream
|
||||
// when non-nil error is met.
|
||||
@ -99,11 +97,10 @@ type ClientStream interface {
|
||||
|
||||
// NewClientStream creates a new Stream for the client side. This is called
|
||||
// by generated code.
|
||||
func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, opts ...CallOption) (ClientStream, error) {
|
||||
func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, opts ...CallOption) (_ ClientStream, err error) {
|
||||
var (
|
||||
t transport.ClientTransport
|
||||
s *transport.Stream
|
||||
err error
|
||||
put func()
|
||||
)
|
||||
c := defaultCallInfo
|
||||
@ -120,27 +117,24 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
if cc.dopts.cp != nil {
|
||||
callHdr.SendCompress = cc.dopts.cp.Type()
|
||||
}
|
||||
cs := &clientStream{
|
||||
opts: opts,
|
||||
c: c,
|
||||
desc: desc,
|
||||
codec: cc.dopts.codec,
|
||||
cp: cc.dopts.cp,
|
||||
dc: cc.dopts.dc,
|
||||
tracing: EnableTracing,
|
||||
}
|
||||
if cc.dopts.cp != nil {
|
||||
callHdr.SendCompress = cc.dopts.cp.Type()
|
||||
cs.cbuf = new(bytes.Buffer)
|
||||
}
|
||||
if cs.tracing {
|
||||
cs.trInfo.tr = trace.New("grpc.Sent."+methodFamily(method), method)
|
||||
cs.trInfo.firstLine.client = true
|
||||
var trInfo traceInfo
|
||||
if EnableTracing {
|
||||
trInfo.tr = trace.New("grpc.Sent."+methodFamily(method), method)
|
||||
trInfo.firstLine.client = true
|
||||
if deadline, ok := ctx.Deadline(); ok {
|
||||
cs.trInfo.firstLine.deadline = deadline.Sub(time.Now())
|
||||
trInfo.firstLine.deadline = deadline.Sub(time.Now())
|
||||
}
|
||||
cs.trInfo.tr.LazyLog(&cs.trInfo.firstLine, false)
|
||||
ctx = trace.NewContext(ctx, cs.trInfo.tr)
|
||||
trInfo.tr.LazyLog(&trInfo.firstLine, false)
|
||||
ctx = trace.NewContext(ctx, trInfo.tr)
|
||||
defer func() {
|
||||
if err != nil {
|
||||
// Need to call tr.finish() if error is returned.
|
||||
// Because tr will not be returned to caller.
|
||||
trInfo.tr.LazyPrintf("RPC: [%v]", err)
|
||||
trInfo.tr.SetError()
|
||||
trInfo.tr.Finish()
|
||||
}
|
||||
}()
|
||||
}
|
||||
gopts := BalancerGetOptions{
|
||||
BlockingWait: !c.failFast,
|
||||
@ -152,9 +146,9 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
if _, ok := err.(*rpcError); ok {
|
||||
return nil, err
|
||||
}
|
||||
if err == errConnClosing {
|
||||
if err == errConnClosing || err == errConnUnavailable {
|
||||
if c.failFast {
|
||||
return nil, Errorf(codes.Unavailable, "%v", errConnClosing)
|
||||
return nil, Errorf(codes.Unavailable, "%v", err)
|
||||
}
|
||||
continue
|
||||
}
|
||||
@ -168,9 +162,8 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
put()
|
||||
put = nil
|
||||
}
|
||||
if _, ok := err.(transport.ConnectionError); ok {
|
||||
if _, ok := err.(transport.ConnectionError); ok || err == transport.ErrStreamDrain {
|
||||
if c.failFast {
|
||||
cs.finish(err)
|
||||
return nil, toRPCErr(err)
|
||||
}
|
||||
continue
|
||||
@ -179,16 +172,43 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
}
|
||||
break
|
||||
}
|
||||
cs.put = put
|
||||
cs.t = t
|
||||
cs.s = s
|
||||
cs.p = &parser{r: s}
|
||||
// Listen on ctx.Done() to detect cancellation when there is no pending
|
||||
// I/O operations on this stream.
|
||||
cs := &clientStream{
|
||||
opts: opts,
|
||||
c: c,
|
||||
desc: desc,
|
||||
codec: cc.dopts.codec,
|
||||
cp: cc.dopts.cp,
|
||||
dc: cc.dopts.dc,
|
||||
|
||||
put: put,
|
||||
t: t,
|
||||
s: s,
|
||||
p: &parser{r: s},
|
||||
|
||||
tracing: EnableTracing,
|
||||
trInfo: trInfo,
|
||||
}
|
||||
if cc.dopts.cp != nil {
|
||||
cs.cbuf = new(bytes.Buffer)
|
||||
}
|
||||
// Listen on ctx.Done() to detect cancellation and s.Done() to detect normal termination
|
||||
// when there is no pending I/O operations on this stream.
|
||||
go func() {
|
||||
select {
|
||||
case <-t.Error():
|
||||
// Incur transport error, simply exit.
|
||||
case <-s.Done():
|
||||
// TODO: The trace of the RPC is terminated here when there is no pending
|
||||
// I/O, which is probably not the optimal solution.
|
||||
if s.StatusCode() == codes.OK {
|
||||
cs.finish(nil)
|
||||
} else {
|
||||
cs.finish(Errorf(s.StatusCode(), "%s", s.StatusDesc()))
|
||||
}
|
||||
cs.closeTransportStream(nil)
|
||||
case <-s.GoAway():
|
||||
cs.finish(errConnDrain)
|
||||
cs.closeTransportStream(errConnDrain)
|
||||
case <-s.Context().Done():
|
||||
err := s.Context().Err()
|
||||
cs.finish(err)
|
||||
@ -251,7 +271,17 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) {
|
||||
if err != nil {
|
||||
cs.finish(err)
|
||||
}
|
||||
if err == nil || err == io.EOF {
|
||||
if err == nil {
|
||||
return
|
||||
}
|
||||
if err == io.EOF {
|
||||
// Specialize the process for server streaming. SendMesg is only called
|
||||
// once when creating the stream object. io.EOF needs to be skipped when
|
||||
// the rpc is early finished (before the stream object is created.).
|
||||
// TODO: It is probably better to move this into the generated code.
|
||||
if !cs.desc.ClientStreams && cs.desc.ServerStreams {
|
||||
err = nil
|
||||
}
|
||||
return
|
||||
}
|
||||
if _, ok := err.(transport.ConnectionError); !ok {
|
||||
@ -272,7 +302,7 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) {
|
||||
}
|
||||
|
||||
func (cs *clientStream) RecvMsg(m interface{}) (err error) {
|
||||
err = recv(cs.p, cs.codec, cs.s, cs.dc, m)
|
||||
err = recv(cs.p, cs.codec, cs.s, cs.dc, m, math.MaxInt32)
|
||||
defer func() {
|
||||
// err != nil indicates the termination of the stream.
|
||||
if err != nil {
|
||||
@ -291,7 +321,7 @@ func (cs *clientStream) RecvMsg(m interface{}) (err error) {
|
||||
return
|
||||
}
|
||||
// Special handling for client streaming rpc.
|
||||
err = recv(cs.p, cs.codec, cs.s, cs.dc, m)
|
||||
err = recv(cs.p, cs.codec, cs.s, cs.dc, m, math.MaxInt32)
|
||||
cs.closeTransportStream(err)
|
||||
if err == nil {
|
||||
return toRPCErr(errors.New("grpc: client streaming protocol violation: get <nil>, want <EOF>"))
|
||||
@ -326,7 +356,7 @@ func (cs *clientStream) CloseSend() (err error) {
|
||||
}
|
||||
}()
|
||||
if err == nil || err == io.EOF {
|
||||
return
|
||||
return nil
|
||||
}
|
||||
if _, ok := err.(transport.ConnectionError); !ok {
|
||||
cs.closeTransportStream(err)
|
||||
@ -392,6 +422,7 @@ type serverStream struct {
|
||||
cp Compressor
|
||||
dc Decompressor
|
||||
cbuf *bytes.Buffer
|
||||
maxMsgSize int
|
||||
statusCode codes.Code
|
||||
statusDesc string
|
||||
trInfo *traceInfo
|
||||
@ -458,5 +489,5 @@ func (ss *serverStream) RecvMsg(m interface{}) (err error) {
|
||||
ss.mu.Unlock()
|
||||
}
|
||||
}()
|
||||
return recv(ss.p, ss.codec, ss.s, ss.dc, m)
|
||||
return recv(ss.p, ss.codec, ss.s, ss.dc, m, ss.maxMsgSize)
|
||||
}
|
||||
|
5
cmd/vendor/google.golang.org/grpc/transport/control.go
generated
vendored
5
cmd/vendor/google.golang.org/grpc/transport/control.go
generated
vendored
@ -72,6 +72,11 @@ type resetStream struct {
|
||||
|
||||
func (*resetStream) item() {}
|
||||
|
||||
type goAway struct {
|
||||
}
|
||||
|
||||
func (*goAway) item() {}
|
||||
|
||||
type flushIO struct {
|
||||
}
|
||||
|
||||
|
46
cmd/vendor/google.golang.org/grpc/transport/go16.go
generated
vendored
Normal file
46
cmd/vendor/google.golang.org/grpc/transport/go16.go
generated
vendored
Normal file
@ -0,0 +1,46 @@
|
||||
// +build go1.6,!go1.7
|
||||
|
||||
/*
|
||||
* Copyright 2016, Google Inc.
|
||||
* All rights reserved.
|
||||
*
|
||||
* Redistribution and use in source and binary forms, with or without
|
||||
* modification, are permitted provided that the following conditions are
|
||||
* met:
|
||||
*
|
||||
* * Redistributions of source code must retain the above copyright
|
||||
* notice, this list of conditions and the following disclaimer.
|
||||
* * Redistributions in binary form must reproduce the above
|
||||
* copyright notice, this list of conditions and the following disclaimer
|
||||
* in the documentation and/or other materials provided with the
|
||||
* distribution.
|
||||
* * Neither the name of Google Inc. nor the names of its
|
||||
* contributors may be used to endorse or promote products derived from
|
||||
* this software without specific prior written permission.
|
||||
*
|
||||
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
*
|
||||
*/
|
||||
|
||||
package transport
|
||||
|
||||
import (
|
||||
"net"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
// dialContext connects to the address on the named network.
|
||||
func dialContext(ctx context.Context, network, address string) (net.Conn, error) {
|
||||
return (&net.Dialer{Cancel: ctx.Done()}).Dial(network, address)
|
||||
}
|
46
cmd/vendor/google.golang.org/grpc/transport/go17.go
generated
vendored
Normal file
46
cmd/vendor/google.golang.org/grpc/transport/go17.go
generated
vendored
Normal file
@ -0,0 +1,46 @@
|
||||
// +build go1.7
|
||||
|
||||
/*
|
||||
* Copyright 2016, Google Inc.
|
||||
* All rights reserved.
|
||||
*
|
||||
* Redistribution and use in source and binary forms, with or without
|
||||
* modification, are permitted provided that the following conditions are
|
||||
* met:
|
||||
*
|
||||
* * Redistributions of source code must retain the above copyright
|
||||
* notice, this list of conditions and the following disclaimer.
|
||||
* * Redistributions in binary form must reproduce the above
|
||||
* copyright notice, this list of conditions and the following disclaimer
|
||||
* in the documentation and/or other materials provided with the
|
||||
* distribution.
|
||||
* * Neither the name of Google Inc. nor the names of its
|
||||
* contributors may be used to endorse or promote products derived from
|
||||
* this software without specific prior written permission.
|
||||
*
|
||||
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
*
|
||||
*/
|
||||
|
||||
package transport
|
||||
|
||||
import (
|
||||
"net"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
// dialContext connects to the address on the named network.
|
||||
func dialContext(ctx context.Context, network, address string) (net.Conn, error) {
|
||||
return (&net.Dialer{}).DialContext(ctx, network, address)
|
||||
}
|
8
cmd/vendor/google.golang.org/grpc/transport/handler_server.go
generated
vendored
8
cmd/vendor/google.golang.org/grpc/transport/handler_server.go
generated
vendored
@ -83,7 +83,7 @@ func NewServerHandlerTransport(w http.ResponseWriter, r *http.Request) (ServerTr
|
||||
}
|
||||
|
||||
if v := r.Header.Get("grpc-timeout"); v != "" {
|
||||
to, err := timeoutDecode(v)
|
||||
to, err := decodeTimeout(v)
|
||||
if err != nil {
|
||||
return nil, StreamErrorf(codes.Internal, "malformed time-out: %v", err)
|
||||
}
|
||||
@ -194,7 +194,7 @@ func (ht *serverHandlerTransport) WriteStatus(s *Stream, statusCode codes.Code,
|
||||
h := ht.rw.Header()
|
||||
h.Set("Grpc-Status", fmt.Sprintf("%d", statusCode))
|
||||
if statusDesc != "" {
|
||||
h.Set("Grpc-Message", statusDesc)
|
||||
h.Set("Grpc-Message", encodeGrpcMessage(statusDesc))
|
||||
}
|
||||
if md := s.Trailer(); len(md) > 0 {
|
||||
for k, vv := range md {
|
||||
@ -370,6 +370,10 @@ func (ht *serverHandlerTransport) runStream() {
|
||||
}
|
||||
}
|
||||
|
||||
func (ht *serverHandlerTransport) Drain() {
|
||||
panic("Drain() is not implemented")
|
||||
}
|
||||
|
||||
// mapRecvMsgError returns the non-nil err into the appropriate
|
||||
// error value as expected by callers of *grpc.parser.recvMsg.
|
||||
// In particular, in can only be:
|
||||
|
208
cmd/vendor/google.golang.org/grpc/transport/http2_client.go
generated
vendored
208
cmd/vendor/google.golang.org/grpc/transport/http2_client.go
generated
vendored
@ -35,6 +35,7 @@ package transport
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
"net"
|
||||
@ -71,6 +72,9 @@ type http2Client struct {
|
||||
shutdownChan chan struct{}
|
||||
// errorChan is closed to notify the I/O error to the caller.
|
||||
errorChan chan struct{}
|
||||
// goAway is closed to notify the upper layer (i.e., addrConn.transportMonitor)
|
||||
// that the server sent GoAway on this transport.
|
||||
goAway chan struct{}
|
||||
|
||||
framer *framer
|
||||
hBuf *bytes.Buffer // the buffer for HPACK encoding
|
||||
@ -97,41 +101,44 @@ type http2Client struct {
|
||||
maxStreams int
|
||||
// the per-stream outbound flow control window size set by the peer.
|
||||
streamSendQuota uint32
|
||||
// goAwayID records the Last-Stream-ID in the GoAway frame from the server.
|
||||
goAwayID uint32
|
||||
// prevGoAway ID records the Last-Stream-ID in the previous GOAway frame.
|
||||
prevGoAwayID uint32
|
||||
}
|
||||
|
||||
func dial(fn func(context.Context, string) (net.Conn, error), ctx context.Context, addr string) (net.Conn, error) {
|
||||
if fn != nil {
|
||||
return fn(ctx, addr)
|
||||
}
|
||||
return dialContext(ctx, "tcp", addr)
|
||||
}
|
||||
|
||||
// newHTTP2Client constructs a connected ClientTransport to addr based on HTTP2
|
||||
// and starts to receive messages on it. Non-nil error returns if construction
|
||||
// fails.
|
||||
func newHTTP2Client(addr string, opts *ConnectOptions) (_ ClientTransport, err error) {
|
||||
if opts.Dialer == nil {
|
||||
// Set the default Dialer.
|
||||
opts.Dialer = func(addr string, timeout time.Duration) (net.Conn, error) {
|
||||
return net.DialTimeout("tcp", addr, timeout)
|
||||
}
|
||||
}
|
||||
func newHTTP2Client(ctx context.Context, addr string, opts ConnectOptions) (_ ClientTransport, err error) {
|
||||
scheme := "http"
|
||||
startT := time.Now()
|
||||
timeout := opts.Timeout
|
||||
conn, connErr := opts.Dialer(addr, timeout)
|
||||
conn, connErr := dial(opts.Dialer, ctx, addr)
|
||||
if connErr != nil {
|
||||
return nil, ConnectionErrorf("transport: %v", connErr)
|
||||
return nil, ConnectionErrorf(true, connErr, "transport: %v", connErr)
|
||||
}
|
||||
var authInfo credentials.AuthInfo
|
||||
if opts.TransportCredentials != nil {
|
||||
scheme = "https"
|
||||
if timeout > 0 {
|
||||
timeout -= time.Since(startT)
|
||||
}
|
||||
conn, authInfo, connErr = opts.TransportCredentials.ClientHandshake(addr, conn, timeout)
|
||||
}
|
||||
if connErr != nil {
|
||||
return nil, ConnectionErrorf("transport: %v", connErr)
|
||||
}
|
||||
defer func() {
|
||||
// Any further errors will close the underlying connection
|
||||
defer func(conn net.Conn) {
|
||||
if err != nil {
|
||||
conn.Close()
|
||||
}
|
||||
}()
|
||||
}(conn)
|
||||
var authInfo credentials.AuthInfo
|
||||
if creds := opts.TransportCredentials; creds != nil {
|
||||
scheme = "https"
|
||||
conn, authInfo, connErr = creds.ClientHandshake(ctx, addr, conn)
|
||||
}
|
||||
if connErr != nil {
|
||||
// Credentials handshake error is not a temporary error (unless the error
|
||||
// was the connection closing).
|
||||
return nil, ConnectionErrorf(connErr == io.EOF, connErr, "transport: %v", connErr)
|
||||
}
|
||||
ua := primaryUA
|
||||
if opts.UserAgent != "" {
|
||||
ua = opts.UserAgent + " " + ua
|
||||
@ -147,6 +154,7 @@ func newHTTP2Client(addr string, opts *ConnectOptions) (_ ClientTransport, err e
|
||||
writableChan: make(chan int, 1),
|
||||
shutdownChan: make(chan struct{}),
|
||||
errorChan: make(chan struct{}),
|
||||
goAway: make(chan struct{}),
|
||||
framer: newFramer(conn),
|
||||
hBuf: &buf,
|
||||
hEnc: hpack.NewEncoder(&buf),
|
||||
@ -168,11 +176,11 @@ func newHTTP2Client(addr string, opts *ConnectOptions) (_ ClientTransport, err e
|
||||
n, err := t.conn.Write(clientPreface)
|
||||
if err != nil {
|
||||
t.Close()
|
||||
return nil, ConnectionErrorf("transport: %v", err)
|
||||
return nil, ConnectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
if n != len(clientPreface) {
|
||||
t.Close()
|
||||
return nil, ConnectionErrorf("transport: preface mismatch, wrote %d bytes; want %d", n, len(clientPreface))
|
||||
return nil, ConnectionErrorf(true, err, "transport: preface mismatch, wrote %d bytes; want %d", n, len(clientPreface))
|
||||
}
|
||||
if initialWindowSize != defaultWindowSize {
|
||||
err = t.framer.writeSettings(true, http2.Setting{
|
||||
@ -184,13 +192,13 @@ func newHTTP2Client(addr string, opts *ConnectOptions) (_ ClientTransport, err e
|
||||
}
|
||||
if err != nil {
|
||||
t.Close()
|
||||
return nil, ConnectionErrorf("transport: %v", err)
|
||||
return nil, ConnectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
// Adjust the connection flow control window if needed.
|
||||
if delta := uint32(initialConnWindowSize - defaultWindowSize); delta > 0 {
|
||||
if err := t.framer.writeWindowUpdate(true, 0, delta); err != nil {
|
||||
t.Close()
|
||||
return nil, ConnectionErrorf("transport: %v", err)
|
||||
return nil, ConnectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
}
|
||||
go t.controller()
|
||||
@ -202,6 +210,8 @@ func (t *http2Client) newStream(ctx context.Context, callHdr *CallHdr) *Stream {
|
||||
// TODO(zhaoq): Handle uint32 overflow of Stream.id.
|
||||
s := &Stream{
|
||||
id: t.nextID,
|
||||
done: make(chan struct{}),
|
||||
goAway: make(chan struct{}),
|
||||
method: callHdr.Method,
|
||||
sendCompress: callHdr.SendCompress,
|
||||
buf: newRecvBuffer(),
|
||||
@ -216,8 +226,9 @@ func (t *http2Client) newStream(ctx context.Context, callHdr *CallHdr) *Stream {
|
||||
// Make a stream be able to cancel the pending operations by itself.
|
||||
s.ctx, s.cancel = context.WithCancel(ctx)
|
||||
s.dec = &recvBufferReader{
|
||||
ctx: s.ctx,
|
||||
recv: s.buf,
|
||||
ctx: s.ctx,
|
||||
goAway: s.goAway,
|
||||
recv: s.buf,
|
||||
}
|
||||
return s
|
||||
}
|
||||
@ -271,6 +282,10 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
t.mu.Unlock()
|
||||
return nil, ErrConnClosing
|
||||
}
|
||||
if t.state == draining {
|
||||
t.mu.Unlock()
|
||||
return nil, ErrStreamDrain
|
||||
}
|
||||
if t.state != reachable {
|
||||
t.mu.Unlock()
|
||||
return nil, ErrConnClosing
|
||||
@ -278,7 +293,7 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
checkStreamsQuota := t.streamsQuota != nil
|
||||
t.mu.Unlock()
|
||||
if checkStreamsQuota {
|
||||
sq, err := wait(ctx, t.shutdownChan, t.streamsQuota.acquire())
|
||||
sq, err := wait(ctx, nil, nil, t.shutdownChan, t.streamsQuota.acquire())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -287,7 +302,7 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
t.streamsQuota.add(sq - 1)
|
||||
}
|
||||
}
|
||||
if _, err := wait(ctx, t.shutdownChan, t.writableChan); err != nil {
|
||||
if _, err := wait(ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil {
|
||||
// Return the quota back now because there is no stream returned to the caller.
|
||||
if _, ok := err.(StreamError); ok && checkStreamsQuota {
|
||||
t.streamsQuota.add(1)
|
||||
@ -295,6 +310,15 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
return nil, err
|
||||
}
|
||||
t.mu.Lock()
|
||||
if t.state == draining {
|
||||
t.mu.Unlock()
|
||||
if checkStreamsQuota {
|
||||
t.streamsQuota.add(1)
|
||||
}
|
||||
// Need to make t writable again so that the rpc in flight can still proceed.
|
||||
t.writableChan <- 0
|
||||
return nil, ErrStreamDrain
|
||||
}
|
||||
if t.state != reachable {
|
||||
t.mu.Unlock()
|
||||
return nil, ErrConnClosing
|
||||
@ -329,7 +353,7 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-encoding", Value: callHdr.SendCompress})
|
||||
}
|
||||
if timeout > 0 {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-timeout", Value: timeoutEncode(timeout)})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-timeout", Value: encodeTimeout(timeout)})
|
||||
}
|
||||
for k, v := range authData {
|
||||
// Capital header names are illegal in HTTP/2.
|
||||
@ -384,7 +408,7 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
}
|
||||
if err != nil {
|
||||
t.notifyError(err)
|
||||
return nil, ConnectionErrorf("transport: %v", err)
|
||||
return nil, ConnectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
}
|
||||
t.writableChan <- 0
|
||||
@ -403,22 +427,17 @@ func (t *http2Client) CloseStream(s *Stream, err error) {
|
||||
if t.streamsQuota != nil {
|
||||
updateStreams = true
|
||||
}
|
||||
if t.state == draining && len(t.activeStreams) == 1 {
|
||||
delete(t.activeStreams, s.id)
|
||||
if t.state == draining && len(t.activeStreams) == 0 {
|
||||
// The transport is draining and s is the last live stream on t.
|
||||
t.mu.Unlock()
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
delete(t.activeStreams, s.id)
|
||||
t.mu.Unlock()
|
||||
if updateStreams {
|
||||
t.streamsQuota.add(1)
|
||||
}
|
||||
// In case stream sending and receiving are invoked in separate
|
||||
// goroutines (e.g., bi-directional streaming), the caller needs
|
||||
// to call cancel on the stream to interrupt the blocking on
|
||||
// other goroutines.
|
||||
s.cancel()
|
||||
s.mu.Lock()
|
||||
if q := s.fc.resetPendingData(); q > 0 {
|
||||
if n := t.fc.onRead(q); n > 0 {
|
||||
@ -445,13 +464,13 @@ func (t *http2Client) CloseStream(s *Stream, err error) {
|
||||
// accessed any more.
|
||||
func (t *http2Client) Close() (err error) {
|
||||
t.mu.Lock()
|
||||
if t.state == reachable {
|
||||
close(t.errorChan)
|
||||
}
|
||||
if t.state == closing {
|
||||
t.mu.Unlock()
|
||||
return
|
||||
}
|
||||
if t.state == reachable || t.state == draining {
|
||||
close(t.errorChan)
|
||||
}
|
||||
t.state = closing
|
||||
t.mu.Unlock()
|
||||
close(t.shutdownChan)
|
||||
@ -475,10 +494,35 @@ func (t *http2Client) Close() (err error) {
|
||||
|
||||
func (t *http2Client) GracefulClose() error {
|
||||
t.mu.Lock()
|
||||
if t.state == closing {
|
||||
switch t.state {
|
||||
case unreachable:
|
||||
// The server may close the connection concurrently. t is not available for
|
||||
// any streams. Close it now.
|
||||
t.mu.Unlock()
|
||||
t.Close()
|
||||
return nil
|
||||
case closing:
|
||||
t.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
// Notify the streams which were initiated after the server sent GOAWAY.
|
||||
select {
|
||||
case <-t.goAway:
|
||||
n := t.prevGoAwayID
|
||||
if n == 0 && t.nextID > 1 {
|
||||
n = t.nextID - 2
|
||||
}
|
||||
m := t.goAwayID + 2
|
||||
if m == 2 {
|
||||
m = 1
|
||||
}
|
||||
for i := m; i <= n; i += 2 {
|
||||
if s, ok := t.activeStreams[i]; ok {
|
||||
close(s.goAway)
|
||||
}
|
||||
}
|
||||
default:
|
||||
}
|
||||
if t.state == draining {
|
||||
t.mu.Unlock()
|
||||
return nil
|
||||
@ -504,15 +548,15 @@ func (t *http2Client) Write(s *Stream, data []byte, opts *Options) error {
|
||||
size := http2MaxFrameLen
|
||||
s.sendQuotaPool.add(0)
|
||||
// Wait until the stream has some quota to send the data.
|
||||
sq, err := wait(s.ctx, t.shutdownChan, s.sendQuotaPool.acquire())
|
||||
sq, err := wait(s.ctx, s.done, s.goAway, t.shutdownChan, s.sendQuotaPool.acquire())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
t.sendQuotaPool.add(0)
|
||||
// Wait until the transport has some quota to send the data.
|
||||
tq, err := wait(s.ctx, t.shutdownChan, t.sendQuotaPool.acquire())
|
||||
tq, err := wait(s.ctx, s.done, s.goAway, t.shutdownChan, t.sendQuotaPool.acquire())
|
||||
if err != nil {
|
||||
if _, ok := err.(StreamError); ok {
|
||||
if _, ok := err.(StreamError); ok || err == io.EOF {
|
||||
t.sendQuotaPool.cancel()
|
||||
}
|
||||
return err
|
||||
@ -544,8 +588,8 @@ func (t *http2Client) Write(s *Stream, data []byte, opts *Options) error {
|
||||
// Indicate there is a writer who is about to write a data frame.
|
||||
t.framer.adjustNumWriters(1)
|
||||
// Got some quota. Try to acquire writing privilege on the transport.
|
||||
if _, err := wait(s.ctx, t.shutdownChan, t.writableChan); err != nil {
|
||||
if _, ok := err.(StreamError); ok {
|
||||
if _, err := wait(s.ctx, s.done, s.goAway, t.shutdownChan, t.writableChan); err != nil {
|
||||
if _, ok := err.(StreamError); ok || err == io.EOF {
|
||||
// Return the connection quota back.
|
||||
t.sendQuotaPool.add(len(p))
|
||||
}
|
||||
@ -578,7 +622,7 @@ func (t *http2Client) Write(s *Stream, data []byte, opts *Options) error {
|
||||
// invoked.
|
||||
if err := t.framer.writeData(forceFlush, s.id, endStream, p); err != nil {
|
||||
t.notifyError(err)
|
||||
return ConnectionErrorf("transport: %v", err)
|
||||
return ConnectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
if t.framer.adjustNumWriters(-1) == 0 {
|
||||
t.framer.flushWrite()
|
||||
@ -593,11 +637,7 @@ func (t *http2Client) Write(s *Stream, data []byte, opts *Options) error {
|
||||
}
|
||||
s.mu.Lock()
|
||||
if s.state != streamDone {
|
||||
if s.state == streamReadDone {
|
||||
s.state = streamDone
|
||||
} else {
|
||||
s.state = streamWriteDone
|
||||
}
|
||||
s.state = streamWriteDone
|
||||
}
|
||||
s.mu.Unlock()
|
||||
return nil
|
||||
@ -630,7 +670,7 @@ func (t *http2Client) updateWindow(s *Stream, n uint32) {
|
||||
func (t *http2Client) handleData(f *http2.DataFrame) {
|
||||
size := len(f.Data())
|
||||
if err := t.fc.onData(uint32(size)); err != nil {
|
||||
t.notifyError(ConnectionErrorf("%v", err))
|
||||
t.notifyError(ConnectionErrorf(true, err, "%v", err))
|
||||
return
|
||||
}
|
||||
// Select the right stream to dispatch.
|
||||
@ -655,6 +695,7 @@ func (t *http2Client) handleData(f *http2.DataFrame) {
|
||||
s.state = streamDone
|
||||
s.statusCode = codes.Internal
|
||||
s.statusDesc = err.Error()
|
||||
close(s.done)
|
||||
s.mu.Unlock()
|
||||
s.write(recvMsg{err: io.EOF})
|
||||
t.controlBuf.put(&resetStream{s.id, http2.ErrCodeFlowControl})
|
||||
@ -672,13 +713,14 @@ func (t *http2Client) handleData(f *http2.DataFrame) {
|
||||
// the read direction is closed, and set the status appropriately.
|
||||
if f.FrameHeader.Flags.Has(http2.FlagDataEndStream) {
|
||||
s.mu.Lock()
|
||||
if s.state == streamWriteDone {
|
||||
s.state = streamDone
|
||||
} else {
|
||||
s.state = streamReadDone
|
||||
if s.state == streamDone {
|
||||
s.mu.Unlock()
|
||||
return
|
||||
}
|
||||
s.state = streamDone
|
||||
s.statusCode = codes.Internal
|
||||
s.statusDesc = "server closed the stream without sending trailers"
|
||||
close(s.done)
|
||||
s.mu.Unlock()
|
||||
s.write(recvMsg{err: io.EOF})
|
||||
}
|
||||
@ -704,6 +746,8 @@ func (t *http2Client) handleRSTStream(f *http2.RSTStreamFrame) {
|
||||
grpclog.Println("transport: http2Client.handleRSTStream found no mapped gRPC status for the received http2 error ", f.ErrCode)
|
||||
s.statusCode = codes.Unknown
|
||||
}
|
||||
s.statusDesc = fmt.Sprintf("stream terminated by RST_STREAM with error code: %d", f.ErrCode)
|
||||
close(s.done)
|
||||
s.mu.Unlock()
|
||||
s.write(recvMsg{err: io.EOF})
|
||||
}
|
||||
@ -728,7 +772,32 @@ func (t *http2Client) handlePing(f *http2.PingFrame) {
|
||||
}
|
||||
|
||||
func (t *http2Client) handleGoAway(f *http2.GoAwayFrame) {
|
||||
// TODO(zhaoq): GoAwayFrame handler to be implemented
|
||||
t.mu.Lock()
|
||||
if t.state == reachable || t.state == draining {
|
||||
if f.LastStreamID > 0 && f.LastStreamID%2 != 1 {
|
||||
t.mu.Unlock()
|
||||
t.notifyError(ConnectionErrorf(true, nil, "received illegal http2 GOAWAY frame: stream ID %d is even", f.LastStreamID))
|
||||
return
|
||||
}
|
||||
select {
|
||||
case <-t.goAway:
|
||||
id := t.goAwayID
|
||||
// t.goAway has been closed (i.e.,multiple GoAways).
|
||||
if id < f.LastStreamID {
|
||||
t.mu.Unlock()
|
||||
t.notifyError(ConnectionErrorf(true, nil, "received illegal http2 GOAWAY frame: previously recv GOAWAY frame with LastStramID %d, currently recv %d", id, f.LastStreamID))
|
||||
return
|
||||
}
|
||||
t.prevGoAwayID = id
|
||||
t.goAwayID = f.LastStreamID
|
||||
t.mu.Unlock()
|
||||
return
|
||||
default:
|
||||
}
|
||||
t.goAwayID = f.LastStreamID
|
||||
close(t.goAway)
|
||||
}
|
||||
t.mu.Unlock()
|
||||
}
|
||||
|
||||
func (t *http2Client) handleWindowUpdate(f *http2.WindowUpdateFrame) {
|
||||
@ -780,11 +849,11 @@ func (t *http2Client) operateHeaders(frame *http2.MetaHeadersFrame) {
|
||||
if len(state.mdata) > 0 {
|
||||
s.trailer = state.mdata
|
||||
}
|
||||
s.state = streamDone
|
||||
s.statusCode = state.statusCode
|
||||
s.statusDesc = state.statusDesc
|
||||
close(s.done)
|
||||
s.state = streamDone
|
||||
s.mu.Unlock()
|
||||
|
||||
s.write(recvMsg{err: io.EOF})
|
||||
}
|
||||
|
||||
@ -937,13 +1006,22 @@ func (t *http2Client) Error() <-chan struct{} {
|
||||
return t.errorChan
|
||||
}
|
||||
|
||||
func (t *http2Client) GoAway() <-chan struct{} {
|
||||
return t.goAway
|
||||
}
|
||||
|
||||
func (t *http2Client) notifyError(err error) {
|
||||
t.mu.Lock()
|
||||
defer t.mu.Unlock()
|
||||
// make sure t.errorChan is closed only once.
|
||||
if t.state == draining {
|
||||
t.mu.Unlock()
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
if t.state == reachable {
|
||||
t.state = unreachable
|
||||
close(t.errorChan)
|
||||
grpclog.Printf("transport: http2Client.notifyError got notified that the client transport was broken %v.", err)
|
||||
}
|
||||
t.mu.Unlock()
|
||||
}
|
||||
|
76
cmd/vendor/google.golang.org/grpc/transport/http2_server.go
generated
vendored
76
cmd/vendor/google.golang.org/grpc/transport/http2_server.go
generated
vendored
@ -111,12 +111,12 @@ func newHTTP2Server(conn net.Conn, maxStreams uint32, authInfo credentials.AuthI
|
||||
Val: uint32(initialWindowSize)})
|
||||
}
|
||||
if err := framer.writeSettings(true, settings...); err != nil {
|
||||
return nil, ConnectionErrorf("transport: %v", err)
|
||||
return nil, ConnectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
// Adjust the connection flow control window if needed.
|
||||
if delta := uint32(initialConnWindowSize - defaultWindowSize); delta > 0 {
|
||||
if err := framer.writeWindowUpdate(true, 0, delta); err != nil {
|
||||
return nil, ConnectionErrorf("transport: %v", err)
|
||||
return nil, ConnectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
}
|
||||
var buf bytes.Buffer
|
||||
@ -142,7 +142,7 @@ func newHTTP2Server(conn net.Conn, maxStreams uint32, authInfo credentials.AuthI
|
||||
}
|
||||
|
||||
// operateHeader takes action on the decoded headers.
|
||||
func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(*Stream)) {
|
||||
func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(*Stream)) (close bool) {
|
||||
buf := newRecvBuffer()
|
||||
s := &Stream{
|
||||
id: frame.Header().StreamID,
|
||||
@ -205,6 +205,13 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(
|
||||
t.controlBuf.put(&resetStream{s.id, http2.ErrCodeRefusedStream})
|
||||
return
|
||||
}
|
||||
if s.id%2 != 1 || s.id <= t.maxStreamID {
|
||||
t.mu.Unlock()
|
||||
// illegal gRPC stream id.
|
||||
grpclog.Println("transport: http2Server.HandleStreams received an illegal stream id: ", s.id)
|
||||
return true
|
||||
}
|
||||
t.maxStreamID = s.id
|
||||
s.sendQuotaPool = newQuotaPool(int(t.streamSendQuota))
|
||||
t.activeStreams[s.id] = s
|
||||
t.mu.Unlock()
|
||||
@ -212,6 +219,7 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(
|
||||
t.updateWindow(s, uint32(n))
|
||||
}
|
||||
handle(s)
|
||||
return
|
||||
}
|
||||
|
||||
// HandleStreams receives incoming streams using the given handler. This is
|
||||
@ -231,6 +239,10 @@ func (t *http2Server) HandleStreams(handle func(*Stream)) {
|
||||
}
|
||||
|
||||
frame, err := t.framer.readFrame()
|
||||
if err == io.EOF || err == io.ErrUnexpectedEOF {
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
if err != nil {
|
||||
grpclog.Printf("transport: http2Server.HandleStreams failed to read frame: %v", err)
|
||||
t.Close()
|
||||
@ -257,20 +269,20 @@ func (t *http2Server) HandleStreams(handle func(*Stream)) {
|
||||
t.controlBuf.put(&resetStream{se.StreamID, se.Code})
|
||||
continue
|
||||
}
|
||||
if err == io.EOF || err == io.ErrUnexpectedEOF {
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
grpclog.Printf("transport: http2Server.HandleStreams failed to read frame: %v", err)
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
switch frame := frame.(type) {
|
||||
case *http2.MetaHeadersFrame:
|
||||
id := frame.Header().StreamID
|
||||
if id%2 != 1 || id <= t.maxStreamID {
|
||||
// illegal gRPC stream id.
|
||||
grpclog.Println("transport: http2Server.HandleStreams received an illegal stream id: ", id)
|
||||
if t.operateHeaders(frame, handle) {
|
||||
t.Close()
|
||||
break
|
||||
}
|
||||
t.maxStreamID = id
|
||||
t.operateHeaders(frame, handle)
|
||||
case *http2.DataFrame:
|
||||
t.handleData(frame)
|
||||
case *http2.RSTStreamFrame:
|
||||
@ -282,7 +294,7 @@ func (t *http2Server) HandleStreams(handle func(*Stream)) {
|
||||
case *http2.WindowUpdateFrame:
|
||||
t.handleWindowUpdate(frame)
|
||||
case *http2.GoAwayFrame:
|
||||
break
|
||||
// TODO: Handle GoAway from the client appropriately.
|
||||
default:
|
||||
grpclog.Printf("transport: http2Server.HandleStreams found unhandled frame type %v.", frame)
|
||||
}
|
||||
@ -364,11 +376,7 @@ func (t *http2Server) handleData(f *http2.DataFrame) {
|
||||
// Received the end of stream from the client.
|
||||
s.mu.Lock()
|
||||
if s.state != streamDone {
|
||||
if s.state == streamWriteDone {
|
||||
s.state = streamDone
|
||||
} else {
|
||||
s.state = streamReadDone
|
||||
}
|
||||
s.state = streamReadDone
|
||||
}
|
||||
s.mu.Unlock()
|
||||
s.write(recvMsg{err: io.EOF})
|
||||
@ -440,7 +448,7 @@ func (t *http2Server) writeHeaders(s *Stream, b *bytes.Buffer, endStream bool) e
|
||||
}
|
||||
if err != nil {
|
||||
t.Close()
|
||||
return ConnectionErrorf("transport: %v", err)
|
||||
return ConnectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
@ -455,7 +463,7 @@ func (t *http2Server) WriteHeader(s *Stream, md metadata.MD) error {
|
||||
}
|
||||
s.headerOk = true
|
||||
s.mu.Unlock()
|
||||
if _, err := wait(s.ctx, t.shutdownChan, t.writableChan); err != nil {
|
||||
if _, err := wait(s.ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil {
|
||||
return err
|
||||
}
|
||||
t.hBuf.Reset()
|
||||
@ -495,7 +503,7 @@ func (t *http2Server) WriteStatus(s *Stream, statusCode codes.Code, statusDesc s
|
||||
headersSent = true
|
||||
}
|
||||
s.mu.Unlock()
|
||||
if _, err := wait(s.ctx, t.shutdownChan, t.writableChan); err != nil {
|
||||
if _, err := wait(s.ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil {
|
||||
return err
|
||||
}
|
||||
t.hBuf.Reset()
|
||||
@ -508,7 +516,7 @@ func (t *http2Server) WriteStatus(s *Stream, statusCode codes.Code, statusDesc s
|
||||
Name: "grpc-status",
|
||||
Value: strconv.Itoa(int(statusCode)),
|
||||
})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-message", Value: statusDesc})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-message", Value: encodeGrpcMessage(statusDesc)})
|
||||
// Attach the trailer metadata.
|
||||
for k, v := range s.trailer {
|
||||
// Clients don't tolerate reading restricted headers after some non restricted ones were sent.
|
||||
@ -544,7 +552,7 @@ func (t *http2Server) Write(s *Stream, data []byte, opts *Options) error {
|
||||
}
|
||||
s.mu.Unlock()
|
||||
if writeHeaderFrame {
|
||||
if _, err := wait(s.ctx, t.shutdownChan, t.writableChan); err != nil {
|
||||
if _, err := wait(s.ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil {
|
||||
return err
|
||||
}
|
||||
t.hBuf.Reset()
|
||||
@ -560,7 +568,7 @@ func (t *http2Server) Write(s *Stream, data []byte, opts *Options) error {
|
||||
}
|
||||
if err := t.framer.writeHeaders(false, p); err != nil {
|
||||
t.Close()
|
||||
return ConnectionErrorf("transport: %v", err)
|
||||
return ConnectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
t.writableChan <- 0
|
||||
}
|
||||
@ -572,13 +580,13 @@ func (t *http2Server) Write(s *Stream, data []byte, opts *Options) error {
|
||||
size := http2MaxFrameLen
|
||||
s.sendQuotaPool.add(0)
|
||||
// Wait until the stream has some quota to send the data.
|
||||
sq, err := wait(s.ctx, t.shutdownChan, s.sendQuotaPool.acquire())
|
||||
sq, err := wait(s.ctx, nil, nil, t.shutdownChan, s.sendQuotaPool.acquire())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
t.sendQuotaPool.add(0)
|
||||
// Wait until the transport has some quota to send the data.
|
||||
tq, err := wait(s.ctx, t.shutdownChan, t.sendQuotaPool.acquire())
|
||||
tq, err := wait(s.ctx, nil, nil, t.shutdownChan, t.sendQuotaPool.acquire())
|
||||
if err != nil {
|
||||
if _, ok := err.(StreamError); ok {
|
||||
t.sendQuotaPool.cancel()
|
||||
@ -604,7 +612,7 @@ func (t *http2Server) Write(s *Stream, data []byte, opts *Options) error {
|
||||
t.framer.adjustNumWriters(1)
|
||||
// Got some quota. Try to acquire writing privilege on the
|
||||
// transport.
|
||||
if _, err := wait(s.ctx, t.shutdownChan, t.writableChan); err != nil {
|
||||
if _, err := wait(s.ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil {
|
||||
if _, ok := err.(StreamError); ok {
|
||||
// Return the connection quota back.
|
||||
t.sendQuotaPool.add(ps)
|
||||
@ -634,7 +642,7 @@ func (t *http2Server) Write(s *Stream, data []byte, opts *Options) error {
|
||||
}
|
||||
if err := t.framer.writeData(forceFlush, s.id, false, p); err != nil {
|
||||
t.Close()
|
||||
return ConnectionErrorf("transport: %v", err)
|
||||
return ConnectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
if t.framer.adjustNumWriters(-1) == 0 {
|
||||
t.framer.flushWrite()
|
||||
@ -679,6 +687,17 @@ func (t *http2Server) controller() {
|
||||
}
|
||||
case *resetStream:
|
||||
t.framer.writeRSTStream(true, i.streamID, i.code)
|
||||
case *goAway:
|
||||
t.mu.Lock()
|
||||
if t.state == closing {
|
||||
t.mu.Unlock()
|
||||
// The transport is closing.
|
||||
return
|
||||
}
|
||||
sid := t.maxStreamID
|
||||
t.state = draining
|
||||
t.mu.Unlock()
|
||||
t.framer.writeGoAway(true, sid, http2.ErrCodeNo, nil)
|
||||
case *flushIO:
|
||||
t.framer.flushWrite()
|
||||
case *ping:
|
||||
@ -724,6 +743,9 @@ func (t *http2Server) Close() (err error) {
|
||||
func (t *http2Server) closeStream(s *Stream) {
|
||||
t.mu.Lock()
|
||||
delete(t.activeStreams, s.id)
|
||||
if t.state == draining && len(t.activeStreams) == 0 {
|
||||
defer t.Close()
|
||||
}
|
||||
t.mu.Unlock()
|
||||
// In case stream sending and receiving are invoked in separate
|
||||
// goroutines (e.g., bi-directional streaming), cancel needs to be
|
||||
@ -746,3 +768,7 @@ func (t *http2Server) closeStream(s *Stream) {
|
||||
func (t *http2Server) RemoteAddr() net.Addr {
|
||||
return t.conn.RemoteAddr()
|
||||
}
|
||||
|
||||
func (t *http2Server) Drain() {
|
||||
t.controlBuf.put(&goAway{})
|
||||
}
|
||||
|
83
cmd/vendor/google.golang.org/grpc/transport/http_util.go
generated
vendored
83
cmd/vendor/google.golang.org/grpc/transport/http_util.go
generated
vendored
@ -35,6 +35,7 @@ package transport
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"bytes"
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
@ -174,11 +175,11 @@ func (d *decodeState) processHeaderField(f hpack.HeaderField) {
|
||||
}
|
||||
d.statusCode = codes.Code(code)
|
||||
case "grpc-message":
|
||||
d.statusDesc = f.Value
|
||||
d.statusDesc = decodeGrpcMessage(f.Value)
|
||||
case "grpc-timeout":
|
||||
d.timeoutSet = true
|
||||
var err error
|
||||
d.timeout, err = timeoutDecode(f.Value)
|
||||
d.timeout, err = decodeTimeout(f.Value)
|
||||
if err != nil {
|
||||
d.setErr(StreamErrorf(codes.Internal, "transport: malformed time-out: %v", err))
|
||||
return
|
||||
@ -251,7 +252,7 @@ func div(d, r time.Duration) int64 {
|
||||
}
|
||||
|
||||
// TODO(zhaoq): It is the simplistic and not bandwidth efficient. Improve it.
|
||||
func timeoutEncode(t time.Duration) string {
|
||||
func encodeTimeout(t time.Duration) string {
|
||||
if d := div(t, time.Nanosecond); d <= maxTimeoutValue {
|
||||
return strconv.FormatInt(d, 10) + "n"
|
||||
}
|
||||
@ -271,7 +272,7 @@ func timeoutEncode(t time.Duration) string {
|
||||
return strconv.FormatInt(div(t, time.Hour), 10) + "H"
|
||||
}
|
||||
|
||||
func timeoutDecode(s string) (time.Duration, error) {
|
||||
func decodeTimeout(s string) (time.Duration, error) {
|
||||
size := len(s)
|
||||
if size < 2 {
|
||||
return 0, fmt.Errorf("transport: timeout string is too short: %q", s)
|
||||
@ -288,6 +289,80 @@ func timeoutDecode(s string) (time.Duration, error) {
|
||||
return d * time.Duration(t), nil
|
||||
}
|
||||
|
||||
const (
|
||||
spaceByte = ' '
|
||||
tildaByte = '~'
|
||||
percentByte = '%'
|
||||
)
|
||||
|
||||
// encodeGrpcMessage is used to encode status code in header field
|
||||
// "grpc-message".
|
||||
// It checks to see if each individual byte in msg is an
|
||||
// allowable byte, and then either percent encoding or passing it through.
|
||||
// When percent encoding, the byte is converted into hexadecimal notation
|
||||
// with a '%' prepended.
|
||||
func encodeGrpcMessage(msg string) string {
|
||||
if msg == "" {
|
||||
return ""
|
||||
}
|
||||
lenMsg := len(msg)
|
||||
for i := 0; i < lenMsg; i++ {
|
||||
c := msg[i]
|
||||
if !(c >= spaceByte && c < tildaByte && c != percentByte) {
|
||||
return encodeGrpcMessageUnchecked(msg)
|
||||
}
|
||||
}
|
||||
return msg
|
||||
}
|
||||
|
||||
func encodeGrpcMessageUnchecked(msg string) string {
|
||||
var buf bytes.Buffer
|
||||
lenMsg := len(msg)
|
||||
for i := 0; i < lenMsg; i++ {
|
||||
c := msg[i]
|
||||
if c >= spaceByte && c < tildaByte && c != percentByte {
|
||||
buf.WriteByte(c)
|
||||
} else {
|
||||
buf.WriteString(fmt.Sprintf("%%%02X", c))
|
||||
}
|
||||
}
|
||||
return buf.String()
|
||||
}
|
||||
|
||||
// decodeGrpcMessage decodes the msg encoded by encodeGrpcMessage.
|
||||
func decodeGrpcMessage(msg string) string {
|
||||
if msg == "" {
|
||||
return ""
|
||||
}
|
||||
lenMsg := len(msg)
|
||||
for i := 0; i < lenMsg; i++ {
|
||||
if msg[i] == percentByte && i+2 < lenMsg {
|
||||
return decodeGrpcMessageUnchecked(msg)
|
||||
}
|
||||
}
|
||||
return msg
|
||||
}
|
||||
|
||||
func decodeGrpcMessageUnchecked(msg string) string {
|
||||
var buf bytes.Buffer
|
||||
lenMsg := len(msg)
|
||||
for i := 0; i < lenMsg; i++ {
|
||||
c := msg[i]
|
||||
if c == percentByte && i+2 < lenMsg {
|
||||
parsed, err := strconv.ParseInt(msg[i+1:i+3], 16, 8)
|
||||
if err != nil {
|
||||
buf.WriteByte(c)
|
||||
} else {
|
||||
buf.WriteByte(byte(parsed))
|
||||
i += 2
|
||||
}
|
||||
} else {
|
||||
buf.WriteByte(c)
|
||||
}
|
||||
}
|
||||
return buf.String()
|
||||
}
|
||||
|
||||
type framer struct {
|
||||
numWriters int32
|
||||
reader io.Reader
|
||||
|
51
cmd/vendor/google.golang.org/grpc/transport/pre_go16.go
generated
vendored
Normal file
51
cmd/vendor/google.golang.org/grpc/transport/pre_go16.go
generated
vendored
Normal file
@ -0,0 +1,51 @@
|
||||
// +build !go1.6
|
||||
|
||||
/*
|
||||
* Copyright 2016, Google Inc.
|
||||
* All rights reserved.
|
||||
*
|
||||
* Redistribution and use in source and binary forms, with or without
|
||||
* modification, are permitted provided that the following conditions are
|
||||
* met:
|
||||
*
|
||||
* * Redistributions of source code must retain the above copyright
|
||||
* notice, this list of conditions and the following disclaimer.
|
||||
* * Redistributions in binary form must reproduce the above
|
||||
* copyright notice, this list of conditions and the following disclaimer
|
||||
* in the documentation and/or other materials provided with the
|
||||
* distribution.
|
||||
* * Neither the name of Google Inc. nor the names of its
|
||||
* contributors may be used to endorse or promote products derived from
|
||||
* this software without specific prior written permission.
|
||||
*
|
||||
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
*
|
||||
*/
|
||||
|
||||
package transport
|
||||
|
||||
import (
|
||||
"net"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
// dialContext connects to the address on the named network.
|
||||
func dialContext(ctx context.Context, network, address string) (net.Conn, error) {
|
||||
var dialer net.Dialer
|
||||
if deadline, ok := ctx.Deadline(); ok {
|
||||
dialer.Timeout = deadline.Sub(time.Now())
|
||||
}
|
||||
return dialer.Dial(network, address)
|
||||
}
|
93
cmd/vendor/google.golang.org/grpc/transport/transport.go
generated
vendored
93
cmd/vendor/google.golang.org/grpc/transport/transport.go
generated
vendored
@ -44,7 +44,6 @@ import (
|
||||
"io"
|
||||
"net"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"golang.org/x/net/trace"
|
||||
@ -120,10 +119,11 @@ func (b *recvBuffer) get() <-chan item {
|
||||
// recvBufferReader implements io.Reader interface to read the data from
|
||||
// recvBuffer.
|
||||
type recvBufferReader struct {
|
||||
ctx context.Context
|
||||
recv *recvBuffer
|
||||
last *bytes.Reader // Stores the remaining data in the previous calls.
|
||||
err error
|
||||
ctx context.Context
|
||||
goAway chan struct{}
|
||||
recv *recvBuffer
|
||||
last *bytes.Reader // Stores the remaining data in the previous calls.
|
||||
err error
|
||||
}
|
||||
|
||||
// Read reads the next len(p) bytes from last. If last is drained, it tries to
|
||||
@ -141,6 +141,8 @@ func (r *recvBufferReader) Read(p []byte) (n int, err error) {
|
||||
select {
|
||||
case <-r.ctx.Done():
|
||||
return 0, ContextErr(r.ctx.Err())
|
||||
case <-r.goAway:
|
||||
return 0, ErrStreamDrain
|
||||
case i := <-r.recv.get():
|
||||
r.recv.load()
|
||||
m := i.(*recvMsg)
|
||||
@ -158,7 +160,7 @@ const (
|
||||
streamActive streamState = iota
|
||||
streamWriteDone // EndStream sent
|
||||
streamReadDone // EndStream received
|
||||
streamDone // sendDone and recvDone or RSTStreamFrame is sent or received.
|
||||
streamDone // the entire stream is finished.
|
||||
)
|
||||
|
||||
// Stream represents an RPC in the transport layer.
|
||||
@ -169,6 +171,10 @@ type Stream struct {
|
||||
// ctx is the associated context of the stream.
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
// done is closed when the final status arrives.
|
||||
done chan struct{}
|
||||
// goAway is closed when the server sent GoAways signal before this stream was initiated.
|
||||
goAway chan struct{}
|
||||
// method records the associated RPC method of the stream.
|
||||
method string
|
||||
recvCompress string
|
||||
@ -214,6 +220,18 @@ func (s *Stream) SetSendCompress(str string) {
|
||||
s.sendCompress = str
|
||||
}
|
||||
|
||||
// Done returns a chanel which is closed when it receives the final status
|
||||
// from the server.
|
||||
func (s *Stream) Done() <-chan struct{} {
|
||||
return s.done
|
||||
}
|
||||
|
||||
// GoAway returns a channel which is closed when the server sent GoAways signal
|
||||
// before this stream was initiated.
|
||||
func (s *Stream) GoAway() <-chan struct{} {
|
||||
return s.goAway
|
||||
}
|
||||
|
||||
// Header acquires the key-value pairs of header metadata once it
|
||||
// is available. It blocks until i) the metadata is ready or ii) there is no
|
||||
// header metadata or iii) the stream is cancelled/expired.
|
||||
@ -221,6 +239,8 @@ func (s *Stream) Header() (metadata.MD, error) {
|
||||
select {
|
||||
case <-s.ctx.Done():
|
||||
return nil, ContextErr(s.ctx.Err())
|
||||
case <-s.goAway:
|
||||
return nil, ErrStreamDrain
|
||||
case <-s.headerChan:
|
||||
return s.header.Copy(), nil
|
||||
}
|
||||
@ -335,19 +355,17 @@ type ConnectOptions struct {
|
||||
// UserAgent is the application user agent.
|
||||
UserAgent string
|
||||
// Dialer specifies how to dial a network address.
|
||||
Dialer func(string, time.Duration) (net.Conn, error)
|
||||
Dialer func(context.Context, string) (net.Conn, error)
|
||||
// PerRPCCredentials stores the PerRPCCredentials required to issue RPCs.
|
||||
PerRPCCredentials []credentials.PerRPCCredentials
|
||||
// TransportCredentials stores the Authenticator required to setup a client connection.
|
||||
TransportCredentials credentials.TransportCredentials
|
||||
// Timeout specifies the timeout for dialing a ClientTransport.
|
||||
Timeout time.Duration
|
||||
}
|
||||
|
||||
// NewClientTransport establishes the transport with the required ConnectOptions
|
||||
// and returns it to the caller.
|
||||
func NewClientTransport(target string, opts *ConnectOptions) (ClientTransport, error) {
|
||||
return newHTTP2Client(target, opts)
|
||||
func NewClientTransport(ctx context.Context, target string, opts ConnectOptions) (ClientTransport, error) {
|
||||
return newHTTP2Client(ctx, target, opts)
|
||||
}
|
||||
|
||||
// Options provides additional hints and information for message
|
||||
@ -417,6 +435,11 @@ type ClientTransport interface {
|
||||
// and create a new one) in error case. It should not return nil
|
||||
// once the transport is initiated.
|
||||
Error() <-chan struct{}
|
||||
|
||||
// GoAway returns a channel that is closed when ClientTranspor
|
||||
// receives the draining signal from the server (e.g., GOAWAY frame in
|
||||
// HTTP/2).
|
||||
GoAway() <-chan struct{}
|
||||
}
|
||||
|
||||
// ServerTransport is the common interface for all gRPC server-side transport
|
||||
@ -448,6 +471,9 @@ type ServerTransport interface {
|
||||
|
||||
// RemoteAddr returns the remote network address.
|
||||
RemoteAddr() net.Addr
|
||||
|
||||
// Drain notifies the client this ServerTransport stops accepting new RPCs.
|
||||
Drain()
|
||||
}
|
||||
|
||||
// StreamErrorf creates an StreamError with the specified error code and description.
|
||||
@ -459,9 +485,11 @@ func StreamErrorf(c codes.Code, format string, a ...interface{}) StreamError {
|
||||
}
|
||||
|
||||
// ConnectionErrorf creates an ConnectionError with the specified error description.
|
||||
func ConnectionErrorf(format string, a ...interface{}) ConnectionError {
|
||||
func ConnectionErrorf(temp bool, e error, format string, a ...interface{}) ConnectionError {
|
||||
return ConnectionError{
|
||||
Desc: fmt.Sprintf(format, a...),
|
||||
temp: temp,
|
||||
err: e,
|
||||
}
|
||||
}
|
||||
|
||||
@ -469,14 +497,36 @@ func ConnectionErrorf(format string, a ...interface{}) ConnectionError {
|
||||
// entire connection and the retry of all the active streams.
|
||||
type ConnectionError struct {
|
||||
Desc string
|
||||
temp bool
|
||||
err error
|
||||
}
|
||||
|
||||
func (e ConnectionError) Error() string {
|
||||
return fmt.Sprintf("connection error: desc = %q", e.Desc)
|
||||
}
|
||||
|
||||
// ErrConnClosing indicates that the transport is closing.
|
||||
var ErrConnClosing = ConnectionError{Desc: "transport is closing"}
|
||||
// Temporary indicates if this connection error is temporary or fatal.
|
||||
func (e ConnectionError) Temporary() bool {
|
||||
return e.temp
|
||||
}
|
||||
|
||||
// Origin returns the original error of this connection error.
|
||||
func (e ConnectionError) Origin() error {
|
||||
// Never return nil error here.
|
||||
// If the original error is nil, return itself.
|
||||
if e.err == nil {
|
||||
return e
|
||||
}
|
||||
return e.err
|
||||
}
|
||||
|
||||
var (
|
||||
// ErrConnClosing indicates that the transport is closing.
|
||||
ErrConnClosing = ConnectionError{Desc: "transport is closing", temp: true}
|
||||
// ErrStreamDrain indicates that the stream is rejected by the server because
|
||||
// the server stops accepting new RPCs.
|
||||
ErrStreamDrain = StreamErrorf(codes.Unavailable, "the server stops accepting new RPCs")
|
||||
)
|
||||
|
||||
// StreamError is an error that only affects one stream within a connection.
|
||||
type StreamError struct {
|
||||
@ -501,12 +551,25 @@ func ContextErr(err error) StreamError {
|
||||
|
||||
// wait blocks until it can receive from ctx.Done, closing, or proceed.
|
||||
// If it receives from ctx.Done, it returns 0, the StreamError for ctx.Err.
|
||||
// If it receives from done, it returns 0, io.EOF if ctx is not done; otherwise
|
||||
// it return the StreamError for ctx.Err.
|
||||
// If it receives from goAway, it returns 0, ErrStreamDrain.
|
||||
// If it receives from closing, it returns 0, ErrConnClosing.
|
||||
// If it receives from proceed, it returns the received integer, nil.
|
||||
func wait(ctx context.Context, closing <-chan struct{}, proceed <-chan int) (int, error) {
|
||||
func wait(ctx context.Context, done, goAway, closing <-chan struct{}, proceed <-chan int) (int, error) {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return 0, ContextErr(ctx.Err())
|
||||
case <-done:
|
||||
// User cancellation has precedence.
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return 0, ContextErr(ctx.Err())
|
||||
default:
|
||||
}
|
||||
return 0, io.EOF
|
||||
case <-goAway:
|
||||
return 0, ErrStreamDrain
|
||||
case <-closing:
|
||||
return 0, ErrConnClosing
|
||||
case i := <-proceed:
|
||||
|
@ -53,8 +53,8 @@ func SRVGetCluster(name, dns string, defaultToken string, apurls types.URLs) (st
|
||||
return err
|
||||
}
|
||||
for _, srv := range addrs {
|
||||
target := strings.TrimSuffix(srv.Target, ".")
|
||||
host := net.JoinHostPort(target, fmt.Sprintf("%d", srv.Port))
|
||||
port := fmt.Sprintf("%d", srv.Port)
|
||||
host := net.JoinHostPort(srv.Target, port)
|
||||
tcpAddr, err := resolveTCPAddr("tcp", host)
|
||||
if err != nil {
|
||||
plog.Warningf("couldn't resolve host %s during SRV discovery", host)
|
||||
@ -70,8 +70,11 @@ func SRVGetCluster(name, dns string, defaultToken string, apurls types.URLs) (st
|
||||
n = fmt.Sprintf("%d", tempName)
|
||||
tempName += 1
|
||||
}
|
||||
stringParts = append(stringParts, fmt.Sprintf("%s=%s%s", n, prefix, host))
|
||||
plog.Noticef("got bootstrap from DNS for %s at %s%s", service, prefix, host)
|
||||
// SRV records have a trailing dot but URL shouldn't.
|
||||
shortHost := strings.TrimSuffix(srv.Target, ".")
|
||||
urlHost := net.JoinHostPort(shortHost, port)
|
||||
stringParts = append(stringParts, fmt.Sprintf("%s=%s%s", n, prefix, urlHost))
|
||||
plog.Noticef("got bootstrap from DNS for %s at %s%s", service, prefix, urlHost)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -17,6 +17,7 @@ package discovery
|
||||
import (
|
||||
"errors"
|
||||
"net"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
@ -29,11 +30,22 @@ func TestSRVGetCluster(t *testing.T) {
|
||||
}()
|
||||
|
||||
name := "dnsClusterTest"
|
||||
dns := map[string]string{
|
||||
"1.example.com.:2480": "10.0.0.1:2480",
|
||||
"2.example.com.:2480": "10.0.0.2:2480",
|
||||
"3.example.com.:2480": "10.0.0.3:2480",
|
||||
"4.example.com.:2380": "10.0.0.3:2380",
|
||||
}
|
||||
srvAll := []*net.SRV{
|
||||
{Target: "1.example.com.", Port: 2480},
|
||||
{Target: "2.example.com.", Port: 2480},
|
||||
{Target: "3.example.com.", Port: 2480},
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
withSSL []*net.SRV
|
||||
withoutSSL []*net.SRV
|
||||
urls []string
|
||||
dns map[string]string
|
||||
|
||||
expected string
|
||||
}{
|
||||
@ -41,61 +53,50 @@ func TestSRVGetCluster(t *testing.T) {
|
||||
[]*net.SRV{},
|
||||
[]*net.SRV{},
|
||||
nil,
|
||||
nil,
|
||||
|
||||
"",
|
||||
},
|
||||
{
|
||||
[]*net.SRV{
|
||||
{Target: "10.0.0.1", Port: 2480},
|
||||
{Target: "10.0.0.2", Port: 2480},
|
||||
{Target: "10.0.0.3", Port: 2480},
|
||||
},
|
||||
srvAll,
|
||||
[]*net.SRV{},
|
||||
nil,
|
||||
|
||||
"0=https://1.example.com:2480,1=https://2.example.com:2480,2=https://3.example.com:2480",
|
||||
},
|
||||
{
|
||||
srvAll,
|
||||
[]*net.SRV{{Target: "4.example.com.", Port: 2380}},
|
||||
nil,
|
||||
|
||||
"0=https://10.0.0.1:2480,1=https://10.0.0.2:2480,2=https://10.0.0.3:2480",
|
||||
"0=https://1.example.com:2480,1=https://2.example.com:2480,2=https://3.example.com:2480,3=http://4.example.com:2380",
|
||||
},
|
||||
{
|
||||
[]*net.SRV{
|
||||
{Target: "10.0.0.1", Port: 2480},
|
||||
{Target: "10.0.0.2", Port: 2480},
|
||||
{Target: "10.0.0.3", Port: 2480},
|
||||
},
|
||||
[]*net.SRV{
|
||||
{Target: "10.0.0.1", Port: 2380},
|
||||
},
|
||||
nil,
|
||||
nil,
|
||||
"0=https://10.0.0.1:2480,1=https://10.0.0.2:2480,2=https://10.0.0.3:2480,3=http://10.0.0.1:2380",
|
||||
},
|
||||
{
|
||||
[]*net.SRV{
|
||||
{Target: "10.0.0.1", Port: 2480},
|
||||
{Target: "10.0.0.2", Port: 2480},
|
||||
{Target: "10.0.0.3", Port: 2480},
|
||||
},
|
||||
[]*net.SRV{
|
||||
{Target: "10.0.0.1", Port: 2380},
|
||||
},
|
||||
srvAll,
|
||||
[]*net.SRV{{Target: "4.example.com.", Port: 2380}},
|
||||
[]string{"https://10.0.0.1:2480"},
|
||||
nil,
|
||||
"dnsClusterTest=https://10.0.0.1:2480,0=https://10.0.0.2:2480,1=https://10.0.0.3:2480,2=http://10.0.0.1:2380",
|
||||
|
||||
"dnsClusterTest=https://1.example.com:2480,0=https://2.example.com:2480,1=https://3.example.com:2480,2=http://4.example.com:2380",
|
||||
},
|
||||
// matching local member with resolved addr and return unresolved hostnames
|
||||
{
|
||||
[]*net.SRV{
|
||||
{Target: "1.example.com.", Port: 2480},
|
||||
{Target: "2.example.com.", Port: 2480},
|
||||
{Target: "3.example.com.", Port: 2480},
|
||||
},
|
||||
srvAll,
|
||||
nil,
|
||||
[]string{"https://10.0.0.1:2480"},
|
||||
map[string]string{"1.example.com:2480": "10.0.0.1:2480", "2.example.com:2480": "10.0.0.2:2480", "3.example.com:2480": "10.0.0.3:2480"},
|
||||
|
||||
"dnsClusterTest=https://1.example.com:2480,0=https://2.example.com:2480,1=https://3.example.com:2480",
|
||||
},
|
||||
// invalid
|
||||
}
|
||||
|
||||
resolveTCPAddr = func(network, addr string) (*net.TCPAddr, error) {
|
||||
if strings.Contains(addr, "10.0.0.") {
|
||||
// accept IP addresses when resolving apurls
|
||||
return net.ResolveTCPAddr(network, addr)
|
||||
}
|
||||
if dns[addr] == "" {
|
||||
return nil, errors.New("missing dns record")
|
||||
}
|
||||
return net.ResolveTCPAddr(network, dns[addr])
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
@ -108,12 +109,6 @@ func TestSRVGetCluster(t *testing.T) {
|
||||
}
|
||||
return "", nil, errors.New("Unknown service in mock")
|
||||
}
|
||||
resolveTCPAddr = func(network, addr string) (*net.TCPAddr, error) {
|
||||
if tt.dns == nil || tt.dns[addr] == "" {
|
||||
return net.ResolveTCPAddr(network, addr)
|
||||
}
|
||||
return net.ResolveTCPAddr(network, tt.dns[addr])
|
||||
}
|
||||
urls := testutil.MustNewURLs(t, tt.urls)
|
||||
str, token, err := SRVGetCluster(name, "example.com", "token", urls)
|
||||
if err != nil {
|
||||
|
@ -39,15 +39,23 @@ func txnTestSuccess(cx ctlCtx) {
|
||||
if err := ctlV3Put(cx, "key2", "value2", ""); err != nil {
|
||||
cx.t.Fatalf("txnTestSuccess ctlV3Put error (%v)", err)
|
||||
}
|
||||
|
||||
rqs := txnRequests{
|
||||
compare: []string{`version("key1") = "1"`, `version("key2") = "1"`},
|
||||
ifSucess: []string{"get key1", "get key2"},
|
||||
ifFail: []string{`put key1 "fail"`, `put key2 "fail"`},
|
||||
results: []string{"SUCCESS", "key1", "value1", "key2", "value2"},
|
||||
rqs := []txnRequests{
|
||||
{
|
||||
compare: []string{`version("key1") = "1"`, `version("key2") = "1"`},
|
||||
ifSucess: []string{"get key1", "get key2", `put "key \"with\" space" "value \x23"`},
|
||||
ifFail: []string{`put key1 "fail"`, `put key2 "fail"`},
|
||||
results: []string{"SUCCESS", "key1", "value1", "key2", "value2"},
|
||||
},
|
||||
{
|
||||
compare: []string{`version("key \"with\" space") = "1"`},
|
||||
ifSucess: []string{`get "key \"with\" space"`},
|
||||
results: []string{"SUCCESS", `key "with" space`, "value \x23"},
|
||||
},
|
||||
}
|
||||
if err := ctlV3Txn(cx, rqs); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
for _, rq := range rqs {
|
||||
if err := ctlV3Txn(cx, rq); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -231,6 +231,8 @@ Watch watches events stream on keys or prefixes, [key or prefix, range_end) if `
|
||||
|
||||
- prefix -- watch on a prefix if prefix is set.
|
||||
|
||||
- prev-kv -- get the previous key-value pair before the event happens.
|
||||
|
||||
- rev -- the revision to start watching. Specifying a revision is useful for observing past events.
|
||||
|
||||
#### Input Format
|
||||
@ -245,7 +247,7 @@ watch [options] <key or prefix>\n
|
||||
|
||||
##### Simple reply
|
||||
|
||||
- \<event\>\n\<key\>\n\<value\>\n\<event\>\n\<next_key\>\n\<next_value\>\n...
|
||||
- \<event\>[\n\<old_key\>\n\<old_value\>]\n\<key\>\n\<value\>\n\<event\>\n\<next_key\>\n\<next_value\>\n...
|
||||
|
||||
- Additional error string if WATCH failed. Exit code is non-zero.
|
||||
|
||||
|
@ -85,13 +85,7 @@ func getPeersFlagValue(c *cli.Context) []string {
|
||||
}
|
||||
|
||||
func getDomainDiscoveryFlagValue(c *cli.Context) ([]string, error) {
|
||||
domainstr := c.GlobalString("discovery-srv")
|
||||
|
||||
// Use an environment variable if nothing was supplied on the
|
||||
// command line
|
||||
if domainstr == "" {
|
||||
domainstr = os.Getenv("ETCDCTL_DISCOVERY_SRV")
|
||||
}
|
||||
domainstr, insecure := getDiscoveryDomain(c)
|
||||
|
||||
// If we still don't have domain discovery, return nothing
|
||||
if domainstr == "" {
|
||||
@ -103,8 +97,30 @@ func getDomainDiscoveryFlagValue(c *cli.Context) ([]string, error) {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if insecure {
|
||||
return eps, err
|
||||
}
|
||||
// strip insecure connections
|
||||
ret := []string{}
|
||||
for _, ep := range eps {
|
||||
if strings.HasPrefix("http://", ep) {
|
||||
fmt.Fprintf(os.Stderr, "ignoring discovered insecure endpoint %q\n", ep)
|
||||
continue
|
||||
}
|
||||
ret = append(ret, ep)
|
||||
}
|
||||
return ret, err
|
||||
}
|
||||
|
||||
return eps, err
|
||||
func getDiscoveryDomain(c *cli.Context) (domainstr string, insecure bool) {
|
||||
domainstr = c.GlobalString("discovery-srv")
|
||||
// Use an environment variable if nothing was supplied on the
|
||||
// command line
|
||||
if domainstr == "" {
|
||||
domainstr = os.Getenv("ETCDCTL_DISCOVERY_SRV")
|
||||
}
|
||||
insecure = c.GlobalBool("insecure-discovery") || (os.Getenv("ETCDCTL_INSECURE_DISCOVERY") != "")
|
||||
return domainstr, insecure
|
||||
}
|
||||
|
||||
func getEndpoints(c *cli.Context) ([]string, error) {
|
||||
@ -151,10 +167,15 @@ func getTransport(c *cli.Context) (*http.Transport, error) {
|
||||
keyfile = os.Getenv("ETCDCTL_KEY_FILE")
|
||||
}
|
||||
|
||||
discoveryDomain, insecure := getDiscoveryDomain(c)
|
||||
if insecure {
|
||||
discoveryDomain = ""
|
||||
}
|
||||
tls := transport.TLSInfo{
|
||||
CAFile: cafile,
|
||||
CertFile: certfile,
|
||||
KeyFile: keyfile,
|
||||
CAFile: cafile,
|
||||
CertFile: certfile,
|
||||
KeyFile: keyfile,
|
||||
ServerName: discoveryDomain,
|
||||
}
|
||||
|
||||
dialTimeout := defaultDialTimeout
|
||||
|
@ -39,6 +39,7 @@ func Start() {
|
||||
cli.BoolFlag{Name: "no-sync", Usage: "don't synchronize cluster information before sending request"},
|
||||
cli.StringFlag{Name: "output, o", Value: "simple", Usage: "output response in the given format (`simple`, `extended` or `json`)"},
|
||||
cli.StringFlag{Name: "discovery-srv, D", Usage: "domain name to query for SRV records describing cluster endpoints"},
|
||||
cli.BoolFlag{Name: "insecure-discovery", Usage: "accept insecure SRV records describing cluster endpoints"},
|
||||
cli.StringFlag{Name: "peers, C", Value: "", Usage: "DEPRECATED - \"--endpoints\" should be used instead"},
|
||||
cli.StringFlag{Name: "endpoint", Value: "", Usage: "DEPRECATED - \"--endpoints\" should be used instead"},
|
||||
cli.StringFlag{Name: "endpoints", Value: "", Usage: "a comma-delimited list of machine addresses in the cluster (default: \"http://127.0.0.1:2379,http://127.0.0.1:4001\")"},
|
||||
|
@ -23,6 +23,7 @@ import (
|
||||
|
||||
var (
|
||||
delPrefix bool
|
||||
delPrevKV bool
|
||||
)
|
||||
|
||||
// NewDelCommand returns the cobra command for "del".
|
||||
@ -34,6 +35,7 @@ func NewDelCommand() *cobra.Command {
|
||||
}
|
||||
|
||||
cmd.Flags().BoolVar(&delPrefix, "prefix", false, "delete keys with matching prefix")
|
||||
cmd.Flags().BoolVar(&delPrevKV, "prev-kv", false, "return deleted key-value pairs")
|
||||
return cmd
|
||||
}
|
||||
|
||||
@ -65,6 +67,9 @@ func getDelOp(cmd *cobra.Command, args []string) (string, []clientv3.OpOption) {
|
||||
if delPrefix {
|
||||
opts = append(opts, clientv3.WithPrefix())
|
||||
}
|
||||
if delPrevKV {
|
||||
opts = append(opts, clientv3.WithPrevKV())
|
||||
}
|
||||
|
||||
return key, opts
|
||||
}
|
||||
|
@ -243,7 +243,7 @@ func authCfgFromCmd(cmd *cobra.Command) *authCfg {
|
||||
var cfg authCfg
|
||||
|
||||
splitted := strings.SplitN(userFlag, ":", 2)
|
||||
if len(splitted) == 0 {
|
||||
if len(splitted) < 2 {
|
||||
cfg.username = userFlag
|
||||
cfg.password, err = speakeasy.Ask("Password: ")
|
||||
if err != nil {
|
||||
|
@ -108,6 +108,9 @@ type simplePrinter struct {
|
||||
|
||||
func (s *simplePrinter) Del(resp v3.DeleteResponse) {
|
||||
fmt.Println(resp.Deleted)
|
||||
for _, kv := range resp.PrevKvs {
|
||||
printKV(s.isHex, kv)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *simplePrinter) Get(resp v3.GetResponse) {
|
||||
@ -116,7 +119,12 @@ func (s *simplePrinter) Get(resp v3.GetResponse) {
|
||||
}
|
||||
}
|
||||
|
||||
func (s *simplePrinter) Put(r v3.PutResponse) { fmt.Println("OK") }
|
||||
func (s *simplePrinter) Put(r v3.PutResponse) {
|
||||
fmt.Println("OK")
|
||||
if r.PrevKv != nil {
|
||||
printKV(s.isHex, r.PrevKv)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *simplePrinter) Txn(resp v3.TxnResponse) {
|
||||
if resp.Succeeded {
|
||||
@ -143,6 +151,9 @@ func (s *simplePrinter) Txn(resp v3.TxnResponse) {
|
||||
func (s *simplePrinter) Watch(resp v3.WatchResponse) {
|
||||
for _, e := range resp.Events {
|
||||
fmt.Println(e.Type)
|
||||
if e.PrevKv != nil {
|
||||
printKV(s.isHex, e.PrevKv)
|
||||
}
|
||||
printKV(s.isHex, e.Kv)
|
||||
}
|
||||
}
|
||||
|
@ -24,7 +24,8 @@ import (
|
||||
)
|
||||
|
||||
var (
|
||||
leaseStr string
|
||||
leaseStr string
|
||||
putPrevKV bool
|
||||
)
|
||||
|
||||
// NewPutCommand returns the cobra command for "put".
|
||||
@ -49,6 +50,7 @@ will store the content of the file to <key>.
|
||||
Run: putCommandFunc,
|
||||
}
|
||||
cmd.Flags().StringVar(&leaseStr, "lease", "0", "lease ID (in hexadecimal) to attach to the key")
|
||||
cmd.Flags().BoolVar(&putPrevKV, "prev-kv", false, "return changed key-value pairs")
|
||||
return cmd
|
||||
}
|
||||
|
||||
@ -85,6 +87,9 @@ func getPutOp(cmd *cobra.Command, args []string) (string, string, []clientv3.OpO
|
||||
if id != 0 {
|
||||
opts = append(opts, clientv3.WithLease(clientv3.LeaseID(id)))
|
||||
}
|
||||
if putPrevKV {
|
||||
opts = append(opts, clientv3.WithPrevKV())
|
||||
}
|
||||
|
||||
return key, value, opts
|
||||
}
|
||||
|
@ -36,7 +36,10 @@ import (
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
"github.com/coreos/etcd/snap"
|
||||
"github.com/coreos/etcd/store"
|
||||
"github.com/coreos/etcd/wal"
|
||||
"github.com/coreos/etcd/wal/walpb"
|
||||
"github.com/spf13/cobra"
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
@ -112,7 +115,7 @@ func snapshotSaveCommandFunc(cmd *cobra.Command, args []string) {
|
||||
|
||||
partpath := path + ".part"
|
||||
f, err := os.Create(partpath)
|
||||
defer f.Close()
|
||||
|
||||
if err != nil {
|
||||
exiterr := fmt.Errorf("could not open %s (%v)", partpath, err)
|
||||
ExitWithError(ExitBadArgs, exiterr)
|
||||
@ -131,6 +134,8 @@ func snapshotSaveCommandFunc(cmd *cobra.Command, args []string) {
|
||||
|
||||
fileutil.Fsync(f)
|
||||
|
||||
f.Close()
|
||||
|
||||
if rerr := os.Rename(partpath, path); rerr != nil {
|
||||
exiterr := fmt.Errorf("could not rename %s to %s (%v)", partpath, path, rerr)
|
||||
ExitWithError(ExitIO, exiterr)
|
||||
@ -186,8 +191,8 @@ func snapshotRestoreCommandFunc(cmd *cobra.Command, args []string) {
|
||||
ExitWithError(ExitInvalidInput, fmt.Errorf("data-dir %q exists", basedir))
|
||||
}
|
||||
|
||||
makeDB(snapdir, args[0])
|
||||
makeWAL(waldir, cl)
|
||||
makeDB(snapdir, args[0], len(cl.Members()))
|
||||
makeWALAndSnap(waldir, snapdir, cl)
|
||||
}
|
||||
|
||||
func initialClusterFromName(name string) string {
|
||||
@ -199,11 +204,18 @@ func initialClusterFromName(name string) string {
|
||||
}
|
||||
|
||||
// makeWAL creates a WAL for the initial cluster
|
||||
func makeWAL(waldir string, cl *membership.RaftCluster) {
|
||||
func makeWALAndSnap(waldir, snapdir string, cl *membership.RaftCluster) {
|
||||
if err := fileutil.CreateDirAll(waldir); err != nil {
|
||||
ExitWithError(ExitIO, err)
|
||||
}
|
||||
|
||||
// add members again to persist them to the store we create.
|
||||
st := store.New(etcdserver.StoreClusterPrefix, etcdserver.StoreKeysPrefix)
|
||||
cl.SetStore(st)
|
||||
for _, m := range cl.Members() {
|
||||
cl.AddMember(m)
|
||||
}
|
||||
|
||||
m := cl.MemberByName(restoreName)
|
||||
md := &etcdserverpb.Metadata{NodeID: uint64(m.ID), ClusterID: uint64(cl.ID())}
|
||||
metadata, merr := md.Marshal()
|
||||
@ -227,7 +239,9 @@ func makeWAL(waldir string, cl *membership.RaftCluster) {
|
||||
}
|
||||
|
||||
ents := make([]raftpb.Entry, len(peers))
|
||||
nodeIDs := make([]uint64, len(peers))
|
||||
for i, p := range peers {
|
||||
nodeIDs[i] = p.ID
|
||||
cc := raftpb.ConfChange{
|
||||
Type: raftpb.ConfChangeAddNode,
|
||||
NodeID: p.ID,
|
||||
@ -245,20 +259,48 @@ func makeWAL(waldir string, cl *membership.RaftCluster) {
|
||||
ents[i] = e
|
||||
}
|
||||
|
||||
w.Save(raftpb.HardState{
|
||||
Term: 1,
|
||||
commit, term := uint64(len(ents)), uint64(1)
|
||||
|
||||
if err := w.Save(raftpb.HardState{
|
||||
Term: term,
|
||||
Vote: peers[0].ID,
|
||||
Commit: uint64(len(ents))}, ents)
|
||||
Commit: commit}, ents); err != nil {
|
||||
ExitWithError(ExitIO, err)
|
||||
}
|
||||
|
||||
b, berr := st.Save()
|
||||
if berr != nil {
|
||||
ExitWithError(ExitError, berr)
|
||||
}
|
||||
|
||||
raftSnap := raftpb.Snapshot{
|
||||
Data: b,
|
||||
Metadata: raftpb.SnapshotMetadata{
|
||||
Index: commit,
|
||||
Term: term,
|
||||
ConfState: raftpb.ConfState{
|
||||
Nodes: nodeIDs,
|
||||
},
|
||||
},
|
||||
}
|
||||
snapshotter := snap.New(snapdir)
|
||||
if err := snapshotter.SaveSnap(raftSnap); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if err := w.SaveSnapshot(walpb.Snapshot{Index: commit, Term: term}); err != nil {
|
||||
ExitWithError(ExitIO, err)
|
||||
}
|
||||
}
|
||||
|
||||
// initIndex implements ConsistentIndexGetter so the snapshot won't block
|
||||
// the new raft instance by waiting for a future raft index.
|
||||
type initIndex struct{}
|
||||
type initIndex int
|
||||
|
||||
func (*initIndex) ConsistentIndex() uint64 { return 1 }
|
||||
func (i *initIndex) ConsistentIndex() uint64 { return uint64(*i) }
|
||||
|
||||
// makeDB copies the database snapshot to the snapshot directory
|
||||
func makeDB(snapdir, dbfile string) {
|
||||
func makeDB(snapdir, dbfile string, commit int) {
|
||||
f, ferr := os.OpenFile(dbfile, os.O_RDONLY, 0600)
|
||||
if ferr != nil {
|
||||
ExitWithError(ExitInvalidInput, ferr)
|
||||
@ -329,7 +371,7 @@ func makeDB(snapdir, dbfile string) {
|
||||
// update consistentIndex so applies go through on etcdserver despite
|
||||
// having a new raft instance
|
||||
be := backend.NewDefaultBackend(dbpath)
|
||||
s := mvcc.NewStore(be, nil, &initIndex{})
|
||||
s := mvcc.NewStore(be, nil, (*initIndex)(&commit))
|
||||
id := s.TxnBegin()
|
||||
btx := be.BatchTx()
|
||||
del := func(k, v []byte) error {
|
||||
@ -339,6 +381,7 @@ func makeDB(snapdir, dbfile string) {
|
||||
|
||||
// delete stored members from old cluster since using new members
|
||||
btx.UnsafeForEach([]byte("members"), del)
|
||||
// todo: add back new members when we start to deprecate old snap file.
|
||||
btx.UnsafeForEach([]byte("members_removed"), del)
|
||||
// trigger write-out of new consistent index
|
||||
s.TxnEnd(id)
|
||||
|
@ -77,12 +77,13 @@ func readCompares(r *bufio.Reader) (cmps []clientv3.Cmp) {
|
||||
if err != nil {
|
||||
ExitWithError(ExitInvalidInput, err)
|
||||
}
|
||||
if len(line) == 1 {
|
||||
|
||||
// remove space from the line
|
||||
line = strings.TrimSpace(line)
|
||||
if len(line) == 0 {
|
||||
break
|
||||
}
|
||||
|
||||
// remove trialling \n
|
||||
line = line[:len(line)-1]
|
||||
cmp, err := parseCompare(line)
|
||||
if err != nil {
|
||||
ExitWithError(ExitInvalidInput, err)
|
||||
@ -99,12 +100,13 @@ func readOps(r *bufio.Reader) (ops []clientv3.Op) {
|
||||
if err != nil {
|
||||
ExitWithError(ExitInvalidInput, err)
|
||||
}
|
||||
if len(line) == 1 {
|
||||
|
||||
// remove space from the line
|
||||
line = strings.TrimSpace(line)
|
||||
if len(line) == 0 {
|
||||
break
|
||||
}
|
||||
|
||||
// remove trialling \n
|
||||
line = line[:len(line)-1]
|
||||
op, err := parseRequestUnion(line)
|
||||
if err != nil {
|
||||
ExitWithError(ExitInvalidInput, err)
|
||||
|
@ -46,8 +46,23 @@ func addHexPrefix(s string) string {
|
||||
}
|
||||
|
||||
func argify(s string) []string {
|
||||
r := regexp.MustCompile("'.+'|\".+\"|\\S+")
|
||||
return r.FindAllString(s, -1)
|
||||
r := regexp.MustCompile(`"(?:[^"\\]|\\.)*"|'[^']*'|[^'"\s]\S*[^'"\s]?`)
|
||||
args := r.FindAllString(s, -1)
|
||||
for i := range args {
|
||||
if len(args[i]) == 0 {
|
||||
continue
|
||||
}
|
||||
if args[i][0] == '\'' {
|
||||
// 'single-quoted string'
|
||||
args[i] = args[i][1 : len(args)-1]
|
||||
} else if args[i][0] == '"' {
|
||||
// "double quoted string"
|
||||
if _, err := fmt.Sscanf(args[i], "%q", &args[i]); err != nil {
|
||||
ExitWithError(ExitInvalidInput, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
return args
|
||||
}
|
||||
|
||||
func commandCtx(cmd *cobra.Command) (context.Context, context.CancelFunc) {
|
||||
|
@ -29,6 +29,7 @@ var (
|
||||
watchRev int64
|
||||
watchPrefix bool
|
||||
watchInteractive bool
|
||||
watchPrevKey bool
|
||||
)
|
||||
|
||||
// NewWatchCommand returns the cobra command for "watch".
|
||||
@ -42,6 +43,7 @@ func NewWatchCommand() *cobra.Command {
|
||||
cmd.Flags().BoolVarP(&watchInteractive, "interactive", "i", false, "Interactive mode")
|
||||
cmd.Flags().BoolVar(&watchPrefix, "prefix", false, "Watch on a prefix if prefix is set")
|
||||
cmd.Flags().Int64Var(&watchRev, "rev", 0, "Revision to start watching")
|
||||
cmd.Flags().BoolVar(&watchPrevKey, "prev-kv", false, "get the previous key-value pair before the event happens")
|
||||
|
||||
return cmd
|
||||
}
|
||||
@ -52,30 +54,18 @@ func watchCommandFunc(cmd *cobra.Command, args []string) {
|
||||
watchInteractiveFunc(cmd, args)
|
||||
return
|
||||
}
|
||||
if len(args) < 1 || len(args) > 2 {
|
||||
ExitWithError(ExitBadArgs, fmt.Errorf("watch in non-interactive mode requires one or two arguments as key or prefix, with range end"))
|
||||
}
|
||||
|
||||
opts := []clientv3.OpOption{clientv3.WithRev(watchRev)}
|
||||
key := args[0]
|
||||
if len(args) == 2 {
|
||||
if watchPrefix {
|
||||
ExitWithError(ExitBadArgs, fmt.Errorf("`range_end` and `--prefix` cannot be set at the same time, choose one"))
|
||||
}
|
||||
opts = append(opts, clientv3.WithRange(args[1]))
|
||||
}
|
||||
|
||||
if watchPrefix {
|
||||
opts = append(opts, clientv3.WithPrefix())
|
||||
}
|
||||
c := mustClientFromCmd(cmd)
|
||||
wc := c.Watch(context.TODO(), key, opts...)
|
||||
printWatchCh(wc)
|
||||
err := c.Close()
|
||||
if err == nil {
|
||||
ExitWithError(ExitInterrupted, fmt.Errorf("watch is canceled by the server"))
|
||||
wc, err := getWatchChan(c, args)
|
||||
if err != nil {
|
||||
ExitWithError(ExitBadArgs, err)
|
||||
}
|
||||
ExitWithError(ExitBadConnection, err)
|
||||
|
||||
printWatchCh(wc)
|
||||
if err = c.Close(); err != nil {
|
||||
ExitWithError(ExitBadConnection, err)
|
||||
}
|
||||
ExitWithError(ExitInterrupted, fmt.Errorf("watch is canceled by the server"))
|
||||
}
|
||||
|
||||
func watchInteractiveFunc(cmd *cobra.Command, args []string) {
|
||||
@ -107,32 +97,36 @@ func watchInteractiveFunc(cmd *cobra.Command, args []string) {
|
||||
fmt.Fprintf(os.Stderr, "Invalid command %s (%v)\n", l, err)
|
||||
continue
|
||||
}
|
||||
moreargs := flagset.Args()
|
||||
if len(moreargs) < 1 || len(moreargs) > 2 {
|
||||
fmt.Fprintf(os.Stderr, "Invalid command %s (Too few or many arguments)\n", l)
|
||||
ch, err := getWatchChan(c, flagset.Args())
|
||||
if err != nil {
|
||||
fmt.Fprintf(os.Stderr, "Invalid command %s (%v)\n", l, err)
|
||||
continue
|
||||
}
|
||||
var key string
|
||||
_, err = fmt.Sscanf(moreargs[0], "%q", &key)
|
||||
if err != nil {
|
||||
key = moreargs[0]
|
||||
}
|
||||
opts := []clientv3.OpOption{clientv3.WithRev(watchRev)}
|
||||
if len(moreargs) == 2 {
|
||||
if watchPrefix {
|
||||
fmt.Fprintf(os.Stderr, "`range_end` and `--prefix` cannot be set at the same time, choose one\n")
|
||||
continue
|
||||
}
|
||||
opts = append(opts, clientv3.WithRange(moreargs[1]))
|
||||
}
|
||||
if watchPrefix {
|
||||
opts = append(opts, clientv3.WithPrefix())
|
||||
}
|
||||
ch := c.Watch(context.TODO(), key, opts...)
|
||||
go printWatchCh(ch)
|
||||
}
|
||||
}
|
||||
|
||||
func getWatchChan(c *clientv3.Client, args []string) (clientv3.WatchChan, error) {
|
||||
if len(args) < 1 || len(args) > 2 {
|
||||
return nil, fmt.Errorf("bad number of arguments")
|
||||
}
|
||||
key := args[0]
|
||||
opts := []clientv3.OpOption{clientv3.WithRev(watchRev)}
|
||||
if len(args) == 2 {
|
||||
if watchPrefix {
|
||||
return nil, fmt.Errorf("`range_end` and `--prefix` are mutually exclusive")
|
||||
}
|
||||
opts = append(opts, clientv3.WithRange(args[1]))
|
||||
}
|
||||
if watchPrefix {
|
||||
opts = append(opts, clientv3.WithPrefix())
|
||||
}
|
||||
if watchPrevKey {
|
||||
opts = append(opts, clientv3.WithPrevKV())
|
||||
}
|
||||
return c.Watch(context.TODO(), key, opts...), nil
|
||||
}
|
||||
|
||||
func printWatchCh(ch clientv3.WatchChan) {
|
||||
for resp := range ch {
|
||||
display.Watch(resp)
|
||||
|
@ -20,6 +20,7 @@ import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"net"
|
||||
"net/url"
|
||||
"os"
|
||||
"runtime"
|
||||
@ -410,6 +411,13 @@ func (cfg *config) configFromFile() error {
|
||||
}
|
||||
|
||||
func (cfg *config) validateConfig(isSet func(field string) bool) error {
|
||||
if err := checkBindURLs(cfg.lpurls); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := checkBindURLs(cfg.lcurls); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// when etcd runs in member mode user needs to set --advertise-client-urls if --listen-client-urls is set.
|
||||
// TODO(yichengq): check this for joining through discovery service case
|
||||
mayFallbackToProxy := isSet("discovery") && cfg.fallback.String() == fallbackFlagProxy
|
||||
@ -456,3 +464,27 @@ func (cfg config) isReadonlyProxy() bool { return cfg.proxy.String() == pr
|
||||
func (cfg config) shouldFallbackToProxy() bool { return cfg.fallback.String() == fallbackFlagProxy }
|
||||
|
||||
func (cfg config) electionTicks() int { return int(cfg.ElectionMs / cfg.TickMs) }
|
||||
|
||||
// checkBindURLs returns an error if any URL uses a domain name.
|
||||
// TODO: return error in 3.2.0
|
||||
func checkBindURLs(urls []url.URL) error {
|
||||
for _, url := range urls {
|
||||
if url.Scheme == "unix" || url.Scheme == "unixs" {
|
||||
continue
|
||||
}
|
||||
host, _, err := net.SplitHostPort(url.Host)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if host == "localhost" {
|
||||
// special case for local address
|
||||
// TODO: support /etc/hosts ?
|
||||
continue
|
||||
}
|
||||
if net.ParseIP(host) == nil {
|
||||
err := fmt.Errorf("expected IP in URL for binding (%s)", url.String())
|
||||
plog.Warning(err)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -561,6 +561,9 @@ func getPeerURLsMapAndToken(cfg *config, which string) (urlsmap types.URLsMap, t
|
||||
if err != nil {
|
||||
return nil, "", err
|
||||
}
|
||||
if strings.Contains(clusterStr, "https://") && cfg.peerTLSInfo.CAFile == "" {
|
||||
cfg.peerTLSInfo.ServerName = cfg.DnsCluster
|
||||
}
|
||||
urlsmap, err = types.NewURLsMap(clusterStr)
|
||||
// only etcd member must belong to the discovered cluster.
|
||||
// proxy does not need to belong to the discovered cluster.
|
||||
|
@ -20,14 +20,19 @@ import (
|
||||
"os"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/client"
|
||||
"github.com/coreos/etcd/pkg/transport"
|
||||
"github.com/coreos/etcd/proxy/tcpproxy"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
var (
|
||||
gatewayListenAddr string
|
||||
gatewayEndpoints []string
|
||||
getewayRetryDelay time.Duration
|
||||
gatewayListenAddr string
|
||||
gatewayEndpoints []string
|
||||
gatewayDNSCluster string
|
||||
gatewayInsecureDiscovery bool
|
||||
getewayRetryDelay time.Duration
|
||||
gatewayCA string
|
||||
)
|
||||
|
||||
var (
|
||||
@ -61,6 +66,10 @@ func newGatewayStartCommand() *cobra.Command {
|
||||
}
|
||||
|
||||
cmd.Flags().StringVar(&gatewayListenAddr, "listen-addr", "127.0.0.1:23790", "listen address")
|
||||
cmd.Flags().StringVar(&gatewayDNSCluster, "discovery-srv", "", "DNS domain used to bootstrap initial cluster")
|
||||
cmd.Flags().BoolVar(&gatewayInsecureDiscovery, "insecure-discovery", false, "accept insecure SRV records")
|
||||
cmd.Flags().StringVar(&gatewayCA, "trusted-ca-file", "", "path to the client server TLS CA file.")
|
||||
|
||||
cmd.Flags().StringSliceVar(&gatewayEndpoints, "endpoints", []string{"127.0.0.1:2379"}, "comma separated etcd cluster endpoints")
|
||||
cmd.Flags().DurationVar(&getewayRetryDelay, "retry-delay", time.Minute, "duration of delay before retrying failed endpoints")
|
||||
|
||||
@ -68,6 +77,33 @@ func newGatewayStartCommand() *cobra.Command {
|
||||
}
|
||||
|
||||
func startGateway(cmd *cobra.Command, args []string) {
|
||||
endpoints := gatewayEndpoints
|
||||
if gatewayDNSCluster != "" {
|
||||
eps, err := client.NewSRVDiscover().Discover(gatewayDNSCluster)
|
||||
if err != nil {
|
||||
fmt.Fprintln(os.Stderr, err)
|
||||
os.Exit(1)
|
||||
}
|
||||
plog.Infof("discovered the cluster %s from %s", eps, gatewayDNSCluster)
|
||||
// confirm TLS connections are good
|
||||
if !gatewayInsecureDiscovery {
|
||||
tlsInfo := transport.TLSInfo{
|
||||
TrustedCAFile: gatewayCA,
|
||||
ServerName: gatewayDNSCluster,
|
||||
}
|
||||
plog.Infof("validating discovered endpoints %v", eps)
|
||||
endpoints, err = transport.ValidateSecureEndpoints(tlsInfo, eps)
|
||||
if err != nil {
|
||||
plog.Warningf("%v", err)
|
||||
}
|
||||
plog.Infof("using discovered endpoints %v", endpoints)
|
||||
}
|
||||
}
|
||||
|
||||
if len(endpoints) == 0 {
|
||||
plog.Fatalf("no endpoints found")
|
||||
}
|
||||
|
||||
l, err := net.Listen("tcp", gatewayListenAddr)
|
||||
if err != nil {
|
||||
fmt.Fprintln(os.Stderr, err)
|
||||
|
@ -116,10 +116,11 @@ func hasKeyPrefixAccess(sec auth.Store, r *http.Request, key string, recursive,
|
||||
}
|
||||
|
||||
var user *auth.User
|
||||
if r.Header.Get("Authorization") == "" && clientCertAuthEnabled {
|
||||
user = userFromClientCertificate(sec, r)
|
||||
if r.Header.Get("Authorization") == "" {
|
||||
if clientCertAuthEnabled {
|
||||
user = userFromClientCertificate(sec, r)
|
||||
}
|
||||
if user == nil {
|
||||
plog.Warningf("auth: no authorization provided, checking guest access")
|
||||
return hasGuestAccess(sec, r, key)
|
||||
}
|
||||
} else {
|
||||
|
@ -717,6 +717,36 @@ func TestPrefixAccess(t *testing.T) {
|
||||
hasKeyPrefixAccess: false,
|
||||
hasRecursiveAccess: false,
|
||||
},
|
||||
{ // guest access in non-TLS mode
|
||||
key: "/foo",
|
||||
req: (func() *http.Request {
|
||||
return mustJSONRequest(t, "GET", "somepath", "")
|
||||
})(),
|
||||
store: &mockAuthStore{
|
||||
enabled: true,
|
||||
users: map[string]*auth.User{
|
||||
"root": {
|
||||
User: "root",
|
||||
Password: goodPassword,
|
||||
Roles: []string{"root"},
|
||||
},
|
||||
},
|
||||
roles: map[string]*auth.Role{
|
||||
"guest": {
|
||||
Role: "guest",
|
||||
Permissions: auth.Permissions{
|
||||
KV: auth.RWPermission{
|
||||
Read: []string{"/foo*"},
|
||||
Write: []string{"/foo*"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
hasRoot: false,
|
||||
hasKeyPrefixAccess: true,
|
||||
hasRecursiveAccess: true,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tt := range table {
|
||||
|
@ -32,7 +32,7 @@ type watchServer struct {
|
||||
clusterID int64
|
||||
memberID int64
|
||||
raftTimer etcdserver.RaftTimer
|
||||
watchable mvcc.Watchable
|
||||
watchable mvcc.WatchableKV
|
||||
}
|
||||
|
||||
func NewWatchServer(s *etcdserver.EtcdServer) pb.WatchServer {
|
||||
@ -82,6 +82,8 @@ type serverWatchStream struct {
|
||||
memberID int64
|
||||
raftTimer etcdserver.RaftTimer
|
||||
|
||||
watchable mvcc.WatchableKV
|
||||
|
||||
gRPCStream pb.Watch_WatchServer
|
||||
watchStream mvcc.WatchStream
|
||||
ctrlStream chan *pb.WatchResponse
|
||||
@ -91,6 +93,7 @@ type serverWatchStream struct {
|
||||
// progress tracks the watchID that stream might need to send
|
||||
// progress to.
|
||||
progress map[mvcc.WatchID]bool
|
||||
prevKV map[mvcc.WatchID]bool
|
||||
|
||||
// closec indicates the stream is closed.
|
||||
closec chan struct{}
|
||||
@ -101,14 +104,18 @@ type serverWatchStream struct {
|
||||
|
||||
func (ws *watchServer) Watch(stream pb.Watch_WatchServer) (err error) {
|
||||
sws := serverWatchStream{
|
||||
clusterID: ws.clusterID,
|
||||
memberID: ws.memberID,
|
||||
raftTimer: ws.raftTimer,
|
||||
clusterID: ws.clusterID,
|
||||
memberID: ws.memberID,
|
||||
raftTimer: ws.raftTimer,
|
||||
|
||||
watchable: ws.watchable,
|
||||
|
||||
gRPCStream: stream,
|
||||
watchStream: ws.watchable.NewWatchStream(),
|
||||
// chan for sending control response like watcher created and canceled.
|
||||
ctrlStream: make(chan *pb.WatchResponse, ctrlStreamBufLen),
|
||||
progress: make(map[mvcc.WatchID]bool),
|
||||
prevKV: make(map[mvcc.WatchID]bool),
|
||||
closec: make(chan struct{}),
|
||||
}
|
||||
|
||||
@ -170,9 +177,14 @@ func (sws *serverWatchStream) recvLoop() error {
|
||||
rev = wsrev + 1
|
||||
}
|
||||
id := sws.watchStream.Watch(creq.Key, creq.RangeEnd, rev)
|
||||
if id != -1 && creq.ProgressNotify {
|
||||
if id != -1 {
|
||||
sws.mu.Lock()
|
||||
sws.progress[id] = true
|
||||
if creq.ProgressNotify {
|
||||
sws.progress[id] = true
|
||||
}
|
||||
if creq.PrevKv {
|
||||
sws.prevKV[id] = true
|
||||
}
|
||||
sws.mu.Unlock()
|
||||
}
|
||||
wr := &pb.WatchResponse{
|
||||
@ -198,6 +210,7 @@ func (sws *serverWatchStream) recvLoop() error {
|
||||
}
|
||||
sws.mu.Lock()
|
||||
delete(sws.progress, mvcc.WatchID(id))
|
||||
delete(sws.prevKV, mvcc.WatchID(id))
|
||||
sws.mu.Unlock()
|
||||
}
|
||||
}
|
||||
@ -244,8 +257,19 @@ func (sws *serverWatchStream) sendLoop() {
|
||||
// or define protocol buffer with []mvccpb.Event.
|
||||
evs := wresp.Events
|
||||
events := make([]*mvccpb.Event, len(evs))
|
||||
sws.mu.Lock()
|
||||
needPrevKV := sws.prevKV[wresp.WatchID]
|
||||
sws.mu.Unlock()
|
||||
for i := range evs {
|
||||
events[i] = &evs[i]
|
||||
|
||||
if needPrevKV {
|
||||
opt := mvcc.RangeOptions{Rev: evs[i].Kv.ModRevision - 1}
|
||||
r, err := sws.watchable.Range(evs[i].Kv.Key, nil, opt)
|
||||
if err == nil && len(r.KVs) != 0 {
|
||||
events[i].PrevKv = &(r.KVs[0])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
wr := &pb.WatchResponse{
|
||||
|
@ -159,6 +159,22 @@ func (a *applierV3backend) Put(txnID int64, p *pb.PutRequest) (*pb.PutResponse,
|
||||
rev int64
|
||||
err error
|
||||
)
|
||||
|
||||
var rr *mvcc.RangeResult
|
||||
if p.PrevKv {
|
||||
if txnID != noTxn {
|
||||
rr, err = a.s.KV().TxnRange(txnID, p.Key, nil, mvcc.RangeOptions{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
} else {
|
||||
rr, err = a.s.KV().Range(p.Key, nil, mvcc.RangeOptions{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if txnID != noTxn {
|
||||
rev, err = a.s.KV().TxnPut(txnID, p.Key, p.Value, lease.LeaseID(p.Lease))
|
||||
if err != nil {
|
||||
@ -174,6 +190,9 @@ func (a *applierV3backend) Put(txnID int64, p *pb.PutRequest) (*pb.PutResponse,
|
||||
rev = a.s.KV().Put(p.Key, p.Value, leaseID)
|
||||
}
|
||||
resp.Header.Revision = rev
|
||||
if rr != nil && len(rr.KVs) != 0 {
|
||||
resp.PrevKv = &rr.KVs[0]
|
||||
}
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
@ -191,6 +210,21 @@ func (a *applierV3backend) DeleteRange(txnID int64, dr *pb.DeleteRangeRequest) (
|
||||
dr.RangeEnd = []byte{}
|
||||
}
|
||||
|
||||
var rr *mvcc.RangeResult
|
||||
if dr.PrevKv {
|
||||
if txnID != noTxn {
|
||||
rr, err = a.s.KV().TxnRange(txnID, dr.Key, dr.RangeEnd, mvcc.RangeOptions{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
} else {
|
||||
rr, err = a.s.KV().Range(dr.Key, dr.RangeEnd, mvcc.RangeOptions{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if txnID != noTxn {
|
||||
n, rev, err = a.s.KV().TxnDeleteRange(txnID, dr.Key, dr.RangeEnd)
|
||||
if err != nil {
|
||||
@ -201,6 +235,11 @@ func (a *applierV3backend) DeleteRange(txnID int64, dr *pb.DeleteRangeRequest) (
|
||||
}
|
||||
|
||||
resp.Deleted = n
|
||||
if rr != nil {
|
||||
for i := range rr.KVs {
|
||||
resp.PrevKvs = append(resp.PrevKvs, &rr.KVs[i])
|
||||
}
|
||||
}
|
||||
resp.Header.Revision = rev
|
||||
return resp, nil
|
||||
}
|
||||
|
@ -56,6 +56,9 @@ func (aa *authApplierV3) Put(txnID int64, r *pb.PutRequest) (*pb.PutResponse, er
|
||||
if !aa.as.IsPutPermitted(aa.user, r.Key) {
|
||||
return nil, auth.ErrPermissionDenied
|
||||
}
|
||||
if r.PrevKv && !aa.as.IsRangePermitted(aa.user, r.Key, nil) {
|
||||
return nil, auth.ErrPermissionDenied
|
||||
}
|
||||
return aa.applierV3.Put(txnID, r)
|
||||
}
|
||||
|
||||
@ -70,6 +73,9 @@ func (aa *authApplierV3) DeleteRange(txnID int64, r *pb.DeleteRangeRequest) (*pb
|
||||
if !aa.as.IsDeleteRangePermitted(aa.user, r.Key, r.RangeEnd) {
|
||||
return nil, auth.ErrPermissionDenied
|
||||
}
|
||||
if r.PrevKv && !aa.as.IsRangePermitted(aa.user, r.Key, r.RangeEnd) {
|
||||
return nil, auth.ErrPermissionDenied
|
||||
}
|
||||
return aa.applierV3.DeleteRange(txnID, r)
|
||||
}
|
||||
|
||||
@ -99,7 +105,7 @@ func (aa *authApplierV3) checkTxnReqsPermission(reqs []*pb.RequestOp) bool {
|
||||
continue
|
||||
}
|
||||
|
||||
if !aa.as.IsDeleteRangePermitted(aa.user, tv.RequestDeleteRange.Key, tv.RequestDeleteRange.RangeEnd) {
|
||||
if tv.RequestDeleteRange.PrevKv && !aa.as.IsRangePermitted(aa.user, tv.RequestDeleteRange.Key, tv.RequestDeleteRange.RangeEnd) {
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
@ -102,9 +102,9 @@ import (
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
|
||||
math "math"
|
||||
)
|
||||
|
||||
import io "io"
|
||||
io "io"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
|
@ -10,9 +10,9 @@ import (
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
|
||||
math "math"
|
||||
)
|
||||
|
||||
import io "io"
|
||||
io "io"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -396,10 +396,16 @@ message PutRequest {
|
||||
// lease is the lease ID to associate with the key in the key-value store. A lease
|
||||
// value of 0 indicates no lease.
|
||||
int64 lease = 3;
|
||||
|
||||
// If prev_kv is set, etcd gets the previous key-value pair before changing it.
|
||||
// The previous key-value pair will be returned in the put response.
|
||||
bool prev_kv = 4;
|
||||
}
|
||||
|
||||
message PutResponse {
|
||||
ResponseHeader header = 1;
|
||||
// if prev_kv is set in the request, the previous key-value pair will be returned.
|
||||
mvccpb.KeyValue prev_kv = 2;
|
||||
}
|
||||
|
||||
message DeleteRangeRequest {
|
||||
@ -409,12 +415,17 @@ message DeleteRangeRequest {
|
||||
// If range_end is not given, the range is defined to contain only the key argument.
|
||||
// If range_end is '\0', the range is all keys greater than or equal to the key argument.
|
||||
bytes range_end = 2;
|
||||
// If prev_kv is set, etcd gets the previous key-value pairs before deleting it.
|
||||
// The previous key-value pairs will be returned in the delte response.
|
||||
bool prev_kv = 3;
|
||||
}
|
||||
|
||||
message DeleteRangeResponse {
|
||||
ResponseHeader header = 1;
|
||||
// deleted is the number of keys deleted by the delete range request.
|
||||
int64 deleted = 2;
|
||||
// if prev_kv is set in the request, the previous key-value pairs will be returned.
|
||||
repeated mvccpb.KeyValue prev_kvs = 3;
|
||||
}
|
||||
|
||||
message RequestOp {
|
||||
@ -563,6 +574,9 @@ message WatchCreateRequest {
|
||||
// wish to recover a disconnected watcher starting from a recent known revision.
|
||||
// The etcd server may decide how often it will send notifications based on current load.
|
||||
bool progress_notify = 4;
|
||||
// If prev_kv is set, created watcher gets the previous KV before the event happens.
|
||||
// If the previous KV is already compacted, nothing will be returned.
|
||||
bool prev_kv = 6;
|
||||
}
|
||||
|
||||
message WatchCancelRequest {
|
||||
|
@ -405,14 +405,24 @@ func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
|
||||
|
||||
srv.be = be
|
||||
srv.lessor = lease.NewLessor(srv.be)
|
||||
|
||||
// always recover lessor before kv. When we recover the mvcc.KV it will reattach keys to its leases.
|
||||
// If we recover mvcc.KV first, it will attach the keys to the wrong lessor before it recovers.
|
||||
srv.lessor = lease.NewLessor(srv.be)
|
||||
srv.kv = mvcc.New(srv.be, srv.lessor, &srv.consistIndex)
|
||||
if beExist {
|
||||
kvindex := srv.kv.ConsistentIndex()
|
||||
// TODO: remove kvindex != 0 checking when we do not expect users to upgrade
|
||||
// etcd from pre-3.0 release.
|
||||
if snapshot != nil && kvindex < snapshot.Metadata.Index {
|
||||
return nil, fmt.Errorf("database file (%v index %d) does not match with snapshot (index %d).", bepath, kvindex, snapshot.Metadata.Index)
|
||||
if kvindex != 0 {
|
||||
return nil, fmt.Errorf("database file (%v index %d) does not match with snapshot (index %d).", bepath, kvindex, snapshot.Metadata.Index)
|
||||
}
|
||||
plog.Warningf("consistent index never saved (snapshot index=%d)", snapshot.Metadata.Index)
|
||||
}
|
||||
}
|
||||
srv.consistIndex.setConsistentIndex(srv.kv.ConsistentIndex())
|
||||
|
||||
srv.authStore = auth.NewAuthStore(srv.be)
|
||||
if h := cfg.AutoCompactionRetention; h != 0 {
|
||||
srv.compactor = compactor.NewPeriodic(h, srv.kv, srv)
|
||||
@ -658,6 +668,14 @@ func (s *EtcdServer) applySnapshot(ep *etcdProgress, apply *apply) {
|
||||
|
||||
newbe := backend.NewDefaultBackend(fn)
|
||||
|
||||
// always recover lessor before kv. When we recover the mvcc.KV it will reattach keys to its leases.
|
||||
// If we recover mvcc.KV first, it will attach the keys to the wrong lessor before it recovers.
|
||||
if s.lessor != nil {
|
||||
plog.Info("recovering lessor...")
|
||||
s.lessor.Recover(newbe, s.kv)
|
||||
plog.Info("finished recovering lessor")
|
||||
}
|
||||
|
||||
plog.Info("restoring mvcc store...")
|
||||
|
||||
if err := s.kv.Restore(newbe); err != nil {
|
||||
@ -684,12 +702,6 @@ func (s *EtcdServer) applySnapshot(ep *etcdProgress, apply *apply) {
|
||||
s.be = newbe
|
||||
s.bemu.Unlock()
|
||||
|
||||
if s.lessor != nil {
|
||||
plog.Info("recovering lessor...")
|
||||
s.lessor.Recover(newbe, s.kv)
|
||||
plog.Info("finished recovering lessor")
|
||||
}
|
||||
|
||||
plog.Info("recovering alarms...")
|
||||
if err := s.restoreAlarms(); err != nil {
|
||||
plog.Panicf("restore alarms error: %v", err)
|
||||
|
@ -551,4 +551,4 @@ func (s *EtcdServer) processInternalRaftRequest(ctx context.Context, r pb.Intern
|
||||
}
|
||||
|
||||
// Watchable returns a watchable interface attached to the etcdserver.
|
||||
func (s *EtcdServer) Watchable() mvcc.Watchable { return s.KV() }
|
||||
func (s *EtcdServer) Watchable() mvcc.WatchableKV { return s.KV() }
|
||||
|
@ -752,6 +752,7 @@ func NewClusterV3(t *testing.T, cfg *ClusterConfig) *ClusterV3 {
|
||||
clus := &ClusterV3{
|
||||
cluster: NewClusterByConfig(t, cfg),
|
||||
}
|
||||
clus.Launch(t)
|
||||
for _, m := range clus.Members {
|
||||
client, err := NewClientV3(m)
|
||||
if err != nil {
|
||||
@ -759,7 +760,6 @@ func NewClusterV3(t *testing.T, cfg *ClusterConfig) *ClusterV3 {
|
||||
}
|
||||
clus.clients = append(clus.clients, client)
|
||||
}
|
||||
clus.Launch(t)
|
||||
|
||||
return clus
|
||||
}
|
||||
|
@ -174,3 +174,28 @@ func TestElectionSessionRecampaign(t *testing.T) {
|
||||
t.Fatalf("expected value=%q, got response %v", "def", resp)
|
||||
}
|
||||
}
|
||||
|
||||
// TestElectionOnPrefixOfExistingKey checks that a single
|
||||
// candidate can be elected on a new key that is a prefix
|
||||
// of an existing key. To wit, check for regression
|
||||
// of bug #6278. https://github.com/coreos/etcd/issues/6278
|
||||
//
|
||||
func TestElectionOnPrefixOfExistingKey(t *testing.T) {
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
if _, err := cli.Put(context.TODO(), "testa", "value"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
e := concurrency.NewElection(cli, "test")
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), 5*time.Second)
|
||||
err := e.Campaign(ctx, "abc")
|
||||
cancel()
|
||||
if err != nil {
|
||||
// after 5 seconds, deadlock results in
|
||||
// 'context deadline exceeded' here.
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
@ -379,6 +379,7 @@ func TestV3DeleteRange(t *testing.T) {
|
||||
keySet []string
|
||||
begin string
|
||||
end string
|
||||
prevKV bool
|
||||
|
||||
wantSet [][]byte
|
||||
deleted int64
|
||||
@ -386,39 +387,45 @@ func TestV3DeleteRange(t *testing.T) {
|
||||
// delete middle
|
||||
{
|
||||
[]string{"foo", "foo/abc", "fop"},
|
||||
"foo/", "fop",
|
||||
"foo/", "fop", false,
|
||||
[][]byte{[]byte("foo"), []byte("fop")}, 1,
|
||||
},
|
||||
// no delete
|
||||
{
|
||||
[]string{"foo", "foo/abc", "fop"},
|
||||
"foo/", "foo/",
|
||||
"foo/", "foo/", false,
|
||||
[][]byte{[]byte("foo"), []byte("foo/abc"), []byte("fop")}, 0,
|
||||
},
|
||||
// delete first
|
||||
{
|
||||
[]string{"foo", "foo/abc", "fop"},
|
||||
"fo", "fop",
|
||||
"fo", "fop", false,
|
||||
[][]byte{[]byte("fop")}, 2,
|
||||
},
|
||||
// delete tail
|
||||
{
|
||||
[]string{"foo", "foo/abc", "fop"},
|
||||
"foo/", "fos",
|
||||
"foo/", "fos", false,
|
||||
[][]byte{[]byte("foo")}, 2,
|
||||
},
|
||||
// delete exact
|
||||
{
|
||||
[]string{"foo", "foo/abc", "fop"},
|
||||
"foo/abc", "",
|
||||
"foo/abc", "", false,
|
||||
[][]byte{[]byte("foo"), []byte("fop")}, 1,
|
||||
},
|
||||
// delete none, [x,x)
|
||||
{
|
||||
[]string{"foo"},
|
||||
"foo", "foo",
|
||||
"foo", "foo", false,
|
||||
[][]byte{[]byte("foo")}, 0,
|
||||
},
|
||||
// delete middle with preserveKVs set
|
||||
{
|
||||
[]string{"foo", "foo/abc", "fop"},
|
||||
"foo/", "fop", true,
|
||||
[][]byte{[]byte("foo"), []byte("fop")}, 1,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
@ -436,7 +443,9 @@ func TestV3DeleteRange(t *testing.T) {
|
||||
|
||||
dreq := &pb.DeleteRangeRequest{
|
||||
Key: []byte(tt.begin),
|
||||
RangeEnd: []byte(tt.end)}
|
||||
RangeEnd: []byte(tt.end),
|
||||
PrevKv: tt.prevKV,
|
||||
}
|
||||
dresp, err := kvc.DeleteRange(context.TODO(), dreq)
|
||||
if err != nil {
|
||||
t.Fatalf("couldn't delete range on test %d (%v)", i, err)
|
||||
@ -444,6 +453,11 @@ func TestV3DeleteRange(t *testing.T) {
|
||||
if tt.deleted != dresp.Deleted {
|
||||
t.Errorf("expected %d on test %v, got %d", tt.deleted, i, dresp.Deleted)
|
||||
}
|
||||
if tt.prevKV {
|
||||
if len(dresp.PrevKvs) != int(dresp.Deleted) {
|
||||
t.Errorf("preserve %d keys, want %d", len(dresp.PrevKvs), dresp.Deleted)
|
||||
}
|
||||
}
|
||||
|
||||
rreq := &pb.RangeRequest{Key: []byte{0x0}, RangeEnd: []byte{0xff}}
|
||||
rresp, err := kvc.Range(context.TODO(), rreq)
|
||||
|
@ -19,9 +19,9 @@ import (
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
|
||||
math "math"
|
||||
)
|
||||
|
||||
import io "io"
|
||||
io "io"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
|
@ -45,13 +45,18 @@ var (
|
||||
|
||||
type LeaseID int64
|
||||
|
||||
// RangeDeleter defines an interface with DeleteRange method.
|
||||
// RangeDeleter defines an interface with Txn and DeleteRange method.
|
||||
// We define this interface only for lessor to limit the number
|
||||
// of methods of mvcc.KV to what lessor actually needs.
|
||||
//
|
||||
// Having a minimum interface makes testing easy.
|
||||
type RangeDeleter interface {
|
||||
DeleteRange(key, end []byte) (int64, int64)
|
||||
// TxnBegin see comments on mvcc.KV
|
||||
TxnBegin() int64
|
||||
// TxnEnd see comments on mvcc.KV
|
||||
TxnEnd(txnID int64) error
|
||||
// TxnDeleteRange see comments on mvcc.KV
|
||||
TxnDeleteRange(txnID int64, key, end []byte) (n, rev int64, err error)
|
||||
}
|
||||
|
||||
// Lessor owns leases. It can grant, revoke, renew and modify leases for lessee.
|
||||
@ -211,16 +216,30 @@ func (le *lessor) Revoke(id LeaseID) error {
|
||||
// unlock before doing external work
|
||||
le.mu.Unlock()
|
||||
|
||||
if le.rd != nil {
|
||||
for item := range l.itemSet {
|
||||
le.rd.DeleteRange([]byte(item.Key), nil)
|
||||
if le.rd == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
tid := le.rd.TxnBegin()
|
||||
for item := range l.itemSet {
|
||||
_, _, err := le.rd.TxnDeleteRange(tid, []byte(item.Key), nil)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
le.mu.Lock()
|
||||
defer le.mu.Unlock()
|
||||
delete(le.leaseMap, l.ID)
|
||||
l.removeFrom(le.b)
|
||||
// lease deletion needs to be in the same backend transaction with the
|
||||
// kv deletion. Or we might end up with not executing the revoke or not
|
||||
// deleting the keys if etcdserver fails in between.
|
||||
le.b.BatchTx().UnsafeDelete(leaseBucketName, int64ToBytes(int64(l.ID)))
|
||||
|
||||
err := le.rd.TxnEnd(tid)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
@ -443,16 +462,7 @@ func (l Lease) persistTo(b backend.Backend) {
|
||||
b.BatchTx().Unlock()
|
||||
}
|
||||
|
||||
func (l Lease) removeFrom(b backend.Backend) {
|
||||
key := int64ToBytes(int64(l.ID))
|
||||
|
||||
b.BatchTx().Lock()
|
||||
b.BatchTx().UnsafeDelete(leaseBucketName, key)
|
||||
b.BatchTx().Unlock()
|
||||
}
|
||||
|
||||
// refresh refreshes the expiry of the lease. It extends the expiry at least
|
||||
// minLeaseTTL second.
|
||||
// refresh refreshes the expiry of the lease.
|
||||
func (l *Lease) refresh(extend time.Duration) {
|
||||
if l.TTL < minLeaseTTL {
|
||||
l.TTL = minLeaseTTL
|
||||
|
@ -223,9 +223,17 @@ type fakeDeleter struct {
|
||||
deleted []string
|
||||
}
|
||||
|
||||
func (fd *fakeDeleter) DeleteRange(key, end []byte) (int64, int64) {
|
||||
func (fd *fakeDeleter) TxnBegin() int64 {
|
||||
return 0
|
||||
}
|
||||
|
||||
func (fd *fakeDeleter) TxnEnd(txnID int64) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fd *fakeDeleter) TxnDeleteRange(tid int64, key, end []byte) (int64, int64, error) {
|
||||
fd.deleted = append(fd.deleted, string(key)+"_"+string(end))
|
||||
return 0, 0
|
||||
return 0, 0, nil
|
||||
}
|
||||
|
||||
func NewTestBackend(t *testing.T) (string, backend.Backend) {
|
||||
|
@ -245,6 +245,9 @@ func testKVRangeLimit(t *testing.T, f rangeFunc) {
|
||||
if r.Rev != wrev {
|
||||
t.Errorf("#%d: rev = %d, want %d", i, r.Rev, wrev)
|
||||
}
|
||||
if r.Count != len(kvs) {
|
||||
t.Errorf("#%d: count = %d, want %d", i, r.Count, len(kvs))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -367,6 +367,8 @@ func (s *store) restore() error {
|
||||
revToBytes(revision{main: 1}, min)
|
||||
revToBytes(revision{main: math.MaxInt64, sub: math.MaxInt64}, max)
|
||||
|
||||
keyToLease := make(map[string]lease.LeaseID)
|
||||
|
||||
// restore index
|
||||
tx := s.b.BatchTx()
|
||||
tx.Lock()
|
||||
@ -390,26 +392,15 @@ func (s *store) restore() error {
|
||||
switch {
|
||||
case isTombstone(key):
|
||||
s.kvindex.Tombstone(kv.Key, rev)
|
||||
if lease.LeaseID(kv.Lease) != lease.NoLease {
|
||||
err := s.le.Detach(lease.LeaseID(kv.Lease), []lease.LeaseItem{{Key: string(kv.Key)}})
|
||||
if err != nil && err != lease.ErrLeaseNotFound {
|
||||
plog.Fatalf("unexpected Detach error %v", err)
|
||||
}
|
||||
}
|
||||
delete(keyToLease, string(kv.Key))
|
||||
|
||||
default:
|
||||
s.kvindex.Restore(kv.Key, revision{kv.CreateRevision, 0}, rev, kv.Version)
|
||||
if lease.LeaseID(kv.Lease) != lease.NoLease {
|
||||
if s.le == nil {
|
||||
panic("no lessor to attach lease")
|
||||
}
|
||||
err := s.le.Attach(lease.LeaseID(kv.Lease), []lease.LeaseItem{{Key: string(kv.Key)}})
|
||||
// We are walking through the kv history here. It is possible that we attached a key to
|
||||
// the lease and the lease was revoked later.
|
||||
// Thus attaching an old version of key to a none existing lease is possible here, and
|
||||
// we should just ignore the error.
|
||||
if err != nil && err != lease.ErrLeaseNotFound {
|
||||
panic("unexpected Attach error")
|
||||
}
|
||||
|
||||
if lid := lease.LeaseID(kv.Lease); lid != lease.NoLease {
|
||||
keyToLease[string(kv.Key)] = lid
|
||||
} else {
|
||||
delete(keyToLease, string(kv.Key))
|
||||
}
|
||||
}
|
||||
|
||||
@ -417,6 +408,16 @@ func (s *store) restore() error {
|
||||
s.currentRev = rev
|
||||
}
|
||||
|
||||
for key, lid := range keyToLease {
|
||||
if s.le == nil {
|
||||
panic("no lessor to attach lease")
|
||||
}
|
||||
err := s.le.Attach(lid, []lease.LeaseItem{{Key: key}})
|
||||
if err != nil {
|
||||
plog.Errorf("unexpected Attach error: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
_, scheduledCompactBytes := tx.UnsafeRange(metaBucketName, scheduledCompactKeyName, nil, 0)
|
||||
scheduledCompact := int64(0)
|
||||
if len(scheduledCompactBytes) != 0 {
|
||||
@ -497,7 +498,7 @@ func (s *store) rangeKeys(key, end []byte, limit, rangeRev int64, countOnly bool
|
||||
break
|
||||
}
|
||||
}
|
||||
return kvs, len(kvs), curRev, nil
|
||||
return kvs, len(revpairs), curRev, nil
|
||||
}
|
||||
|
||||
func (s *store) put(key, value []byte, leaseID lease.LeaseID) {
|
||||
@ -550,7 +551,7 @@ func (s *store) put(key, value []byte, leaseID lease.LeaseID) {
|
||||
|
||||
err = s.le.Detach(oldLease, []lease.LeaseItem{{Key: string(key)}})
|
||||
if err != nil {
|
||||
panic("unexpected error from lease detach")
|
||||
plog.Errorf("unexpected error from lease detach: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
@ -619,7 +620,7 @@ func (s *store) delete(key []byte, rev revision) {
|
||||
if lease.LeaseID(kv.Lease) != lease.NoLease {
|
||||
err = s.le.Detach(lease.LeaseID(kv.Lease), []lease.LeaseItem{{Key: string(kv.Key)}})
|
||||
if err != nil {
|
||||
plog.Fatalf("cannot detach %v", err)
|
||||
plog.Errorf("cannot detach %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -20,9 +20,9 @@ import (
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
|
||||
math "math"
|
||||
)
|
||||
|
||||
import io "io"
|
||||
io "io"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
@ -89,6 +89,8 @@ type Event struct {
|
||||
// A DELETE/EXPIRE event contains the deleted key with
|
||||
// its modification revision set to the revision of deletion.
|
||||
Kv *KeyValue `protobuf:"bytes,2,opt,name=kv" json:"kv,omitempty"`
|
||||
// prev_kv holds the key-value pair before the event happens.
|
||||
PrevKv *KeyValue `protobuf:"bytes,3,opt,name=prev_kv,json=prevKv" json:"prev_kv,omitempty"`
|
||||
}
|
||||
|
||||
func (m *Event) Reset() { *m = Event{} }
|
||||
@ -181,6 +183,16 @@ func (m *Event) MarshalTo(data []byte) (int, error) {
|
||||
}
|
||||
i += n1
|
||||
}
|
||||
if m.PrevKv != nil {
|
||||
data[i] = 0x1a
|
||||
i++
|
||||
i = encodeVarintKv(data, i, uint64(m.PrevKv.Size()))
|
||||
n2, err := m.PrevKv.MarshalTo(data[i:])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i += n2
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
|
||||
@ -247,6 +259,10 @@ func (m *Event) Size() (n int) {
|
||||
l = m.Kv.Size()
|
||||
n += 1 + l + sovKv(uint64(l))
|
||||
}
|
||||
if m.PrevKv != nil {
|
||||
l = m.PrevKv.Size()
|
||||
n += 1 + l + sovKv(uint64(l))
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
@ -532,6 +548,39 @@ func (m *Event) Unmarshal(data []byte) error {
|
||||
return err
|
||||
}
|
||||
iNdEx = postIndex
|
||||
case 3:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowKv
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[iNdEx]
|
||||
iNdEx++
|
||||
msglen |= (int(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
if msglen < 0 {
|
||||
return ErrInvalidLengthKv
|
||||
}
|
||||
postIndex := iNdEx + msglen
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
if m.PrevKv == nil {
|
||||
m.PrevKv = &KeyValue{}
|
||||
}
|
||||
if err := m.PrevKv.Unmarshal(data[iNdEx:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
iNdEx = postIndex
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skipKv(data[iNdEx:])
|
||||
@ -659,23 +708,24 @@ var (
|
||||
)
|
||||
|
||||
var fileDescriptorKv = []byte{
|
||||
// 285 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x44, 0x90, 0x41, 0x4a, 0xc3, 0x40,
|
||||
0x14, 0x86, 0x33, 0x4d, 0x9b, 0xd6, 0xd7, 0x52, 0xc3, 0x50, 0x70, 0x70, 0x31, 0xc4, 0x6e, 0x2c,
|
||||
0x08, 0x11, 0xea, 0x0d, 0xc4, 0xac, 0x74, 0x21, 0x21, 0xba, 0x95, 0x34, 0x7d, 0x94, 0x92, 0xa6,
|
||||
0x13, 0xd2, 0x38, 0x98, 0x9b, 0x78, 0x0a, 0xcf, 0xd1, 0x65, 0x8f, 0x60, 0xe3, 0x45, 0x24, 0x6f,
|
||||
0x4c, 0xdd, 0x0c, 0xef, 0xff, 0xff, 0x6f, 0x98, 0xff, 0x0d, 0x0c, 0x52, 0xed, 0xe7, 0x85, 0x2a,
|
||||
0x15, 0x77, 0x32, 0x9d, 0x24, 0xf9, 0xe2, 0x72, 0xb2, 0x52, 0x2b, 0x45, 0xd6, 0x6d, 0x33, 0x99,
|
||||
0x74, 0xfa, 0xc5, 0x60, 0xf0, 0x88, 0xd5, 0x6b, 0xbc, 0x79, 0x47, 0xee, 0x82, 0x9d, 0x62, 0x25,
|
||||
0x98, 0xc7, 0x66, 0xa3, 0xb0, 0x19, 0xf9, 0x35, 0x9c, 0x27, 0x05, 0xc6, 0x25, 0xbe, 0x15, 0xa8,
|
||||
0xd7, 0xbb, 0xb5, 0xda, 0x8a, 0x8e, 0xc7, 0x66, 0x76, 0x38, 0x36, 0x76, 0xf8, 0xe7, 0xf2, 0x2b,
|
||||
0x18, 0x65, 0x6a, 0xf9, 0x4f, 0xd9, 0x44, 0x0d, 0x33, 0xb5, 0x3c, 0x21, 0x02, 0xfa, 0x1a, 0x0b,
|
||||
0x4a, 0xbb, 0x94, 0xb6, 0x92, 0x4f, 0xa0, 0xa7, 0x9b, 0x02, 0xa2, 0x47, 0x2f, 0x1b, 0xd1, 0xb8,
|
||||
0x1b, 0x8c, 0x77, 0x28, 0x1c, 0xa2, 0x8d, 0x98, 0x7e, 0x40, 0x2f, 0xd0, 0xb8, 0x2d, 0xf9, 0x0d,
|
||||
0x74, 0xcb, 0x2a, 0x47, 0x6a, 0x3b, 0x9e, 0x5f, 0xf8, 0x66, 0x4d, 0x9f, 0x42, 0x73, 0x46, 0x55,
|
||||
0x8e, 0x21, 0x41, 0xdc, 0x83, 0x4e, 0xaa, 0xa9, 0xfa, 0x70, 0xee, 0xb6, 0x68, 0xbb, 0x77, 0xd8,
|
||||
0x49, 0xf5, 0xd4, 0x83, 0xb3, 0xd3, 0x25, 0xde, 0x07, 0xfb, 0xf9, 0x25, 0x72, 0x2d, 0x0e, 0xe0,
|
||||
0x3c, 0x04, 0x4f, 0x41, 0x14, 0xb8, 0xec, 0x5e, 0xec, 0x8f, 0xd2, 0x3a, 0x1c, 0xa5, 0xb5, 0xaf,
|
||||
0x25, 0x3b, 0xd4, 0x92, 0x7d, 0xd7, 0x92, 0x7d, 0xfe, 0x48, 0x6b, 0xe1, 0xd0, 0x5f, 0xde, 0xfd,
|
||||
0x06, 0x00, 0x00, 0xff, 0xff, 0xd6, 0x21, 0x8f, 0x2c, 0x75, 0x01, 0x00, 0x00,
|
||||
// 303 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x6c, 0x90, 0x41, 0x4e, 0xc2, 0x40,
|
||||
0x14, 0x86, 0x3b, 0x14, 0x0a, 0x3e, 0x08, 0x36, 0x13, 0x12, 0x27, 0x2e, 0x26, 0x95, 0x8d, 0x18,
|
||||
0x13, 0x4c, 0xf0, 0x06, 0xc6, 0xae, 0x70, 0x61, 0x1a, 0x74, 0x4b, 0x4a, 0x79, 0x21, 0xa4, 0x94,
|
||||
0x69, 0x4a, 0x9d, 0xa4, 0x37, 0x71, 0xef, 0xde, 0x73, 0xb0, 0xe4, 0x08, 0x52, 0x2f, 0x62, 0xfa,
|
||||
0xc6, 0xe2, 0xc6, 0xcd, 0xe4, 0xfd, 0xff, 0xff, 0x65, 0xe6, 0x7f, 0x03, 0x9d, 0x58, 0x8f, 0xd3,
|
||||
0x4c, 0xe5, 0x8a, 0x3b, 0x89, 0x8e, 0xa2, 0x74, 0x71, 0x39, 0x58, 0xa9, 0x95, 0x22, 0xeb, 0xae,
|
||||
0x9a, 0x4c, 0x3a, 0xfc, 0x64, 0xd0, 0x99, 0x62, 0xf1, 0x1a, 0x6e, 0xde, 0x90, 0xbb, 0x60, 0xc7,
|
||||
0x58, 0x08, 0xe6, 0xb1, 0x51, 0x2f, 0xa8, 0x46, 0x7e, 0x0d, 0xe7, 0x51, 0x86, 0x61, 0x8e, 0xf3,
|
||||
0x0c, 0xf5, 0x7a, 0xb7, 0x56, 0x5b, 0xd1, 0xf0, 0xd8, 0xc8, 0x0e, 0xfa, 0xc6, 0x0e, 0x7e, 0x5d,
|
||||
0x7e, 0x05, 0xbd, 0x44, 0x2d, 0xff, 0x28, 0x9b, 0xa8, 0x6e, 0xa2, 0x96, 0x27, 0x44, 0x40, 0x5b,
|
||||
0x63, 0x46, 0x69, 0x93, 0xd2, 0x5a, 0xf2, 0x01, 0xb4, 0x74, 0x55, 0x40, 0xb4, 0xe8, 0x65, 0x23,
|
||||
0x2a, 0x77, 0x83, 0xe1, 0x0e, 0x85, 0x43, 0xb4, 0x11, 0xc3, 0x0f, 0x06, 0x2d, 0x5f, 0xe3, 0x36,
|
||||
0xe7, 0xb7, 0xd0, 0xcc, 0x8b, 0x14, 0xa9, 0x6e, 0x7f, 0x72, 0x31, 0x36, 0x7b, 0x8e, 0x29, 0x34,
|
||||
0xe7, 0xac, 0x48, 0x31, 0x20, 0x88, 0x7b, 0xd0, 0x88, 0x35, 0x75, 0xef, 0x4e, 0xdc, 0x1a, 0xad,
|
||||
0x17, 0x0f, 0x1a, 0xb1, 0xe6, 0x37, 0xd0, 0x4e, 0x33, 0xd4, 0xf3, 0x58, 0x53, 0xf9, 0xff, 0x30,
|
||||
0xa7, 0x02, 0xa6, 0x7a, 0xe8, 0xc1, 0xd9, 0xe9, 0x7e, 0xde, 0x06, 0xfb, 0xf9, 0x65, 0xe6, 0x5a,
|
||||
0x1c, 0xc0, 0x79, 0xf4, 0x9f, 0xfc, 0x99, 0xef, 0xb2, 0x07, 0xb1, 0x3f, 0x4a, 0xeb, 0x70, 0x94,
|
||||
0xd6, 0xbe, 0x94, 0xec, 0x50, 0x4a, 0xf6, 0x55, 0x4a, 0xf6, 0xfe, 0x2d, 0xad, 0x85, 0x43, 0xff,
|
||||
0x7e, 0xff, 0x13, 0x00, 0x00, 0xff, 0xff, 0xb5, 0x45, 0x92, 0x5d, 0xa1, 0x01, 0x00, 0x00,
|
||||
}
|
||||
|
@ -43,4 +43,6 @@ message Event {
|
||||
// A DELETE/EXPIRE event contains the deleted key with
|
||||
// its modification revision set to the revision of deletion.
|
||||
KeyValue kv = 2;
|
||||
// prev_kv holds the key-value pair before the event happens.
|
||||
KeyValue prev_kv = 3;
|
||||
}
|
||||
|
22
pkg/fileutil/dir_unix.go
Normal file
22
pkg/fileutil/dir_unix.go
Normal file
@ -0,0 +1,22 @@
|
||||
// Copyright 2016 The etcd Authors
|
||||
//
|
||||
// Licensed 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.
|
||||
|
||||
// +build !windows
|
||||
|
||||
package fileutil
|
||||
|
||||
import "os"
|
||||
|
||||
// OpenDir opens a directory for syncing.
|
||||
func OpenDir(path string) (*os.File, error) { return os.Open(path) }
|
46
pkg/fileutil/dir_windows.go
Normal file
46
pkg/fileutil/dir_windows.go
Normal file
@ -0,0 +1,46 @@
|
||||
// Copyright 2016 The etcd Authors
|
||||
//
|
||||
// Licensed 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.
|
||||
|
||||
// +build windows
|
||||
|
||||
package fileutil
|
||||
|
||||
import (
|
||||
"os"
|
||||
"syscall"
|
||||
)
|
||||
|
||||
// OpenDir opens a directory in windows with write access for syncing.
|
||||
func OpenDir(path string) (*os.File, error) {
|
||||
fd, err := openDir(path)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return os.NewFile(uintptr(fd), path), nil
|
||||
}
|
||||
|
||||
func openDir(path string) (fd syscall.Handle, err error) {
|
||||
if len(path) == 0 {
|
||||
return syscall.InvalidHandle, syscall.ERROR_FILE_NOT_FOUND
|
||||
}
|
||||
pathp, err := syscall.UTF16PtrFromString(path)
|
||||
if err != nil {
|
||||
return syscall.InvalidHandle, err
|
||||
}
|
||||
access := uint32(syscall.GENERIC_READ | syscall.GENERIC_WRITE)
|
||||
sharemode := uint32(syscall.FILE_SHARE_READ | syscall.FILE_SHARE_WRITE)
|
||||
createmode := uint32(syscall.OPEN_EXISTING)
|
||||
fl := uint32(syscall.FILE_FLAG_BACKUP_SEMANTICS)
|
||||
return syscall.CreateFile(pathp, access, sharemode, nil, createmode, fl, 0)
|
||||
}
|
@ -96,3 +96,26 @@ func Exist(name string) bool {
|
||||
_, err := os.Stat(name)
|
||||
return err == nil
|
||||
}
|
||||
|
||||
// ZeroToEnd zeros a file starting from SEEK_CUR to its SEEK_END. May temporarily
|
||||
// shorten the length of the file.
|
||||
func ZeroToEnd(f *os.File) error {
|
||||
// TODO: support FALLOC_FL_ZERO_RANGE
|
||||
off, err := f.Seek(0, os.SEEK_CUR)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
lenf, lerr := f.Seek(0, os.SEEK_END)
|
||||
if lerr != nil {
|
||||
return lerr
|
||||
}
|
||||
if err = f.Truncate(off); err != nil {
|
||||
return err
|
||||
}
|
||||
// make sure blocks remain allocated
|
||||
if err = Preallocate(f, lenf, true); err != nil {
|
||||
return err
|
||||
}
|
||||
_, err = f.Seek(off, os.SEEK_SET)
|
||||
return err
|
||||
}
|
||||
|
@ -118,3 +118,42 @@ func TestExist(t *testing.T) {
|
||||
t.Errorf("exist = %v, want false", g)
|
||||
}
|
||||
}
|
||||
|
||||
func TestZeroToEnd(t *testing.T) {
|
||||
f, err := ioutil.TempFile(os.TempDir(), "fileutil")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer f.Close()
|
||||
|
||||
b := make([]byte, 1024)
|
||||
for i := range b {
|
||||
b[i] = 12
|
||||
}
|
||||
if _, err = f.Write(b); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if _, err = f.Seek(512, os.SEEK_SET); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err = ZeroToEnd(f); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
off, serr := f.Seek(0, os.SEEK_CUR)
|
||||
if serr != nil {
|
||||
t.Fatal(serr)
|
||||
}
|
||||
if off != 512 {
|
||||
t.Fatalf("expected offset 512, got %d", off)
|
||||
}
|
||||
|
||||
b = make([]byte, 512)
|
||||
if _, err = f.Read(b); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
for i := range b {
|
||||
if b[i] != 0 {
|
||||
t.Errorf("expected b[%d] = 0, got %d", i, b[i])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
106
pkg/ioutil/pagewriter.go
Normal file
106
pkg/ioutil/pagewriter.go
Normal file
@ -0,0 +1,106 @@
|
||||
// Copyright 2016 The etcd Authors
|
||||
//
|
||||
// Licensed 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 ioutil
|
||||
|
||||
import (
|
||||
"io"
|
||||
)
|
||||
|
||||
var defaultBufferBytes = 128 * 1024
|
||||
|
||||
// PageWriter implements the io.Writer interface so that writes will
|
||||
// either be in page chunks or from flushing.
|
||||
type PageWriter struct {
|
||||
w io.Writer
|
||||
// pageOffset tracks the page offset of the base of the buffer
|
||||
pageOffset int
|
||||
// pageBytes is the number of bytes per page
|
||||
pageBytes int
|
||||
// bufferedBytes counts the number of bytes pending for write in the buffer
|
||||
bufferedBytes int
|
||||
// buf holds the write buffer
|
||||
buf []byte
|
||||
// bufWatermarkBytes is the number of bytes the buffer can hold before it needs
|
||||
// to be flushed. It is less than len(buf) so there is space for slack writes
|
||||
// to bring the writer to page alignment.
|
||||
bufWatermarkBytes int
|
||||
}
|
||||
|
||||
// NewPageWriter creates a new PageWriter. pageBytes is the number of bytes
|
||||
// to write per page. pageOffset is the starting offset of io.Writer.
|
||||
func NewPageWriter(w io.Writer, pageBytes, pageOffset int) *PageWriter {
|
||||
return &PageWriter{
|
||||
w: w,
|
||||
pageOffset: pageOffset,
|
||||
pageBytes: pageBytes,
|
||||
buf: make([]byte, defaultBufferBytes+pageBytes),
|
||||
bufWatermarkBytes: defaultBufferBytes,
|
||||
}
|
||||
}
|
||||
|
||||
func (pw *PageWriter) Write(p []byte) (n int, err error) {
|
||||
if len(p)+pw.bufferedBytes <= pw.bufWatermarkBytes {
|
||||
// no overflow
|
||||
copy(pw.buf[pw.bufferedBytes:], p)
|
||||
pw.bufferedBytes += len(p)
|
||||
return len(p), nil
|
||||
}
|
||||
// complete the slack page in the buffer if unaligned
|
||||
slack := pw.pageBytes - ((pw.pageOffset + pw.bufferedBytes) % pw.pageBytes)
|
||||
if slack != pw.pageBytes {
|
||||
partial := slack > len(p)
|
||||
if partial {
|
||||
// not enough data to complete the slack page
|
||||
slack = len(p)
|
||||
}
|
||||
// special case: writing to slack page in buffer
|
||||
copy(pw.buf[pw.bufferedBytes:], p[:slack])
|
||||
pw.bufferedBytes += slack
|
||||
n = slack
|
||||
p = p[slack:]
|
||||
if partial {
|
||||
// avoid forcing an unaligned flush
|
||||
return n, nil
|
||||
}
|
||||
}
|
||||
// buffer contents are now page-aligned; clear out
|
||||
if err = pw.Flush(); err != nil {
|
||||
return n, err
|
||||
}
|
||||
// directly write all complete pages without copying
|
||||
if len(p) > pw.pageBytes {
|
||||
pages := len(p) / pw.pageBytes
|
||||
c, werr := pw.w.Write(p[:pages*pw.pageBytes])
|
||||
n += c
|
||||
if werr != nil {
|
||||
return n, werr
|
||||
}
|
||||
p = p[pages*pw.pageBytes:]
|
||||
}
|
||||
// write remaining tail to buffer
|
||||
c, werr := pw.Write(p)
|
||||
n += c
|
||||
return n, werr
|
||||
}
|
||||
|
||||
func (pw *PageWriter) Flush() error {
|
||||
if pw.bufferedBytes == 0 {
|
||||
return nil
|
||||
}
|
||||
_, err := pw.w.Write(pw.buf[:pw.bufferedBytes])
|
||||
pw.pageOffset = (pw.pageOffset + pw.bufferedBytes) % pw.pageBytes
|
||||
pw.bufferedBytes = 0
|
||||
return err
|
||||
}
|
129
pkg/ioutil/pagewriter_test.go
Normal file
129
pkg/ioutil/pagewriter_test.go
Normal file
@ -0,0 +1,129 @@
|
||||
// Copyright 2016 The etcd Authors
|
||||
//
|
||||
// Licensed 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 ioutil
|
||||
|
||||
import (
|
||||
"math/rand"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestPageWriterRandom(t *testing.T) {
|
||||
// smaller buffer for stress testing
|
||||
defaultBufferBytes = 8 * 1024
|
||||
pageBytes := 128
|
||||
buf := make([]byte, 4*defaultBufferBytes)
|
||||
cw := &checkPageWriter{pageBytes: pageBytes, t: t}
|
||||
w := NewPageWriter(cw, pageBytes, 0)
|
||||
n := 0
|
||||
for i := 0; i < 4096; i++ {
|
||||
c, err := w.Write(buf[:rand.Intn(len(buf))])
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
n += c
|
||||
}
|
||||
if cw.writeBytes > n {
|
||||
t.Fatalf("wrote %d bytes to io.Writer, but only wrote %d bytes", cw.writeBytes, n)
|
||||
}
|
||||
if cw.writeBytes-n > pageBytes {
|
||||
t.Fatalf("got %d bytes pending, expected less than %d bytes", cw.writeBytes-n, pageBytes)
|
||||
}
|
||||
t.Logf("total writes: %d", cw.writes)
|
||||
t.Logf("total write bytes: %d (of %d)", cw.writeBytes, n)
|
||||
}
|
||||
|
||||
// TestPageWriterPariallack tests the case where a write overflows the buffer
|
||||
// but there is not enough data to complete the slack write.
|
||||
func TestPageWriterPartialSlack(t *testing.T) {
|
||||
defaultBufferBytes = 1024
|
||||
pageBytes := 128
|
||||
buf := make([]byte, defaultBufferBytes)
|
||||
cw := &checkPageWriter{pageBytes: 64, t: t}
|
||||
w := NewPageWriter(cw, pageBytes, 0)
|
||||
// put writer in non-zero page offset
|
||||
if _, err := w.Write(buf[:64]); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err := w.Flush(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if cw.writes != 1 {
|
||||
t.Fatalf("got %d writes, expected 1", cw.writes)
|
||||
}
|
||||
// nearly fill buffer
|
||||
if _, err := w.Write(buf[:1022]); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
// overflow buffer, but without enough to write as aligned
|
||||
if _, err := w.Write(buf[:8]); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if cw.writes != 1 {
|
||||
t.Fatalf("got %d writes, expected 1", cw.writes)
|
||||
}
|
||||
// finish writing slack space
|
||||
if _, err := w.Write(buf[:128]); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if cw.writes != 2 {
|
||||
t.Fatalf("got %d writes, expected 2", cw.writes)
|
||||
}
|
||||
}
|
||||
|
||||
// TestPageWriterOffset tests if page writer correctly repositions when offset is given.
|
||||
func TestPageWriterOffset(t *testing.T) {
|
||||
defaultBufferBytes = 1024
|
||||
pageBytes := 128
|
||||
buf := make([]byte, defaultBufferBytes)
|
||||
cw := &checkPageWriter{pageBytes: 64, t: t}
|
||||
w := NewPageWriter(cw, pageBytes, 0)
|
||||
if _, err := w.Write(buf[:64]); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err := w.Flush(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if w.pageOffset != 64 {
|
||||
t.Fatalf("w.pageOffset expected 64, got %d", w.pageOffset)
|
||||
}
|
||||
|
||||
w = NewPageWriter(cw, w.pageOffset, pageBytes)
|
||||
if _, err := w.Write(buf[:64]); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err := w.Flush(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if w.pageOffset != 0 {
|
||||
t.Fatalf("w.pageOffset expected 0, got %d", w.pageOffset)
|
||||
}
|
||||
}
|
||||
|
||||
// checkPageWriter implements an io.Writer that fails a test on unaligned writes.
|
||||
type checkPageWriter struct {
|
||||
pageBytes int
|
||||
writes int
|
||||
writeBytes int
|
||||
t *testing.T
|
||||
}
|
||||
|
||||
func (cw *checkPageWriter) Write(p []byte) (int, error) {
|
||||
if len(p)%cw.pageBytes != 0 {
|
||||
cw.t.Fatalf("got write len(p) = %d, expected len(p) == k*cw.pageBytes", len(p))
|
||||
}
|
||||
cw.writes++
|
||||
cw.writeBytes += len(p)
|
||||
return len(p), nil
|
||||
}
|
@ -64,6 +64,9 @@ type TLSInfo struct {
|
||||
TrustedCAFile string
|
||||
ClientCertAuth bool
|
||||
|
||||
// ServerName ensures the cert matches the given host in case of discovery / virtual hosting
|
||||
ServerName string
|
||||
|
||||
selfCert bool
|
||||
|
||||
// parseFunc exists to simplify testing. Typically, parseFunc
|
||||
@ -164,6 +167,7 @@ func (info TLSInfo) baseConfig() (*tls.Config, error) {
|
||||
cfg := &tls.Config{
|
||||
Certificates: []tls.Certificate{*tlsCert},
|
||||
MinVersion: tls.VersionTLS12,
|
||||
ServerName: info.ServerName,
|
||||
}
|
||||
return cfg, nil
|
||||
}
|
||||
@ -215,7 +219,7 @@ func (info TLSInfo) ClientConfig() (*tls.Config, error) {
|
||||
return nil, err
|
||||
}
|
||||
} else {
|
||||
cfg = &tls.Config{}
|
||||
cfg = &tls.Config{ServerName: info.ServerName}
|
||||
}
|
||||
|
||||
CAFiles := info.cafiles()
|
||||
@ -224,6 +228,8 @@ func (info TLSInfo) ClientConfig() (*tls.Config, error) {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// if given a CA, trust any host with a cert signed by the CA
|
||||
cfg.ServerName = ""
|
||||
}
|
||||
|
||||
if info.selfCert {
|
||||
|
49
pkg/transport/tls.go
Normal file
49
pkg/transport/tls.go
Normal file
@ -0,0 +1,49 @@
|
||||
// Copyright 2016 The etcd Authors
|
||||
//
|
||||
// Licensed 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 transport
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
||||
// ValidateSecureEndpoints scans the given endpoints against tls info, returning only those
|
||||
// endpoints that could be validated as secure.
|
||||
func ValidateSecureEndpoints(tlsInfo TLSInfo, eps []string) ([]string, error) {
|
||||
t, err := NewTransport(tlsInfo, 5*time.Second)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var errs []string
|
||||
var endpoints []string
|
||||
for _, ep := range eps {
|
||||
if !strings.HasPrefix(ep, "https://") {
|
||||
errs = append(errs, fmt.Sprintf("%q is insecure", ep))
|
||||
continue
|
||||
}
|
||||
conn, cerr := t.Dial("tcp", ep[len("https://"):])
|
||||
if cerr != nil {
|
||||
errs = append(errs, fmt.Sprintf("%q failed to dial (%v)", ep, cerr))
|
||||
continue
|
||||
}
|
||||
conn.Close()
|
||||
endpoints = append(endpoints, ep)
|
||||
}
|
||||
if len(errs) != 0 {
|
||||
err = fmt.Errorf("%s", strings.Join(errs, ","))
|
||||
}
|
||||
return endpoints, err
|
||||
}
|
@ -25,9 +25,9 @@ import (
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
|
||||
math "math"
|
||||
)
|
||||
|
||||
import io "io"
|
||||
io "io"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
|
@ -49,6 +49,7 @@ var (
|
||||
"2.1.0": {streamTypeMsgAppV2, streamTypeMessage},
|
||||
"2.2.0": {streamTypeMsgAppV2, streamTypeMessage},
|
||||
"2.3.0": {streamTypeMsgAppV2, streamTypeMessage},
|
||||
"3.0.0": {streamTypeMsgAppV2, streamTypeMessage},
|
||||
}
|
||||
)
|
||||
|
||||
@ -332,7 +333,16 @@ func (cr *streamReader) decodeLoop(rc io.ReadCloser, t streamType) error {
|
||||
default:
|
||||
plog.Panicf("unhandled stream type %s", t)
|
||||
}
|
||||
cr.closer = rc
|
||||
select {
|
||||
case <-cr.stopc:
|
||||
cr.mu.Unlock()
|
||||
if err := rc.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
return io.EOF
|
||||
default:
|
||||
cr.closer = rc
|
||||
}
|
||||
cr.mu.Unlock()
|
||||
|
||||
for {
|
||||
|
@ -17,6 +17,7 @@ package rafthttp
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"reflect"
|
||||
@ -180,6 +181,60 @@ func TestStreamReaderDialResult(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// TestStreamReaderStopOnDial tests a stream reader closes the connection on stop.
|
||||
func TestStreamReaderStopOnDial(t *testing.T) {
|
||||
defer testutil.AfterTest(t)
|
||||
h := http.Header{}
|
||||
h.Add("X-Server-Version", version.Version)
|
||||
tr := &respWaitRoundTripper{rrt: &respRoundTripper{code: http.StatusOK, header: h}}
|
||||
sr := &streamReader{
|
||||
peerID: types.ID(2),
|
||||
tr: &Transport{streamRt: tr, ClusterID: types.ID(1)},
|
||||
picker: mustNewURLPicker(t, []string{"http://localhost:2380"}),
|
||||
errorc: make(chan error, 1),
|
||||
typ: streamTypeMessage,
|
||||
status: newPeerStatus(types.ID(2)),
|
||||
}
|
||||
tr.onResp = func() {
|
||||
// stop() waits for the run() goroutine to exit, but that exit
|
||||
// needs a response from RoundTrip() first; use goroutine
|
||||
go sr.stop()
|
||||
// wait so that stop() is blocked on run() exiting
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
// sr.run() completes dialing then begins decoding while stopped
|
||||
}
|
||||
sr.start()
|
||||
select {
|
||||
case <-sr.done:
|
||||
case <-time.After(time.Second):
|
||||
t.Fatal("streamReader did not stop in time")
|
||||
}
|
||||
}
|
||||
|
||||
type respWaitRoundTripper struct {
|
||||
rrt *respRoundTripper
|
||||
onResp func()
|
||||
}
|
||||
|
||||
func (t *respWaitRoundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
|
||||
resp, err := t.rrt.RoundTrip(req)
|
||||
resp.Body = newWaitReadCloser()
|
||||
t.onResp()
|
||||
return resp, err
|
||||
}
|
||||
|
||||
type waitReadCloser struct{ closec chan struct{} }
|
||||
|
||||
func newWaitReadCloser() *waitReadCloser { return &waitReadCloser{make(chan struct{})} }
|
||||
func (wrc *waitReadCloser) Read(p []byte) (int, error) {
|
||||
<-wrc.closec
|
||||
return 0, io.EOF
|
||||
}
|
||||
func (wrc *waitReadCloser) Close() error {
|
||||
close(wrc.closec)
|
||||
return nil
|
||||
}
|
||||
|
||||
// TestStreamReaderDialDetectUnsupport tests that dial func could find
|
||||
// out that the stream type is not supported by the remote.
|
||||
func TestStreamReaderDialDetectUnsupport(t *testing.T) {
|
||||
|
@ -19,9 +19,9 @@ import (
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
|
||||
math "math"
|
||||
)
|
||||
|
||||
import io "io"
|
||||
io "io"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
|
@ -29,7 +29,7 @@ import (
|
||||
var (
|
||||
// MinClusterVersion is the min cluster version this etcd binary is compatible with.
|
||||
MinClusterVersion = "2.3.0"
|
||||
Version = "3.0.4"
|
||||
Version = "3.0.12"
|
||||
|
||||
// Git SHA Value will be set during build
|
||||
GitSHA = "Not provided (use ./build instead of go build)"
|
||||
|
@ -15,28 +15,34 @@
|
||||
package wal
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"encoding/binary"
|
||||
"hash"
|
||||
"io"
|
||||
"os"
|
||||
"sync"
|
||||
|
||||
"github.com/coreos/etcd/pkg/crc"
|
||||
"github.com/coreos/etcd/pkg/ioutil"
|
||||
"github.com/coreos/etcd/wal/walpb"
|
||||
)
|
||||
|
||||
// walPageBytes is the alignment for flushing records to the backing Writer.
|
||||
// It should be a multiple of the minimum sector size so that WAL repair can
|
||||
// safely between torn writes and ordinary data corruption.
|
||||
const walPageBytes = 8 * minSectorSize
|
||||
|
||||
type encoder struct {
|
||||
mu sync.Mutex
|
||||
bw *bufio.Writer
|
||||
bw *ioutil.PageWriter
|
||||
|
||||
crc hash.Hash32
|
||||
buf []byte
|
||||
uint64buf []byte
|
||||
}
|
||||
|
||||
func newEncoder(w io.Writer, prevCrc uint32) *encoder {
|
||||
func newEncoder(w io.Writer, prevCrc uint32, pageOffset int) *encoder {
|
||||
return &encoder{
|
||||
bw: bufio.NewWriter(w),
|
||||
bw: ioutil.NewPageWriter(w, walPageBytes, pageOffset),
|
||||
crc: crc.New(prevCrc, crcTable),
|
||||
// 1MB buffer
|
||||
buf: make([]byte, 1024*1024),
|
||||
@ -44,6 +50,15 @@ func newEncoder(w io.Writer, prevCrc uint32) *encoder {
|
||||
}
|
||||
}
|
||||
|
||||
// newFileEncoder creates a new encoder with current file offset for the page writer.
|
||||
func newFileEncoder(f *os.File, prevCrc uint32) (*encoder, error) {
|
||||
offset, err := f.Seek(0, os.SEEK_CUR)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return newEncoder(f, prevCrc, int(offset)), nil
|
||||
}
|
||||
|
||||
func (e *encoder) encode(rec *walpb.Record) error {
|
||||
e.mu.Lock()
|
||||
defer e.mu.Unlock()
|
||||
|
@ -69,7 +69,7 @@ func TestWriteRecord(t *testing.T) {
|
||||
typ := int64(0xABCD)
|
||||
d := []byte("Hello world!")
|
||||
buf := new(bytes.Buffer)
|
||||
e := newEncoder(buf, 0)
|
||||
e := newEncoder(buf, 0, 0)
|
||||
e.encode(&walpb.Record{Type: typ, Data: d})
|
||||
e.flush()
|
||||
decoder := newDecoder(ioutil.NopCloser(buf))
|
||||
|
89
wal/wal.go
89
wal/wal.go
@ -67,7 +67,11 @@ var (
|
||||
// A just opened WAL is in read mode, and ready for reading records.
|
||||
// The WAL will be ready for appending after reading out all the previous records.
|
||||
type WAL struct {
|
||||
dir string // the living directory of the underlay files
|
||||
dir string // the living directory of the underlay files
|
||||
|
||||
// dirFile is a fd for the wal directory for syncing on Rename
|
||||
dirFile *os.File
|
||||
|
||||
metadata []byte // metadata recorded at the head of each WAL
|
||||
state raftpb.HardState // hardstate recorded at the head of WAL
|
||||
|
||||
@ -106,45 +110,49 @@ func Create(dirpath string, metadata []byte) (*WAL, error) {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if _, err := f.Seek(0, os.SEEK_END); err != nil {
|
||||
if _, err = f.Seek(0, os.SEEK_END); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := fileutil.Preallocate(f.File, segmentSizeBytes, true); err != nil {
|
||||
if err = fileutil.Preallocate(f.File, segmentSizeBytes, true); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
w := &WAL{
|
||||
dir: dirpath,
|
||||
metadata: metadata,
|
||||
encoder: newEncoder(f, 0),
|
||||
}
|
||||
w.encoder, err = newFileEncoder(f.File, 0)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
w.locks = append(w.locks, f)
|
||||
if err := w.saveCrc(0); err != nil {
|
||||
if err = w.saveCrc(0); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := w.encoder.encode(&walpb.Record{Type: metadataType, Data: metadata}); err != nil {
|
||||
if err = w.encoder.encode(&walpb.Record{Type: metadataType, Data: metadata}); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := w.SaveSnapshot(walpb.Snapshot{}); err != nil {
|
||||
if err = w.SaveSnapshot(walpb.Snapshot{}); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// rename of directory with locked files doesn't work on windows; close
|
||||
// the WAL to release the locks so the directory can be renamed
|
||||
w.Close()
|
||||
if err := os.Rename(tmpdirpath, dirpath); err != nil {
|
||||
if w, err = w.renameWal(tmpdirpath); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// reopen and relock
|
||||
newWAL, oerr := Open(dirpath, walpb.Snapshot{})
|
||||
if oerr != nil {
|
||||
return nil, oerr
|
||||
|
||||
// directory was renamed; sync parent dir to persist rename
|
||||
pdir, perr := fileutil.OpenDir(path.Dir(w.dir))
|
||||
if perr != nil {
|
||||
return nil, perr
|
||||
}
|
||||
if _, _, _, err := newWAL.ReadAll(); err != nil {
|
||||
newWAL.Close()
|
||||
return nil, err
|
||||
if perr = fileutil.Fsync(pdir); perr != nil {
|
||||
return nil, perr
|
||||
}
|
||||
return newWAL, nil
|
||||
if perr = pdir.Close(); err != nil {
|
||||
return nil, perr
|
||||
}
|
||||
|
||||
return w, nil
|
||||
}
|
||||
|
||||
// Open opens the WAL at the given snap.
|
||||
@ -154,7 +162,14 @@ func Create(dirpath string, metadata []byte) (*WAL, error) {
|
||||
// the given snap. The WAL cannot be appended to before reading out all of its
|
||||
// previous records.
|
||||
func Open(dirpath string, snap walpb.Snapshot) (*WAL, error) {
|
||||
return openAtIndex(dirpath, snap, true)
|
||||
w, err := openAtIndex(dirpath, snap, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if w.dirFile, err = fileutil.OpenDir(w.dir); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return w, nil
|
||||
}
|
||||
|
||||
// OpenForRead only opens the wal files for read.
|
||||
@ -299,6 +314,18 @@ func (w *WAL) ReadAll() (metadata []byte, state raftpb.HardState, ents []raftpb.
|
||||
state.Reset()
|
||||
return nil, state, nil, err
|
||||
}
|
||||
// decodeRecord() will return io.EOF if it detects a zero record,
|
||||
// but this zero record may be followed by non-zero records from
|
||||
// a torn write. Overwriting some of these non-zero records, but
|
||||
// not all, will cause CRC errors on WAL open. Since the records
|
||||
// were never fully synced to disk in the first place, it's safe
|
||||
// to zero them out to avoid any CRC errors from new writes.
|
||||
if _, err = w.tail().Seek(w.decoder.lastOffset(), os.SEEK_SET); err != nil {
|
||||
return nil, state, nil, err
|
||||
}
|
||||
if err = fileutil.ZeroToEnd(w.tail().File); err != nil {
|
||||
return nil, state, nil, err
|
||||
}
|
||||
}
|
||||
|
||||
err = nil
|
||||
@ -317,8 +344,10 @@ func (w *WAL) ReadAll() (metadata []byte, state raftpb.HardState, ents []raftpb.
|
||||
|
||||
if w.tail() != nil {
|
||||
// create encoder (chain crc with the decoder), enable appending
|
||||
_, err = w.tail().Seek(w.decoder.lastOffset(), os.SEEK_SET)
|
||||
w.encoder = newEncoder(w.tail(), w.decoder.lastCRC())
|
||||
w.encoder, err = newFileEncoder(w.tail().File, w.decoder.lastCRC())
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
w.decoder = nil
|
||||
|
||||
@ -352,7 +381,10 @@ func (w *WAL) cut() error {
|
||||
// update writer and save the previous crc
|
||||
w.locks = append(w.locks, newTail)
|
||||
prevCrc := w.encoder.crc.Sum32()
|
||||
w.encoder = newEncoder(w.tail(), prevCrc)
|
||||
w.encoder, err = newFileEncoder(w.tail().File, prevCrc)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err = w.saveCrc(prevCrc); err != nil {
|
||||
return err
|
||||
}
|
||||
@ -375,6 +407,10 @@ func (w *WAL) cut() error {
|
||||
if err = os.Rename(newTail.Name(), fpath); err != nil {
|
||||
return err
|
||||
}
|
||||
if err = fileutil.Fsync(w.dirFile); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
newTail.Close()
|
||||
|
||||
if newTail, err = fileutil.LockFile(fpath, os.O_WRONLY, fileutil.PrivateFileMode); err != nil {
|
||||
@ -387,7 +423,10 @@ func (w *WAL) cut() error {
|
||||
w.locks[len(w.locks)-1] = newTail
|
||||
|
||||
prevCrc = w.encoder.crc.Sum32()
|
||||
w.encoder = newEncoder(w.tail(), prevCrc)
|
||||
w.encoder, err = newFileEncoder(w.tail().File, prevCrc)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
plog.Infof("segmented wal file %v is created", fpath)
|
||||
return nil
|
||||
@ -477,7 +516,7 @@ func (w *WAL) Close() error {
|
||||
plog.Errorf("failed to unlock during closing wal: %s", err)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
return w.dirFile.Close()
|
||||
}
|
||||
|
||||
func (w *WAL) saveEntry(e *raftpb.Entry) error {
|
||||
|
@ -61,7 +61,7 @@ func TestNew(t *testing.T) {
|
||||
}
|
||||
|
||||
var wb bytes.Buffer
|
||||
e := newEncoder(&wb, 0)
|
||||
e := newEncoder(&wb, 0, 0)
|
||||
err = e.encode(&walpb.Record{Type: crcType, Crc: 0})
|
||||
if err != nil {
|
||||
t.Fatalf("err = %v, want nil", err)
|
||||
@ -465,7 +465,7 @@ func TestSaveEmpty(t *testing.T) {
|
||||
var buf bytes.Buffer
|
||||
var est raftpb.HardState
|
||||
w := WAL{
|
||||
encoder: newEncoder(&buf, 0),
|
||||
encoder: newEncoder(&buf, 0, 0),
|
||||
}
|
||||
if err := w.saveState(&est); err != nil {
|
||||
t.Errorf("err = %v, want nil", err)
|
||||
@ -636,3 +636,89 @@ func TestRestartCreateWal(t *testing.T) {
|
||||
t.Fatalf("got error %v and meta %q, expected nil and %q", rerr, meta, "abc")
|
||||
}
|
||||
}
|
||||
|
||||
// TestOpenOnTornWrite ensures that entries past the torn write are truncated.
|
||||
func TestOpenOnTornWrite(t *testing.T) {
|
||||
maxEntries := 40
|
||||
clobberIdx := 20
|
||||
overwriteEntries := 5
|
||||
|
||||
p, err := ioutil.TempDir(os.TempDir(), "waltest")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer os.RemoveAll(p)
|
||||
w, err := Create(p, nil)
|
||||
defer w.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// get offset of end of each saved entry
|
||||
offsets := make([]int64, maxEntries)
|
||||
for i := range offsets {
|
||||
es := []raftpb.Entry{{Index: uint64(i)}}
|
||||
if err = w.Save(raftpb.HardState{}, es); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if offsets[i], err = w.tail().Seek(0, os.SEEK_CUR); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
fn := path.Join(p, path.Base(w.tail().Name()))
|
||||
w.Close()
|
||||
|
||||
// clobber some entry with 0's to simulate a torn write
|
||||
f, ferr := os.OpenFile(fn, os.O_WRONLY, fileutil.PrivateFileMode)
|
||||
if ferr != nil {
|
||||
t.Fatal(ferr)
|
||||
}
|
||||
defer f.Close()
|
||||
_, err = f.Seek(offsets[clobberIdx], os.SEEK_SET)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
zeros := make([]byte, offsets[clobberIdx+1]-offsets[clobberIdx])
|
||||
_, err = f.Write(zeros)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
f.Close()
|
||||
|
||||
w, err = Open(p, walpb.Snapshot{})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
// seek up to clobbered entry
|
||||
_, _, _, err = w.ReadAll()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// write a few entries past the clobbered entry
|
||||
for i := 0; i < overwriteEntries; i++ {
|
||||
// Index is different from old, truncated entries
|
||||
es := []raftpb.Entry{{Index: uint64(i + clobberIdx), Data: []byte("new")}}
|
||||
if err = w.Save(raftpb.HardState{}, es); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
w.Close()
|
||||
|
||||
// read back the entries, confirm number of entries matches expectation
|
||||
w, err = OpenForRead(p, walpb.Snapshot{})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
_, _, ents, rerr := w.ReadAll()
|
||||
if rerr != nil {
|
||||
// CRC error? the old entries were likely never truncated away
|
||||
t.Fatal(rerr)
|
||||
}
|
||||
wEntries := (clobberIdx - 1) + overwriteEntries
|
||||
if len(ents) != wEntries {
|
||||
t.Fatalf("expected len(ents) = %d, got %d", wEntries, len(ents))
|
||||
}
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user