Compare commits
89 Commits
v3.4.0-rc.
...
v3.4.5
Author | SHA1 | Date | |
---|---|---|---|
![]() |
e784ba73c2 | ||
![]() |
35dc623a98 | ||
![]() |
130342152a | ||
![]() |
fc93fbf9de | ||
![]() |
0dc5d577fc | ||
![]() |
e63db56cc9 | ||
![]() |
1471e12108 | ||
![]() |
0b9cfa8677 | ||
![]() |
b66c53ff5f | ||
![]() |
8f6c3f4d09 | ||
![]() |
379d01a0d2 | ||
![]() |
c65a9e2dd1 | ||
![]() |
7862f6ed2c | ||
![]() |
257319fb18 | ||
![]() |
cdb2dc11b8 | ||
![]() |
770674e4a6 | ||
![]() |
c10168f718 | ||
![]() |
94673a6ba4 | ||
![]() |
a1bf5574fc | ||
![]() |
6d646c442a | ||
![]() |
1226686cf3 | ||
![]() |
50e12328ac | ||
![]() |
0dc78a144b | ||
![]() |
7cf32c262c | ||
![]() |
4a9247a47e | ||
![]() |
ac63c2fbd0 | ||
![]() |
ae5bd3c268 | ||
![]() |
94e46ba0d7 | ||
![]() |
8c10973820 | ||
![]() |
1af0b51537 | ||
![]() |
f4669c3b62 | ||
![]() |
55c3476abc | ||
![]() |
b66203c0a1 | ||
![]() |
4388404f56 | ||
![]() |
a447d51f23 | ||
![]() |
4f3c81d81d | ||
![]() |
478da3bf24 | ||
![]() |
d6b30e43cd | ||
![]() |
1e98c9642e | ||
![]() |
3cf2f69b57 | ||
![]() |
d617055284 | ||
![]() |
84db9b0878 | ||
![]() |
6cf418ff6d | ||
![]() |
97e68cf4e7 | ||
![]() |
90556d550d | ||
![]() |
a00abf5f2a | ||
![]() |
b3329ebcd2 | ||
![]() |
b67862c0a6 | ||
![]() |
6a699b6b7f | ||
![]() |
bb5ba14aac | ||
![]() |
c3dc994567 | ||
![]() |
f3fbed5b72 | ||
![]() |
e2547907c5 | ||
![]() |
14c239030f | ||
![]() |
7b67e8a5c5 | ||
![]() |
bbe86b066c | ||
![]() |
2c36cab87d | ||
![]() |
480d5510f9 | ||
![]() |
9245518363 | ||
![]() |
daa432cfa7 | ||
![]() |
8717327697 | ||
![]() |
4f1bbff888 | ||
![]() |
28bb8037d9 | ||
![]() |
03b5e7229b | ||
![]() |
0781c0327d | ||
![]() |
99774d8ed4 | ||
![]() |
c454344f14 | ||
![]() |
dae0a72a42 | ||
![]() |
c91a6bf14f | ||
![]() |
b7ff97f54e | ||
![]() |
d08bb07d6d | ||
![]() |
3a736a81e8 | ||
![]() |
a14579fbfb | ||
![]() |
ade66a5722 | ||
![]() |
67cc70926d | ||
![]() |
21dcadc83c | ||
![]() |
c7c379e52e | ||
![]() |
9ed5f76dc0 | ||
![]() |
994865c89e | ||
![]() |
ccbbb2f8d6 | ||
![]() |
d5f79adc9c | ||
![]() |
8b053b0f44 | ||
![]() |
11980f8165 | ||
![]() |
41d4e2b276 | ||
![]() |
898bd1351f | ||
![]() |
d04d96c9ac | ||
![]() |
21edf98fdb | ||
![]() |
a4f7c65ef8 | ||
![]() |
c3a9eec843 |
@@ -6,7 +6,7 @@ sudo: required
|
||||
services: docker
|
||||
|
||||
go:
|
||||
- 1.12.9
|
||||
- 1.12.12
|
||||
|
||||
notifications:
|
||||
on_success: never
|
||||
@@ -27,9 +27,9 @@ env:
|
||||
matrix:
|
||||
fast_finish: true
|
||||
allow_failures:
|
||||
- go: 1.12.9
|
||||
- go: 1.12.12
|
||||
env: TARGET=linux-amd64-grpcproxy
|
||||
- go: 1.12.9
|
||||
- go: 1.12.12
|
||||
env: TARGET=linux-386-unit
|
||||
|
||||
before_install:
|
||||
|
3
.words
3
.words
@@ -36,6 +36,8 @@ iff
|
||||
inflight
|
||||
keepalive
|
||||
keepalives
|
||||
hasleader
|
||||
racey
|
||||
keyspace
|
||||
linearization
|
||||
liveness
|
||||
@@ -95,6 +97,7 @@ jitter
|
||||
WithBackoff
|
||||
BackoffLinearWithJitter
|
||||
jitter
|
||||
WithDialer
|
||||
WithMax
|
||||
ServerStreams
|
||||
BidiStreams
|
||||
|
@@ -72,7 +72,7 @@ etcdctl provides a `snapshot` command to create backups. See [backup][backup] fo
|
||||
|
||||
When replacing an etcd node, it's important to remove the member first and then add its replacement.
|
||||
|
||||
etcd employs distributed consensus based on a quorum model; (n+1)/2 members, a majority, must agree on a proposal before it can be committed to the cluster. These proposals include key-value updates and membership changes. This model totally avoids any possibility of split brain inconsistency. The downside is permanent quorum loss is catastrophic.
|
||||
etcd employs distributed consensus based on a quorum model; (n/2)+1 members, a majority, must agree on a proposal before it can be committed to the cluster. These proposals include key-value updates and membership changes. This model totally avoids any possibility of split brain inconsistency. The downside is permanent quorum loss is catastrophic.
|
||||
|
||||
How this applies to membership: If a 3-member cluster has 1 downed member, it can still make forward progress because the quorum is 2 and 2 members are still live. However, adding a new member to a 3-member cluster will increase the quorum to 3 because 3 votes are required for a majority of 4 members. Since the quorum increased, this extra member buys nothing in terms of fault tolerance; the cluster is still one node failure away from being unrecoverable.
|
||||
|
||||
|
6
Makefile
6
Makefile
@@ -51,7 +51,7 @@ docker-remove:
|
||||
|
||||
|
||||
|
||||
GO_VERSION ?= 1.12.9
|
||||
GO_VERSION ?= 1.12.12
|
||||
ETCD_VERSION ?= $(shell git rev-parse --short HEAD || echo "GitNotFound")
|
||||
|
||||
TEST_SUFFIX = $(shell date +%s | base64 | head -c 15)
|
||||
@@ -65,11 +65,11 @@ endif
|
||||
|
||||
|
||||
# Example:
|
||||
# GO_VERSION=1.12.9 make build-docker-test
|
||||
# GO_VERSION=1.12.12 make build-docker-test
|
||||
# make build-docker-test
|
||||
#
|
||||
# gcloud docker -- login -u _json_key -p "$(cat /etc/gcp-key-etcd-development.json)" https://gcr.io
|
||||
# GO_VERSION=1.12.9 make push-docker-test
|
||||
# GO_VERSION=1.12.12 make push-docker-test
|
||||
# make push-docker-test
|
||||
#
|
||||
# gsutil -m acl ch -u allUsers:R -r gs://artifacts.etcd-development.appspot.com
|
||||
|
@@ -105,7 +105,7 @@ func (t *tokenJWT) assign(ctx context.Context, username string, revision uint64)
|
||||
token, err := tk.SignedString(t.key)
|
||||
if err != nil {
|
||||
if t.lg != nil {
|
||||
t.lg.Warn(
|
||||
t.lg.Debug(
|
||||
"failed to sign a JWT token",
|
||||
zap.String("user-name", username),
|
||||
zap.Uint64("revision", revision),
|
||||
@@ -118,7 +118,7 @@ func (t *tokenJWT) assign(ctx context.Context, username string, revision uint64)
|
||||
}
|
||||
|
||||
if t.lg != nil {
|
||||
t.lg.Info(
|
||||
t.lg.Debug(
|
||||
"created/assigned a new JWT token",
|
||||
zap.String("user-name", username),
|
||||
zap.Uint64("revision", revision),
|
||||
@@ -136,7 +136,7 @@ func newTokenProviderJWT(lg *zap.Logger, optMap map[string]string) (*tokenJWT, e
|
||||
err = opts.ParseWithDefaults(optMap)
|
||||
if err != nil {
|
||||
if lg != nil {
|
||||
lg.Warn("problem loading JWT options", zap.Error(err))
|
||||
lg.Error("problem loading JWT options", zap.Error(err))
|
||||
} else {
|
||||
plog.Errorf("problem loading JWT options: %s", err)
|
||||
}
|
||||
|
@@ -306,7 +306,7 @@ func (as *authStore) Authenticate(ctx context.Context, username, password string
|
||||
return nil, ErrAuthFailed
|
||||
}
|
||||
|
||||
if user.Options.NoPassword {
|
||||
if user.Options != nil && user.Options.NoPassword {
|
||||
return nil, ErrAuthFailed
|
||||
}
|
||||
|
||||
@@ -344,7 +344,7 @@ func (as *authStore) CheckPassword(username, password string) (uint64, error) {
|
||||
return 0, ErrAuthFailed
|
||||
}
|
||||
|
||||
if user.Options.NoPassword {
|
||||
if user.Options != nil && user.Options.NoPassword {
|
||||
return 0, ErrAuthFailed
|
||||
}
|
||||
|
||||
@@ -388,7 +388,8 @@ func (as *authStore) UserAdd(r *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse,
|
||||
var hashed []byte
|
||||
var err error
|
||||
|
||||
if r.Options != nil && !r.Options.NoPassword {
|
||||
noPassword := r.Options != nil && r.Options.NoPassword
|
||||
if !noPassword {
|
||||
hashed, err = bcrypt.GenerateFromPassword([]byte(r.Password), as.bcryptCost)
|
||||
if err != nil {
|
||||
if as.lg != nil {
|
||||
|
@@ -16,7 +16,9 @@
|
||||
package endpoint
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/url"
|
||||
"strings"
|
||||
"sync"
|
||||
@@ -228,13 +230,18 @@ func ParseTarget(target string) (string, string, error) {
|
||||
return parts[0], parts[1], nil
|
||||
}
|
||||
|
||||
// ParseHostPort splits a "<host>:<port>" string into the host and port parts.
|
||||
// The port part is optional.
|
||||
func ParseHostPort(hostPort string) (host string, port string) {
|
||||
parts := strings.SplitN(hostPort, ":", 2)
|
||||
host = parts[0]
|
||||
if len(parts) > 1 {
|
||||
port = parts[1]
|
||||
// Dialer dials a endpoint using net.Dialer.
|
||||
// Context cancelation and timeout are supported.
|
||||
func Dialer(ctx context.Context, dialEp string) (net.Conn, error) {
|
||||
proto, host, _ := ParseEndpoint(dialEp)
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
default:
|
||||
}
|
||||
return host, port
|
||||
dialer := &net.Dialer{}
|
||||
if deadline, ok := ctx.Deadline(); ok {
|
||||
dialer.Deadline = deadline
|
||||
}
|
||||
return dialer.DialContext(ctx, proto, host)
|
||||
}
|
||||
|
@@ -37,7 +37,6 @@ import (
|
||||
"google.golang.org/grpc/codes"
|
||||
grpccredentials "google.golang.org/grpc/credentials"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
"google.golang.org/grpc/metadata"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
@@ -230,24 +229,17 @@ func (c *Client) dialSetupOpts(creds grpccredentials.TransportCredentials, dopts
|
||||
}
|
||||
opts = append(opts, dopts...)
|
||||
|
||||
// Provide a net dialer that supports cancelation and timeout.
|
||||
f := func(dialEp string, t time.Duration) (net.Conn, error) {
|
||||
proto, host, _ := endpoint.ParseEndpoint(dialEp)
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
return nil, c.ctx.Err()
|
||||
default:
|
||||
}
|
||||
dialer := &net.Dialer{Timeout: t}
|
||||
return dialer.DialContext(c.ctx, proto, host)
|
||||
}
|
||||
opts = append(opts, grpc.WithDialer(f))
|
||||
|
||||
dialer := endpoint.Dialer
|
||||
if creds != nil {
|
||||
opts = append(opts, grpc.WithTransportCredentials(creds))
|
||||
// gRPC load balancer workaround. See credentials.transportCredential for details.
|
||||
if credsDialer, ok := creds.(TransportCredentialsWithDialer); ok {
|
||||
dialer = credsDialer.Dialer
|
||||
}
|
||||
} else {
|
||||
opts = append(opts, grpc.WithInsecure())
|
||||
}
|
||||
opts = append(opts, grpc.WithContextDialer(dialer))
|
||||
|
||||
// Interceptor retry and backoff.
|
||||
// TODO: Replace all of clientv3/retry.go with interceptor based retry, or with
|
||||
@@ -266,7 +258,10 @@ func (c *Client) dialSetupOpts(creds grpccredentials.TransportCredentials, dopts
|
||||
|
||||
// Dial connects to a single endpoint using the client's config.
|
||||
func (c *Client) Dial(ep string) (*grpc.ClientConn, error) {
|
||||
creds := c.directDialCreds(ep)
|
||||
creds, err := c.directDialCreds(ep)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// Use the grpc passthrough resolver to directly dial a single endpoint.
|
||||
// This resolver passes through the 'unix' and 'unixs' endpoints schemes used
|
||||
// by etcd without modification, allowing us to directly dial endpoints and
|
||||
@@ -369,8 +364,8 @@ func (c *Client) dial(target string, creds grpccredentials.TransportCredentials,
|
||||
return conn, nil
|
||||
}
|
||||
|
||||
func (c *Client) directDialCreds(ep string) grpccredentials.TransportCredentials {
|
||||
_, hostPort, scheme := endpoint.ParseEndpoint(ep)
|
||||
func (c *Client) directDialCreds(ep string) (grpccredentials.TransportCredentials, error) {
|
||||
_, host, scheme := endpoint.ParseEndpoint(ep)
|
||||
creds := c.creds
|
||||
if len(scheme) != 0 {
|
||||
creds = c.processCreds(scheme)
|
||||
@@ -379,12 +374,17 @@ func (c *Client) directDialCreds(ep string) grpccredentials.TransportCredentials
|
||||
// Set the server name must to the endpoint hostname without port since grpc
|
||||
// otherwise attempts to check if x509 cert is valid for the full endpoint
|
||||
// including the scheme and port, which fails.
|
||||
host, _ := endpoint.ParseHostPort(hostPort)
|
||||
clone.OverrideServerName(host)
|
||||
overrideServerName, _, err := net.SplitHostPort(host)
|
||||
if err != nil {
|
||||
// Either the host didn't have a port or the host could not be parsed. Either way, continue with the
|
||||
// original host string.
|
||||
overrideServerName = host
|
||||
}
|
||||
clone.OverrideServerName(overrideServerName)
|
||||
creds = clone
|
||||
}
|
||||
}
|
||||
return creds
|
||||
return creds, nil
|
||||
}
|
||||
|
||||
func (c *Client) dialWithBalancerCreds(ep string) grpccredentials.TransportCredentials {
|
||||
@@ -396,13 +396,6 @@ func (c *Client) dialWithBalancerCreds(ep string) grpccredentials.TransportCrede
|
||||
return creds
|
||||
}
|
||||
|
||||
// WithRequireLeader requires client requests to only succeed
|
||||
// when the cluster has a leader.
|
||||
func WithRequireLeader(ctx context.Context) context.Context {
|
||||
md := metadata.Pairs(rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
|
||||
return metadata.NewOutgoingContext(ctx, md)
|
||||
}
|
||||
|
||||
func newClient(cfg *Config) (*Client, error) {
|
||||
if cfg == nil {
|
||||
cfg = &Config{}
|
||||
@@ -663,3 +656,9 @@ func IsConnCanceled(err error) bool {
|
||||
// <= gRPC v1.7.x returns 'errors.New("grpc: the client connection is closing")'
|
||||
return strings.Contains(err.Error(), "grpc: the client connection is closing")
|
||||
}
|
||||
|
||||
// TransportCredentialsWithDialer is for a gRPC load balancer workaround. See credentials.transportCredential for details.
|
||||
type TransportCredentialsWithDialer interface {
|
||||
grpccredentials.TransportCredentials
|
||||
Dialer(ctx context.Context, dialEp string) (net.Conn, error)
|
||||
}
|
||||
|
@@ -22,6 +22,7 @@ import (
|
||||
"net"
|
||||
"sync"
|
||||
|
||||
"go.etcd.io/etcd/clientv3/balancer/resolver/endpoint"
|
||||
"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
grpccredentials "google.golang.org/grpc/credentials"
|
||||
)
|
||||
@@ -65,38 +66,37 @@ func (b *bundle) NewWithMode(mode string) (grpccredentials.Bundle, error) {
|
||||
}
|
||||
|
||||
// transportCredential implements "grpccredentials.TransportCredentials" interface.
|
||||
// transportCredential wraps TransportCredentials to track which
|
||||
// addresses are dialed for which endpoints, and then sets the authority when checking the endpoint's cert to the
|
||||
// hostname or IP of the dialed endpoint.
|
||||
// This is a workaround of a gRPC load balancer issue. gRPC uses the dialed target's service name as the authority when
|
||||
// checking all endpoint certs, which does not work for etcd servers using their hostname or IP as the Subject Alternative Name
|
||||
// in their TLS certs.
|
||||
// To enable, include both WithTransportCredentials(creds) and WithContextDialer(creds.Dialer)
|
||||
// when dialing.
|
||||
type transportCredential struct {
|
||||
gtc grpccredentials.TransportCredentials
|
||||
mu sync.Mutex
|
||||
// addrToEndpoint maps from the connection addresses that are dialed to the hostname or IP of the
|
||||
// endpoint provided to the dialer when dialing
|
||||
addrToEndpoint map[string]string
|
||||
}
|
||||
|
||||
func newTransportCredential(cfg *tls.Config) *transportCredential {
|
||||
return &transportCredential{
|
||||
gtc: grpccredentials.NewTLS(cfg),
|
||||
gtc: grpccredentials.NewTLS(cfg),
|
||||
addrToEndpoint: map[string]string{},
|
||||
}
|
||||
}
|
||||
|
||||
func (tc *transportCredential) ClientHandshake(ctx context.Context, authority string, rawConn net.Conn) (net.Conn, grpccredentials.AuthInfo, error) {
|
||||
// Only overwrite when authority is an IP address!
|
||||
// Let's say, a server runs SRV records on "etcd.local" that resolves
|
||||
// to "m1.etcd.local", and its SAN field also includes "m1.etcd.local".
|
||||
// But what if SAN does not include its resolved IP address (e.g. 127.0.0.1)?
|
||||
// Then, the server should only authenticate using its DNS hostname "m1.etcd.local",
|
||||
// instead of overwriting it with its IP address.
|
||||
// And we do not overwrite "localhost" either. Only overwrite IP addresses!
|
||||
if isIP(authority) {
|
||||
target := rawConn.RemoteAddr().String()
|
||||
if authority != target {
|
||||
// When user dials with "grpc.WithDialer", "grpc.DialContext" "cc.parsedTarget"
|
||||
// update only happens once. This is problematic, because when TLS is enabled,
|
||||
// retries happen through "grpc.WithDialer" with static "cc.parsedTarget" from
|
||||
// the initial dial call.
|
||||
// If the server authenticates by IP addresses, we want to set a new endpoint as
|
||||
// a new authority. Otherwise
|
||||
// "transport: authentication handshake failed: x509: certificate is valid for 127.0.0.1, 192.168.121.180, not 192.168.223.156"
|
||||
// when the new dial target is "192.168.121.180" whose certificate host name is also "192.168.121.180"
|
||||
// but client tries to authenticate with previously set "cc.parsedTarget" field "192.168.223.156"
|
||||
authority = target
|
||||
}
|
||||
// Set the authority when checking the endpoint's cert to the hostname or IP of the dialed endpoint
|
||||
tc.mu.Lock()
|
||||
dialEp, ok := tc.addrToEndpoint[rawConn.RemoteAddr().String()]
|
||||
tc.mu.Unlock()
|
||||
if ok {
|
||||
_, host, _ := endpoint.ParseEndpoint(dialEp)
|
||||
authority = host
|
||||
}
|
||||
return tc.gtc.ClientHandshake(ctx, authority, rawConn)
|
||||
}
|
||||
@@ -115,8 +115,15 @@ func (tc *transportCredential) Info() grpccredentials.ProtocolInfo {
|
||||
}
|
||||
|
||||
func (tc *transportCredential) Clone() grpccredentials.TransportCredentials {
|
||||
copy := map[string]string{}
|
||||
tc.mu.Lock()
|
||||
for k, v := range tc.addrToEndpoint {
|
||||
copy[k] = v
|
||||
}
|
||||
tc.mu.Unlock()
|
||||
return &transportCredential{
|
||||
gtc: tc.gtc.Clone(),
|
||||
gtc: tc.gtc.Clone(),
|
||||
addrToEndpoint: copy,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -124,6 +131,17 @@ func (tc *transportCredential) OverrideServerName(serverNameOverride string) err
|
||||
return tc.gtc.OverrideServerName(serverNameOverride)
|
||||
}
|
||||
|
||||
func (tc *transportCredential) Dialer(ctx context.Context, dialEp string) (net.Conn, error) {
|
||||
// Keep track of which addresses are dialed for which endpoints
|
||||
conn, err := endpoint.Dialer(ctx, dialEp)
|
||||
if conn != nil {
|
||||
tc.mu.Lock()
|
||||
tc.addrToEndpoint[conn.RemoteAddr().String()] = dialEp
|
||||
tc.mu.Unlock()
|
||||
}
|
||||
return conn, err
|
||||
}
|
||||
|
||||
// perRPCCredential implements "grpccredentials.PerRPCCredentials" interface.
|
||||
type perRPCCredential struct {
|
||||
authToken string
|
||||
|
64
clientv3/ctx.go
Normal file
64
clientv3/ctx.go
Normal file
@@ -0,0 +1,64 @@
|
||||
// Copyright 2020 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 (
|
||||
"context"
|
||||
"strings"
|
||||
|
||||
"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
"go.etcd.io/etcd/version"
|
||||
"google.golang.org/grpc/metadata"
|
||||
)
|
||||
|
||||
// WithRequireLeader requires client requests to only succeed
|
||||
// when the cluster has a leader.
|
||||
func WithRequireLeader(ctx context.Context) context.Context {
|
||||
md, ok := metadata.FromOutgoingContext(ctx)
|
||||
if !ok { // no outgoing metadata ctx key, create one
|
||||
md = metadata.Pairs(rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
|
||||
return metadata.NewOutgoingContext(ctx, md)
|
||||
}
|
||||
copied := md.Copy() // avoid racey updates
|
||||
// overwrite/add 'hasleader' key/value
|
||||
metadataSet(copied, rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
|
||||
return metadata.NewOutgoingContext(ctx, copied)
|
||||
}
|
||||
|
||||
// embeds client version
|
||||
func withVersion(ctx context.Context) context.Context {
|
||||
md, ok := metadata.FromOutgoingContext(ctx)
|
||||
if !ok { // no outgoing metadata ctx key, create one
|
||||
md = metadata.Pairs(rpctypes.MetadataClientAPIVersionKey, version.APIVersion)
|
||||
return metadata.NewOutgoingContext(ctx, md)
|
||||
}
|
||||
copied := md.Copy() // avoid racey updates
|
||||
// overwrite/add version key/value
|
||||
metadataSet(copied, rpctypes.MetadataClientAPIVersionKey, version.APIVersion)
|
||||
return metadata.NewOutgoingContext(ctx, copied)
|
||||
}
|
||||
|
||||
func metadataGet(md metadata.MD, k string) []string {
|
||||
k = strings.ToLower(k)
|
||||
return md[k]
|
||||
}
|
||||
|
||||
func metadataSet(md metadata.MD, k string, vals ...string) {
|
||||
if len(vals) == 0 {
|
||||
return
|
||||
}
|
||||
k = strings.ToLower(k)
|
||||
md[k] = vals
|
||||
}
|
67
clientv3/ctx_test.go
Normal file
67
clientv3/ctx_test.go
Normal file
@@ -0,0 +1,67 @@
|
||||
// Copyright 2020 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 (
|
||||
"context"
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
"go.etcd.io/etcd/version"
|
||||
"google.golang.org/grpc/metadata"
|
||||
)
|
||||
|
||||
func TestMetadataWithRequireLeader(t *testing.T) {
|
||||
ctx := context.TODO()
|
||||
md, ok := metadata.FromOutgoingContext(ctx)
|
||||
if ok {
|
||||
t.Fatal("expected no outgoing metadata ctx key")
|
||||
}
|
||||
|
||||
// add a conflicting key with some other value
|
||||
md = metadata.Pairs(rpctypes.MetadataRequireLeaderKey, "invalid")
|
||||
// add a key, and expect not be overwritten
|
||||
metadataSet(md, "hello", "1", "2")
|
||||
ctx = metadata.NewOutgoingContext(ctx, md)
|
||||
|
||||
// expect overwrites but still keep other keys
|
||||
ctx = WithRequireLeader(ctx)
|
||||
md, ok = metadata.FromOutgoingContext(ctx)
|
||||
if !ok {
|
||||
t.Fatal("expected outgoing metadata ctx key")
|
||||
}
|
||||
if ss := metadataGet(md, rpctypes.MetadataRequireLeaderKey); !reflect.DeepEqual(ss, []string{rpctypes.MetadataHasLeader}) {
|
||||
t.Fatalf("unexpected metadata for %q %v", rpctypes.MetadataRequireLeaderKey, ss)
|
||||
}
|
||||
if ss := metadataGet(md, "hello"); !reflect.DeepEqual(ss, []string{"1", "2"}) {
|
||||
t.Fatalf("unexpected metadata for 'hello' %v", ss)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMetadataWithClientAPIVersion(t *testing.T) {
|
||||
ctx := withVersion(WithRequireLeader(context.TODO()))
|
||||
|
||||
md, ok := metadata.FromOutgoingContext(ctx)
|
||||
if !ok {
|
||||
t.Fatal("expected outgoing metadata ctx key")
|
||||
}
|
||||
if ss := metadataGet(md, rpctypes.MetadataRequireLeaderKey); !reflect.DeepEqual(ss, []string{rpctypes.MetadataHasLeader}) {
|
||||
t.Fatalf("unexpected metadata for %q %v", rpctypes.MetadataRequireLeaderKey, ss)
|
||||
}
|
||||
if ss := metadataGet(md, rpctypes.MetadataClientAPIVersionKey); !reflect.DeepEqual(ss, []string{version.APIVersion}) {
|
||||
t.Fatalf("unexpected metadata for %q %v", rpctypes.MetadataClientAPIVersionKey, ss)
|
||||
}
|
||||
}
|
@@ -276,8 +276,7 @@ func TestMemberPromote(t *testing.T) {
|
||||
select {
|
||||
case <-time.After(500 * time.Millisecond):
|
||||
case <-timeout:
|
||||
t.Errorf("failed all attempts to promote learner member, last error: %v", err)
|
||||
break
|
||||
t.Fatalf("failed all attempts to promote learner member, last error: %v", err)
|
||||
}
|
||||
|
||||
_, err = capi.MemberPromote(context.Background(), learnerID)
|
||||
|
@@ -38,6 +38,7 @@ import (
|
||||
func (c *Client) unaryClientInterceptor(logger *zap.Logger, optFuncs ...retryOption) grpc.UnaryClientInterceptor {
|
||||
intOpts := reuseOrNewWithCallOptions(defaultOptions, optFuncs)
|
||||
return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
|
||||
ctx = withVersion(ctx)
|
||||
grpcOpts, retryOpts := filterCallOptions(opts)
|
||||
callOpts := reuseOrNewWithCallOptions(intOpts, retryOpts)
|
||||
// short circuit for simplicity, and avoiding allocations.
|
||||
@@ -103,6 +104,7 @@ func (c *Client) unaryClientInterceptor(logger *zap.Logger, optFuncs ...retryOpt
|
||||
func (c *Client) streamClientInterceptor(logger *zap.Logger, optFuncs ...retryOption) grpc.StreamClientInterceptor {
|
||||
intOpts := reuseOrNewWithCallOptions(defaultOptions, optFuncs)
|
||||
return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
|
||||
ctx = withVersion(ctx)
|
||||
grpcOpts, retryOpts := filterCallOptions(opts)
|
||||
callOpts := reuseOrNewWithCallOptions(intOpts, retryOpts)
|
||||
// short circuit for simplicity, and avoiding allocations.
|
||||
@@ -113,10 +115,9 @@ func (c *Client) streamClientInterceptor(logger *zap.Logger, optFuncs ...retryOp
|
||||
return nil, status.Errorf(codes.Unimplemented, "clientv3/retry_interceptor: cannot retry on ClientStreams, set Disable()")
|
||||
}
|
||||
newStreamer, err := streamer(ctx, desc, cc, method, grpcOpts...)
|
||||
logger.Warn("retry stream intercept", zap.Error(err))
|
||||
if err != nil {
|
||||
// TODO(mwitkow): Maybe dial and transport errors should be retriable?
|
||||
return nil, err
|
||||
logger.Error("streamer failed to create ClientStream", zap.Error(err))
|
||||
return nil, err // TODO(mwitkow): Maybe dial and transport errors should be retriable?
|
||||
}
|
||||
retryingStreamer := &serverStreamingRetryingStream{
|
||||
client: c,
|
||||
@@ -185,6 +186,7 @@ func (s *serverStreamingRetryingStream) RecvMsg(m interface{}) error {
|
||||
if !attemptRetry {
|
||||
return lastErr // success or hard failure
|
||||
}
|
||||
|
||||
// We start off from attempt 1, because zeroth was already made on normal SendMsg().
|
||||
for attempt := uint(1); attempt < s.callOpts.max; attempt++ {
|
||||
if err := waitRetryBackoff(s.ctx, attempt, s.callOpts); err != nil {
|
||||
@@ -192,12 +194,13 @@ func (s *serverStreamingRetryingStream) RecvMsg(m interface{}) error {
|
||||
}
|
||||
newStream, err := s.reestablishStreamAndResendBuffer(s.ctx)
|
||||
if err != nil {
|
||||
// TODO(mwitkow): Maybe dial and transport errors should be retriable?
|
||||
return err
|
||||
s.client.lg.Error("failed reestablishStreamAndResendBuffer", zap.Error(err))
|
||||
return err // TODO(mwitkow): Maybe dial and transport errors should be retriable?
|
||||
}
|
||||
s.setStream(newStream)
|
||||
|
||||
s.client.lg.Warn("retrying RecvMsg", zap.Error(lastErr))
|
||||
attemptRetry, lastErr = s.receiveMsgAndIndicateRetry(m)
|
||||
//fmt.Printf("Received message and indicate: %v %v\n", attemptRetry, lastErr)
|
||||
if !attemptRetry {
|
||||
return lastErr
|
||||
}
|
||||
|
@@ -39,6 +39,7 @@ import (
|
||||
"go.etcd.io/etcd/mvcc"
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/pkg/fileutil"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
"go.etcd.io/etcd/pkg/types"
|
||||
"go.etcd.io/etcd/raft"
|
||||
"go.etcd.io/etcd/raft/raftpb"
|
||||
@@ -384,7 +385,7 @@ func (s *v3Manager) saveDB() error {
|
||||
lessor := lease.NewLessor(s.lg, be, lease.LessorConfig{MinLeaseTTL: math.MaxInt64})
|
||||
|
||||
mvs := mvcc.NewStore(s.lg, be, lessor, (*initIndex)(&commit), mvcc.StoreConfig{CompactionBatchLimit: math.MaxInt32})
|
||||
txn := mvs.Write()
|
||||
txn := mvs.Write(traceutil.TODO())
|
||||
btx := be.BatchTx()
|
||||
del := func(k, v []byte) error {
|
||||
txn.DeleteRange(k, nil)
|
||||
|
@@ -303,8 +303,8 @@ type Config struct {
|
||||
// It can be multiple when "Logger" is zap.
|
||||
LogOutputs []string `json:"log-outputs"`
|
||||
|
||||
// zapLoggerBuilder is used to build the zap logger.
|
||||
zapLoggerBuilder func(*Config) error
|
||||
// ZapLoggerBuilder is used to build the zap logger.
|
||||
ZapLoggerBuilder func(*Config) error
|
||||
|
||||
// logger logs server-side operations. The default is nil,
|
||||
// and "setupLogging" must be called before starting server.
|
||||
|
@@ -181,8 +181,8 @@ func (cfg *Config) setupLogging() error {
|
||||
// TODO: remove "Debug" check in v3.5
|
||||
grpc.EnableTracing = true
|
||||
}
|
||||
if cfg.zapLoggerBuilder == nil {
|
||||
cfg.zapLoggerBuilder = func(c *Config) error {
|
||||
if cfg.ZapLoggerBuilder == nil {
|
||||
cfg.ZapLoggerBuilder = func(c *Config) error {
|
||||
var err error
|
||||
c.logger, err = copied.Build()
|
||||
if err != nil {
|
||||
@@ -235,8 +235,8 @@ func (cfg *Config) setupLogging() error {
|
||||
syncer,
|
||||
lvl,
|
||||
)
|
||||
if cfg.zapLoggerBuilder == nil {
|
||||
cfg.zapLoggerBuilder = func(c *Config) error {
|
||||
if cfg.ZapLoggerBuilder == nil {
|
||||
cfg.ZapLoggerBuilder = func(c *Config) error {
|
||||
c.logger = zap.New(cr, zap.AddCaller(), zap.ErrorOutput(syncer))
|
||||
c.loggerMu.Lock()
|
||||
defer c.loggerMu.Unlock()
|
||||
@@ -252,7 +252,7 @@ func (cfg *Config) setupLogging() error {
|
||||
}
|
||||
}
|
||||
|
||||
err := cfg.zapLoggerBuilder(cfg)
|
||||
err := cfg.ZapLoggerBuilder(cfg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@@ -189,7 +189,7 @@ func (sctx *serveCtx) serve(
|
||||
sctx.serversC <- &servers{secure: true, grpc: gs, http: srv}
|
||||
if sctx.lg != nil {
|
||||
sctx.lg.Info(
|
||||
"serving client traffic insecurely",
|
||||
"serving client traffic securely",
|
||||
zap.String("address", sctx.l.Addr().String()),
|
||||
)
|
||||
} else {
|
||||
|
@@ -156,28 +156,8 @@ func memberAddCommandFunc(cmd *cobra.Command, args []string) {
|
||||
display.MemberAdd(*resp)
|
||||
|
||||
if _, ok := (display).(*simplePrinter); ok {
|
||||
ctx, cancel = commandCtx(cmd)
|
||||
listResp, err := cli.MemberList(ctx)
|
||||
// get latest member list; if there's failover new member might have outdated list
|
||||
for {
|
||||
if err != nil {
|
||||
ExitWithError(ExitError, err)
|
||||
}
|
||||
if listResp.Header.MemberId == resp.Header.MemberId {
|
||||
break
|
||||
}
|
||||
// quorum get to sync cluster list
|
||||
gresp, gerr := cli.Get(ctx, "_")
|
||||
if gerr != nil {
|
||||
ExitWithError(ExitError, err)
|
||||
}
|
||||
resp.Header.MemberId = gresp.Header.MemberId
|
||||
listResp, err = cli.MemberList(ctx)
|
||||
}
|
||||
cancel()
|
||||
|
||||
conf := []string{}
|
||||
for _, memb := range listResp.Members {
|
||||
for _, memb := range resp.Members {
|
||||
for _, u := range memb.PeerURLs {
|
||||
n := memb.Name
|
||||
if memb.ID == newID {
|
||||
|
@@ -50,6 +50,7 @@ func NewHealthHandler(hfunc func() Health) http.HandlerFunc {
|
||||
if r.Method != http.MethodGet {
|
||||
w.Header().Set("Allow", http.MethodGet)
|
||||
http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
|
||||
plog.Warningf("/health error (status code %d)", http.StatusMethodNotAllowed)
|
||||
return
|
||||
}
|
||||
h := hfunc()
|
||||
@@ -97,11 +98,15 @@ func checkHealth(srv etcdserver.ServerV2) Health {
|
||||
as := srv.Alarms()
|
||||
if len(as) > 0 {
|
||||
h.Health = "false"
|
||||
for _, v := range as {
|
||||
plog.Warningf("/health error due to an alarm %s", v.String())
|
||||
}
|
||||
}
|
||||
|
||||
if h.Health == "true" {
|
||||
if uint64(srv.Leader()) == raft.None {
|
||||
h.Health = "false"
|
||||
plog.Warningf("/health error; no leader (status code %d)", http.StatusServiceUnavailable)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -111,11 +116,13 @@ func checkHealth(srv etcdserver.ServerV2) Health {
|
||||
cancel()
|
||||
if err != nil {
|
||||
h.Health = "false"
|
||||
plog.Warningf("/health error; QGET failed %v (status code %d)", err, http.StatusServiceUnavailable)
|
||||
}
|
||||
}
|
||||
|
||||
if h.Health == "true" {
|
||||
healthSuccess.Inc()
|
||||
plog.Infof("/health OK (status code %d)", http.StatusOK)
|
||||
} else {
|
||||
healthFailed.Inc()
|
||||
}
|
||||
|
@@ -37,11 +37,17 @@ const (
|
||||
)
|
||||
|
||||
// NewPeerHandler generates an http.Handler to handle etcd peer requests.
|
||||
func NewPeerHandler(lg *zap.Logger, s etcdserver.ServerPeer) http.Handler {
|
||||
return newPeerHandler(lg, s, s.RaftHandler(), s.LeaseHandler())
|
||||
func NewPeerHandler(lg *zap.Logger, s etcdserver.ServerPeerV2) http.Handler {
|
||||
return newPeerHandler(lg, s, s.RaftHandler(), s.LeaseHandler(), s.HashKVHandler())
|
||||
}
|
||||
|
||||
func newPeerHandler(lg *zap.Logger, s etcdserver.Server, raftHandler http.Handler, leaseHandler http.Handler) http.Handler {
|
||||
func newPeerHandler(
|
||||
lg *zap.Logger,
|
||||
s etcdserver.Server,
|
||||
raftHandler http.Handler,
|
||||
leaseHandler http.Handler,
|
||||
hashKVHandler http.Handler,
|
||||
) http.Handler {
|
||||
peerMembersHandler := newPeerMembersHandler(lg, s.Cluster())
|
||||
peerMemberPromoteHandler := newPeerMemberPromoteHandler(lg, s)
|
||||
|
||||
@@ -55,6 +61,9 @@ func newPeerHandler(lg *zap.Logger, s etcdserver.Server, raftHandler http.Handle
|
||||
mux.Handle(leasehttp.LeasePrefix, leaseHandler)
|
||||
mux.Handle(leasehttp.LeaseInternalPrefix, leaseHandler)
|
||||
}
|
||||
if hashKVHandler != nil {
|
||||
mux.Handle(etcdserver.PeerHashKVPath, hashKVHandler)
|
||||
}
|
||||
mux.HandleFunc(versionPath, versionHandler(s.Cluster(), serveVersion))
|
||||
return mux
|
||||
}
|
||||
|
@@ -83,7 +83,7 @@ var fakeRaftHandler = http.HandlerFunc(func(w http.ResponseWriter, r *http.Reque
|
||||
// TestNewPeerHandlerOnRaftPrefix tests that NewPeerHandler returns a handler that
|
||||
// handles raft-prefix requests well.
|
||||
func TestNewPeerHandlerOnRaftPrefix(t *testing.T) {
|
||||
ph := newPeerHandler(zap.NewExample(), &fakeServer{cluster: &fakeCluster{}}, fakeRaftHandler, nil)
|
||||
ph := newPeerHandler(zap.NewExample(), &fakeServer{cluster: &fakeCluster{}}, fakeRaftHandler, nil, nil)
|
||||
srv := httptest.NewServer(ph)
|
||||
defer srv.Close()
|
||||
|
||||
@@ -231,7 +231,7 @@ func TestServeMemberPromoteFails(t *testing.T) {
|
||||
|
||||
// TestNewPeerHandlerOnMembersPromotePrefix verifies the request with members promote prefix is routed correctly
|
||||
func TestNewPeerHandlerOnMembersPromotePrefix(t *testing.T) {
|
||||
ph := newPeerHandler(zap.NewExample(), &fakeServer{cluster: &fakeCluster{}}, fakeRaftHandler, nil)
|
||||
ph := newPeerHandler(zap.NewExample(), &fakeServer{cluster: &fakeCluster{}}, fakeRaftHandler, nil, nil)
|
||||
srv := httptest.NewServer(ph)
|
||||
defer srv.Close()
|
||||
|
||||
|
@@ -565,6 +565,7 @@ func (c *RaftCluster) SetVersion(ver *semver.Version, onSet func(*zap.Logger, *s
|
||||
plog.Noticef("set the initial cluster version to %v", version.Cluster(ver.String()))
|
||||
}
|
||||
}
|
||||
oldVer := c.version
|
||||
c.version = ver
|
||||
mustDetectDowngrade(c.lg, c.version)
|
||||
if c.v2store != nil {
|
||||
@@ -573,7 +574,10 @@ func (c *RaftCluster) SetVersion(ver *semver.Version, onSet func(*zap.Logger, *s
|
||||
if c.be != nil {
|
||||
mustSaveClusterVersionToBackend(c.be, ver)
|
||||
}
|
||||
ClusterVersionMetrics.With(prometheus.Labels{"cluster_version": ver.String()}).Set(1)
|
||||
if oldVer != nil {
|
||||
ClusterVersionMetrics.With(prometheus.Labels{"cluster_version": version.Cluster(oldVer.String())}).Set(0)
|
||||
}
|
||||
ClusterVersionMetrics.With(prometheus.Labels{"cluster_version": version.Cluster(ver.String())}).Set(1)
|
||||
onSet(c.lg, ver)
|
||||
}
|
||||
|
||||
@@ -653,8 +657,8 @@ func (c *RaftCluster) IsReadyToRemoveVotingMember(id uint64) bool {
|
||||
}
|
||||
|
||||
func (c *RaftCluster) IsReadyToPromoteMember(id uint64) bool {
|
||||
nmembers := 1
|
||||
nstarted := 0
|
||||
nmembers := 1 // We count the learner to be promoted for the future quorum
|
||||
nstarted := 1 // and we also count it as started.
|
||||
|
||||
for _, member := range c.VotingMembers() {
|
||||
if member.IsStarted() {
|
||||
|
@@ -859,3 +859,90 @@ func TestIsReadyToRemoveVotingMember(t *testing.T) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestIsReadyToPromoteMember(t *testing.T) {
|
||||
tests := []struct {
|
||||
members []*Member
|
||||
promoteID uint64
|
||||
want bool
|
||||
}{
|
||||
{
|
||||
// 1/1 members ready, should succeed (quorum = 1, new quorum = 2)
|
||||
[]*Member{
|
||||
newTestMember(1, nil, "1", nil),
|
||||
newTestMemberAsLearner(2, nil, "2", nil),
|
||||
},
|
||||
2,
|
||||
true,
|
||||
},
|
||||
{
|
||||
// 0/1 members ready, should fail (quorum = 1)
|
||||
[]*Member{
|
||||
newTestMember(1, nil, "", nil),
|
||||
newTestMemberAsLearner(2, nil, "2", nil),
|
||||
},
|
||||
2,
|
||||
false,
|
||||
},
|
||||
{
|
||||
// 2/2 members ready, should succeed (quorum = 2)
|
||||
[]*Member{
|
||||
newTestMember(1, nil, "1", nil),
|
||||
newTestMember(2, nil, "2", nil),
|
||||
newTestMemberAsLearner(3, nil, "3", nil),
|
||||
},
|
||||
3,
|
||||
true,
|
||||
},
|
||||
{
|
||||
// 1/2 members ready, should succeed (quorum = 2)
|
||||
[]*Member{
|
||||
newTestMember(1, nil, "1", nil),
|
||||
newTestMember(2, nil, "", nil),
|
||||
newTestMemberAsLearner(3, nil, "3", nil),
|
||||
},
|
||||
3,
|
||||
true,
|
||||
},
|
||||
{
|
||||
// 1/3 members ready, should fail (quorum = 2)
|
||||
[]*Member{
|
||||
newTestMember(1, nil, "1", nil),
|
||||
newTestMember(2, nil, "", nil),
|
||||
newTestMember(3, nil, "", nil),
|
||||
newTestMemberAsLearner(4, nil, "4", nil),
|
||||
},
|
||||
4,
|
||||
false,
|
||||
},
|
||||
{
|
||||
// 2/3 members ready, should succeed (quorum = 2, new quorum = 3)
|
||||
[]*Member{
|
||||
newTestMember(1, nil, "1", nil),
|
||||
newTestMember(2, nil, "2", nil),
|
||||
newTestMember(3, nil, "", nil),
|
||||
newTestMemberAsLearner(4, nil, "4", nil),
|
||||
},
|
||||
4,
|
||||
true,
|
||||
},
|
||||
{
|
||||
// 2/4 members ready, should succeed (quorum = 3)
|
||||
[]*Member{
|
||||
newTestMember(1, nil, "1", nil),
|
||||
newTestMember(2, nil, "2", nil),
|
||||
newTestMember(3, nil, "", nil),
|
||||
newTestMember(4, nil, "", nil),
|
||||
newTestMemberAsLearner(5, nil, "5", nil),
|
||||
},
|
||||
5,
|
||||
true,
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
c := newTestCluster(tt.members)
|
||||
if got := c.IsReadyToPromoteMember(tt.promoteID); got != tt.want {
|
||||
t.Errorf("%d: isReadyToPromoteMember returned %t, want %t", i, got, tt.want)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@@ -57,6 +57,7 @@ var (
|
||||
"3.1.0": {streamTypeMsgAppV2, streamTypeMessage},
|
||||
"3.2.0": {streamTypeMsgAppV2, streamTypeMessage},
|
||||
"3.3.0": {streamTypeMsgAppV2, streamTypeMessage},
|
||||
"3.4.0": {streamTypeMsgAppV2, streamTypeMessage},
|
||||
}
|
||||
)
|
||||
|
||||
|
@@ -226,6 +226,9 @@ func (s *Snapshotter) snapNames() ([]string, error) {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err = s.cleanupSnapdir(names); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
snaps := checkSuffix(s.lg, names)
|
||||
if len(snaps) == 0 {
|
||||
return nil, ErrNoSnapshot
|
||||
@@ -253,3 +256,21 @@ func checkSuffix(lg *zap.Logger, names []string) []string {
|
||||
}
|
||||
return snaps
|
||||
}
|
||||
|
||||
// cleanupSnapdir removes any files that should not be in the snapshot directory:
|
||||
// - db.tmp prefixed files that can be orphaned by defragmentation
|
||||
func (s *Snapshotter) cleanupSnapdir(filenames []string) error {
|
||||
for _, filename := range filenames {
|
||||
if strings.HasPrefix(filename, "db.tmp") {
|
||||
if s.lg != nil {
|
||||
s.lg.Info("found orphaned defragmentation file; deleting", zap.String("path", filename))
|
||||
} else {
|
||||
plog.Infof("found orphaned defragmentation file; deleting: %s", filename)
|
||||
}
|
||||
if rmErr := os.Remove(filepath.Join(s.dir, filename)); rmErr != nil && !os.IsNotExist(rmErr) {
|
||||
return fmt.Errorf("failed to remove orphaned defragmentation file %s: %v", filename, rmErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@@ -16,17 +16,17 @@ package v3rpc
|
||||
|
||||
import (
|
||||
"context"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/pkg/capnslog"
|
||||
"go.etcd.io/etcd/etcdserver"
|
||||
"go.etcd.io/etcd/etcdserver/api"
|
||||
"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
|
||||
"go.etcd.io/etcd/pkg/types"
|
||||
"go.etcd.io/etcd/raft"
|
||||
|
||||
"github.com/coreos/pkg/capnslog"
|
||||
pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/metadata"
|
||||
@@ -54,6 +54,12 @@ func newUnaryInterceptor(s *etcdserver.EtcdServer) grpc.UnaryServerInterceptor {
|
||||
|
||||
md, ok := metadata.FromIncomingContext(ctx)
|
||||
if ok {
|
||||
ver, vs := "unknown", metadataGet(md, rpctypes.MetadataClientAPIVersionKey)
|
||||
if len(vs) > 0 {
|
||||
ver = vs[0]
|
||||
}
|
||||
clientRequests.WithLabelValues("unary", ver).Inc()
|
||||
|
||||
if ks := md[rpctypes.MetadataRequireLeaderKey]; len(ks) > 0 && ks[0] == rpctypes.MetadataHasLeader {
|
||||
if s.Leader() == types.ID(raft.None) {
|
||||
return nil, rpctypes.ErrGRPCNoLeader
|
||||
@@ -200,6 +206,12 @@ func newStreamInterceptor(s *etcdserver.EtcdServer) grpc.StreamServerInterceptor
|
||||
|
||||
md, ok := metadata.FromIncomingContext(ss.Context())
|
||||
if ok {
|
||||
ver, vs := "unknown", metadataGet(md, rpctypes.MetadataClientAPIVersionKey)
|
||||
if len(vs) > 0 {
|
||||
ver = vs[0]
|
||||
}
|
||||
clientRequests.WithLabelValues("stream", ver).Inc()
|
||||
|
||||
if ks := md[rpctypes.MetadataRequireLeaderKey]; len(ks) > 0 && ks[0] == rpctypes.MetadataHasLeader {
|
||||
if s.Leader() == types.ID(raft.None) {
|
||||
return rpctypes.ErrGRPCNoLeader
|
||||
@@ -218,7 +230,6 @@ func newStreamInterceptor(s *etcdserver.EtcdServer) grpc.StreamServerInterceptor
|
||||
smap.mu.Unlock()
|
||||
cancel()
|
||||
}()
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@@ -274,3 +285,8 @@ func monitorLeader(s *etcdserver.EtcdServer) *streamsMap {
|
||||
|
||||
return smap
|
||||
}
|
||||
|
||||
func metadataGet(md metadata.MD, k string) []string {
|
||||
k = strings.ToLower(k)
|
||||
return md[k]
|
||||
}
|
||||
|
@@ -39,10 +39,20 @@ var (
|
||||
},
|
||||
[]string{"Type", "API"},
|
||||
)
|
||||
|
||||
clientRequests = prometheus.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: "etcd",
|
||||
Subsystem: "server",
|
||||
Name: "client_requests_total",
|
||||
Help: "The total number of client requests per client version.",
|
||||
},
|
||||
[]string{"type", "client_api_version"},
|
||||
)
|
||||
)
|
||||
|
||||
func init() {
|
||||
prometheus.MustRegister(sentBytes)
|
||||
prometheus.MustRegister(receivedBytes)
|
||||
prometheus.MustRegister(streamFailures)
|
||||
prometheus.MustRegister(clientRequests)
|
||||
}
|
||||
|
@@ -17,4 +17,6 @@ package rpctypes
|
||||
var (
|
||||
MetadataRequireLeaderKey = "hasleader"
|
||||
MetadataHasLeader = "true"
|
||||
|
||||
MetadataClientAPIVersionKey = "client-api-version"
|
||||
)
|
||||
|
@@ -26,6 +26,7 @@ import (
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/mvcc"
|
||||
"go.etcd.io/etcd/mvcc/mvccpb"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
"go.etcd.io/etcd/pkg/types"
|
||||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
@@ -43,17 +44,18 @@ type applyResult struct {
|
||||
// to being logically reflected by the node. Currently only used for
|
||||
// Compaction requests.
|
||||
physc <-chan struct{}
|
||||
trace *traceutil.Trace
|
||||
}
|
||||
|
||||
// applierV3 is the interface for processing V3 raft messages
|
||||
type applierV3 interface {
|
||||
Apply(r *pb.InternalRaftRequest) *applyResult
|
||||
|
||||
Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, error)
|
||||
Range(txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.RangeResponse, error)
|
||||
Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, *traceutil.Trace, error)
|
||||
Range(ctx context.Context, txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.RangeResponse, error)
|
||||
DeleteRange(txn mvcc.TxnWrite, dr *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error)
|
||||
Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error)
|
||||
Compaction(compaction *pb.CompactionRequest) (*pb.CompactionResponse, <-chan struct{}, error)
|
||||
Compaction(compaction *pb.CompactionRequest) (*pb.CompactionResponse, <-chan struct{}, *traceutil.Trace, error)
|
||||
|
||||
LeaseGrant(lc *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error)
|
||||
LeaseRevoke(lc *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error)
|
||||
@@ -119,15 +121,15 @@ func (a *applierV3backend) Apply(r *pb.InternalRaftRequest) *applyResult {
|
||||
// call into a.s.applyV3.F instead of a.F so upper appliers can check individual calls
|
||||
switch {
|
||||
case r.Range != nil:
|
||||
ar.resp, ar.err = a.s.applyV3.Range(nil, r.Range)
|
||||
ar.resp, ar.err = a.s.applyV3.Range(context.TODO(), nil, r.Range)
|
||||
case r.Put != nil:
|
||||
ar.resp, ar.err = a.s.applyV3.Put(nil, r.Put)
|
||||
ar.resp, ar.trace, ar.err = a.s.applyV3.Put(nil, r.Put)
|
||||
case r.DeleteRange != nil:
|
||||
ar.resp, ar.err = a.s.applyV3.DeleteRange(nil, r.DeleteRange)
|
||||
case r.Txn != nil:
|
||||
ar.resp, ar.err = a.s.applyV3.Txn(r.Txn)
|
||||
case r.Compaction != nil:
|
||||
ar.resp, ar.physc, ar.err = a.s.applyV3.Compaction(r.Compaction)
|
||||
ar.resp, ar.physc, ar.trace, ar.err = a.s.applyV3.Compaction(r.Compaction)
|
||||
case r.LeaseGrant != nil:
|
||||
ar.resp, ar.err = a.s.applyV3.LeaseGrant(r.LeaseGrant)
|
||||
case r.LeaseRevoke != nil:
|
||||
@@ -174,32 +176,39 @@ func (a *applierV3backend) Apply(r *pb.InternalRaftRequest) *applyResult {
|
||||
return ar
|
||||
}
|
||||
|
||||
func (a *applierV3backend) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (resp *pb.PutResponse, err error) {
|
||||
func (a *applierV3backend) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (resp *pb.PutResponse, trace *traceutil.Trace, err error) {
|
||||
resp = &pb.PutResponse{}
|
||||
resp.Header = &pb.ResponseHeader{}
|
||||
|
||||
trace = traceutil.New("put",
|
||||
a.s.getLogger(),
|
||||
traceutil.Field{Key: "key", Value: string(p.Key)},
|
||||
traceutil.Field{Key: "req_size", Value: proto.Size(p)},
|
||||
)
|
||||
val, leaseID := p.Value, lease.LeaseID(p.Lease)
|
||||
if txn == nil {
|
||||
if leaseID != lease.NoLease {
|
||||
if l := a.s.lessor.Lookup(leaseID); l == nil {
|
||||
return nil, lease.ErrLeaseNotFound
|
||||
return nil, nil, lease.ErrLeaseNotFound
|
||||
}
|
||||
}
|
||||
txn = a.s.KV().Write()
|
||||
txn = a.s.KV().Write(trace)
|
||||
defer txn.End()
|
||||
}
|
||||
|
||||
var rr *mvcc.RangeResult
|
||||
if p.IgnoreValue || p.IgnoreLease || p.PrevKv {
|
||||
trace.DisableStep()
|
||||
rr, err = txn.Range(p.Key, nil, mvcc.RangeOptions{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, nil, err
|
||||
}
|
||||
trace.EnableStep()
|
||||
trace.Step("get previous kv pair")
|
||||
}
|
||||
if p.IgnoreValue || p.IgnoreLease {
|
||||
if rr == nil || len(rr.KVs) == 0 {
|
||||
// ignore_{lease,value} flag expects previous key-value pair
|
||||
return nil, ErrKeyNotFound
|
||||
return nil, nil, ErrKeyNotFound
|
||||
}
|
||||
}
|
||||
if p.IgnoreValue {
|
||||
@@ -215,7 +224,8 @@ func (a *applierV3backend) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (resp *pb.Pu
|
||||
}
|
||||
|
||||
resp.Header.Revision = txn.Put(p.Key, val, leaseID)
|
||||
return resp, nil
|
||||
trace.AddField(traceutil.Field{Key: "response_revision", Value: resp.Header.Revision})
|
||||
return resp, trace, nil
|
||||
}
|
||||
|
||||
func (a *applierV3backend) DeleteRange(txn mvcc.TxnWrite, dr *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) {
|
||||
@@ -224,7 +234,7 @@ func (a *applierV3backend) DeleteRange(txn mvcc.TxnWrite, dr *pb.DeleteRangeRequ
|
||||
end := mkGteRange(dr.RangeEnd)
|
||||
|
||||
if txn == nil {
|
||||
txn = a.s.kv.Write()
|
||||
txn = a.s.kv.Write(traceutil.TODO())
|
||||
defer txn.End()
|
||||
}
|
||||
|
||||
@@ -245,12 +255,14 @@ func (a *applierV3backend) DeleteRange(txn mvcc.TxnWrite, dr *pb.DeleteRangeRequ
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
func (a *applierV3backend) Range(txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.RangeResponse, error) {
|
||||
func (a *applierV3backend) Range(ctx context.Context, txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.RangeResponse, error) {
|
||||
trace := traceutil.Get(ctx)
|
||||
|
||||
resp := &pb.RangeResponse{}
|
||||
resp.Header = &pb.ResponseHeader{}
|
||||
|
||||
if txn == nil {
|
||||
txn = a.s.kv.Read()
|
||||
txn = a.s.kv.Read(trace)
|
||||
defer txn.End()
|
||||
}
|
||||
|
||||
@@ -327,7 +339,7 @@ func (a *applierV3backend) Range(txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.Rang
|
||||
rr.KVs = rr.KVs[:r.Limit]
|
||||
resp.More = true
|
||||
}
|
||||
|
||||
trace.Step("filter and sort the key-value pairs")
|
||||
resp.Header.Revision = rr.Rev
|
||||
resp.Count = int64(rr.Count)
|
||||
resp.Kvs = make([]*mvccpb.KeyValue, len(rr.KVs))
|
||||
@@ -337,12 +349,13 @@ func (a *applierV3backend) Range(txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.Rang
|
||||
}
|
||||
resp.Kvs[i] = &rr.KVs[i]
|
||||
}
|
||||
trace.Step("assemble the response")
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
func (a *applierV3backend) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) {
|
||||
isWrite := !isTxnReadonly(rt)
|
||||
txn := mvcc.NewReadOnlyTxnWrite(a.s.KV().Read())
|
||||
txn := mvcc.NewReadOnlyTxnWrite(a.s.KV().Read(traceutil.TODO()))
|
||||
|
||||
txnPath := compareToPath(txn, rt)
|
||||
if isWrite {
|
||||
@@ -364,7 +377,7 @@ func (a *applierV3backend) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) {
|
||||
// be the revision of the write txn.
|
||||
if isWrite {
|
||||
txn.End()
|
||||
txn = a.s.KV().Write()
|
||||
txn = a.s.KV().Write(traceutil.TODO())
|
||||
}
|
||||
a.applyTxn(txn, rt, txnPath, txnResp)
|
||||
rev := txn.Rev()
|
||||
@@ -516,7 +529,7 @@ func (a *applierV3backend) applyTxn(txn mvcc.TxnWrite, rt *pb.TxnRequest, txnPat
|
||||
respi := tresp.Responses[i].Response
|
||||
switch tv := req.Request.(type) {
|
||||
case *pb.RequestOp_RequestRange:
|
||||
resp, err := a.Range(txn, tv.RequestRange)
|
||||
resp, err := a.Range(context.TODO(), txn, tv.RequestRange)
|
||||
if err != nil {
|
||||
if lg != nil {
|
||||
lg.Panic("unexpected error during txn", zap.Error(err))
|
||||
@@ -526,7 +539,7 @@ func (a *applierV3backend) applyTxn(txn mvcc.TxnWrite, rt *pb.TxnRequest, txnPat
|
||||
}
|
||||
respi.(*pb.ResponseOp_ResponseRange).ResponseRange = resp
|
||||
case *pb.RequestOp_RequestPut:
|
||||
resp, err := a.Put(txn, tv.RequestPut)
|
||||
resp, _, err := a.Put(txn, tv.RequestPut)
|
||||
if err != nil {
|
||||
if lg != nil {
|
||||
lg.Panic("unexpected error during txn", zap.Error(err))
|
||||
@@ -557,17 +570,22 @@ func (a *applierV3backend) applyTxn(txn mvcc.TxnWrite, rt *pb.TxnRequest, txnPat
|
||||
return txns
|
||||
}
|
||||
|
||||
func (a *applierV3backend) Compaction(compaction *pb.CompactionRequest) (*pb.CompactionResponse, <-chan struct{}, error) {
|
||||
func (a *applierV3backend) Compaction(compaction *pb.CompactionRequest) (*pb.CompactionResponse, <-chan struct{}, *traceutil.Trace, error) {
|
||||
resp := &pb.CompactionResponse{}
|
||||
resp.Header = &pb.ResponseHeader{}
|
||||
ch, err := a.s.KV().Compact(compaction.Revision)
|
||||
trace := traceutil.New("compact",
|
||||
a.s.getLogger(),
|
||||
traceutil.Field{Key: "revision", Value: compaction.Revision},
|
||||
)
|
||||
|
||||
ch, err := a.s.KV().Compact(trace, compaction.Revision)
|
||||
if err != nil {
|
||||
return nil, ch, err
|
||||
return nil, ch, nil, err
|
||||
}
|
||||
// get the current revision. which key to get is not important.
|
||||
rr, _ := a.s.KV().Range([]byte("compaction"), nil, mvcc.RangeOptions{})
|
||||
resp.Header.Revision = rr.Rev
|
||||
return resp, ch, err
|
||||
return resp, ch, trace, err
|
||||
}
|
||||
|
||||
func (a *applierV3backend) LeaseGrant(lc *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error) {
|
||||
@@ -674,8 +692,8 @@ type applierV3Capped struct {
|
||||
// with Puts so that the number of keys in the store is capped.
|
||||
func newApplierV3Capped(base applierV3) applierV3 { return &applierV3Capped{applierV3: base} }
|
||||
|
||||
func (a *applierV3Capped) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, error) {
|
||||
return nil, ErrNoSpace
|
||||
func (a *applierV3Capped) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, *traceutil.Trace, error) {
|
||||
return nil, nil, ErrNoSpace
|
||||
}
|
||||
|
||||
func (a *applierV3Capped) Txn(r *pb.TxnRequest) (*pb.TxnResponse, error) {
|
||||
@@ -824,13 +842,13 @@ func newQuotaApplierV3(s *EtcdServer, app applierV3) applierV3 {
|
||||
return "aApplierV3{app, NewBackendQuota(s, "v3-applier")}
|
||||
}
|
||||
|
||||
func (a *quotaApplierV3) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, error) {
|
||||
func (a *quotaApplierV3) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, *traceutil.Trace, error) {
|
||||
ok := a.q.Available(p)
|
||||
resp, err := a.applierV3.Put(txn, p)
|
||||
resp, trace, err := a.applierV3.Put(txn, p)
|
||||
if err == nil && !ok {
|
||||
err = ErrNoSpace
|
||||
}
|
||||
return resp, err
|
||||
return resp, trace, err
|
||||
}
|
||||
|
||||
func (a *quotaApplierV3) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) {
|
||||
|
@@ -15,12 +15,14 @@
|
||||
package etcdserver
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
|
||||
"go.etcd.io/etcd/auth"
|
||||
pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/mvcc"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
)
|
||||
|
||||
type authApplierV3 struct {
|
||||
@@ -61,9 +63,9 @@ func (aa *authApplierV3) Apply(r *pb.InternalRaftRequest) *applyResult {
|
||||
return ret
|
||||
}
|
||||
|
||||
func (aa *authApplierV3) Put(txn mvcc.TxnWrite, r *pb.PutRequest) (*pb.PutResponse, error) {
|
||||
func (aa *authApplierV3) Put(txn mvcc.TxnWrite, r *pb.PutRequest) (*pb.PutResponse, *traceutil.Trace, error) {
|
||||
if err := aa.as.IsPutPermitted(&aa.authInfo, r.Key); err != nil {
|
||||
return nil, err
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
if err := aa.checkLeasePuts(lease.LeaseID(r.Lease)); err != nil {
|
||||
@@ -71,23 +73,23 @@ func (aa *authApplierV3) Put(txn mvcc.TxnWrite, r *pb.PutRequest) (*pb.PutRespon
|
||||
// be written by this user. It means the user cannot revoke the
|
||||
// lease so attaching the lease to the newly written key should
|
||||
// be forbidden.
|
||||
return nil, err
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
if r.PrevKv {
|
||||
err := aa.as.IsRangePermitted(&aa.authInfo, r.Key, nil)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, nil, err
|
||||
}
|
||||
}
|
||||
return aa.applierV3.Put(txn, r)
|
||||
}
|
||||
|
||||
func (aa *authApplierV3) Range(txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.RangeResponse, error) {
|
||||
func (aa *authApplierV3) Range(ctx context.Context, txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.RangeResponse, error) {
|
||||
if err := aa.as.IsRangePermitted(&aa.authInfo, r.Key, r.RangeEnd); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return aa.applierV3.Range(txn, r)
|
||||
return aa.applierV3.Range(ctx, txn, r)
|
||||
}
|
||||
|
||||
func (aa *authApplierV3) DeleteRange(txn mvcc.TxnWrite, r *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) {
|
||||
|
@@ -15,14 +15,19 @@
|
||||
package etcdserver
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
|
||||
"go.etcd.io/etcd/mvcc"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
"go.etcd.io/etcd/pkg/types"
|
||||
|
||||
"go.uber.org/zap"
|
||||
@@ -229,10 +234,12 @@ func (s *EtcdServer) checkHashKV() error {
|
||||
mismatch(uint64(s.ID()))
|
||||
}
|
||||
|
||||
checkedCount := 0
|
||||
for _, p := range peers {
|
||||
if p.resp == nil {
|
||||
continue
|
||||
}
|
||||
checkedCount++
|
||||
id := p.resp.Header.MemberId
|
||||
|
||||
// leader expects follower's latest revision less than or equal to leader's
|
||||
@@ -297,62 +304,56 @@ func (s *EtcdServer) checkHashKV() error {
|
||||
mismatch(id)
|
||||
}
|
||||
}
|
||||
if lg != nil {
|
||||
lg.Info("finished peer corruption check", zap.Int("number-of-peers-checked", checkedCount))
|
||||
} else {
|
||||
plog.Infof("finished peer corruption check")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
type peerHashKVResp struct {
|
||||
type peerInfo struct {
|
||||
id types.ID
|
||||
eps []string
|
||||
}
|
||||
|
||||
resp *clientv3.HashKVResponse
|
||||
type peerHashKVResp struct {
|
||||
peerInfo
|
||||
resp *pb.HashKVResponse
|
||||
err error
|
||||
}
|
||||
|
||||
func (s *EtcdServer) getPeerHashKVs(rev int64) (resps []*peerHashKVResp) {
|
||||
func (s *EtcdServer) getPeerHashKVs(rev int64) []*peerHashKVResp {
|
||||
// TODO: handle the case when "s.cluster.Members" have not
|
||||
// been populated (e.g. no snapshot to load from disk)
|
||||
mbs := s.cluster.Members()
|
||||
pss := make([]peerHashKVResp, len(mbs))
|
||||
for _, m := range mbs {
|
||||
members := s.cluster.Members()
|
||||
peers := make([]peerInfo, 0, len(members))
|
||||
for _, m := range members {
|
||||
if m.ID == s.ID() {
|
||||
continue
|
||||
}
|
||||
pss = append(pss, peerHashKVResp{id: m.ID, eps: m.PeerURLs})
|
||||
peers = append(peers, peerInfo{id: m.ID, eps: m.PeerURLs})
|
||||
}
|
||||
|
||||
lg := s.getLogger()
|
||||
|
||||
for _, p := range pss {
|
||||
var resps []*peerHashKVResp
|
||||
for _, p := range peers {
|
||||
if len(p.eps) == 0 {
|
||||
continue
|
||||
}
|
||||
cli, cerr := clientv3.New(clientv3.Config{
|
||||
DialTimeout: s.Cfg.ReqTimeout(),
|
||||
Endpoints: p.eps,
|
||||
})
|
||||
if cerr != nil {
|
||||
if lg != nil {
|
||||
lg.Warn(
|
||||
"failed to create client to peer URL",
|
||||
zap.String("local-member-id", s.ID().String()),
|
||||
zap.String("remote-peer-id", p.id.String()),
|
||||
zap.Strings("remote-peer-endpoints", p.eps),
|
||||
zap.Error(cerr),
|
||||
)
|
||||
} else {
|
||||
plog.Warningf("%s failed to create client to peer %q for hash checking (%q)", s.ID(), p.eps, cerr.Error())
|
||||
}
|
||||
continue
|
||||
}
|
||||
|
||||
respsLen := len(resps)
|
||||
for _, c := range cli.Endpoints() {
|
||||
var lastErr error
|
||||
for _, ep := range p.eps {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), s.Cfg.ReqTimeout())
|
||||
var resp *clientv3.HashKVResponse
|
||||
resp, cerr = cli.HashKV(ctx, c, rev)
|
||||
|
||||
var resp *pb.HashKVResponse
|
||||
resp, lastErr = s.getPeerHashKVHTTP(ctx, ep, rev)
|
||||
cancel()
|
||||
if cerr == nil {
|
||||
resps = append(resps, &peerHashKVResp{id: p.id, eps: p.eps, resp: resp, err: nil})
|
||||
if lastErr == nil {
|
||||
resps = append(resps, &peerHashKVResp{peerInfo: p, resp: resp, err: nil})
|
||||
break
|
||||
}
|
||||
if lg != nil {
|
||||
@@ -360,17 +361,17 @@ func (s *EtcdServer) getPeerHashKVs(rev int64) (resps []*peerHashKVResp) {
|
||||
"failed hash kv request",
|
||||
zap.String("local-member-id", s.ID().String()),
|
||||
zap.Int64("requested-revision", rev),
|
||||
zap.String("remote-peer-endpoint", c),
|
||||
zap.Error(cerr),
|
||||
zap.String("remote-peer-endpoint", ep),
|
||||
zap.Error(lastErr),
|
||||
)
|
||||
} else {
|
||||
plog.Warningf("%s hash-kv error %q on peer %q with revision %d", s.ID(), cerr.Error(), c, rev)
|
||||
plog.Warningf("%s hash-kv error %q on peer %q with revision %d", s.ID(), lastErr.Error(), ep, rev)
|
||||
}
|
||||
}
|
||||
cli.Close()
|
||||
|
||||
// failed to get hashKV from all endpoints of this peer
|
||||
if respsLen == len(resps) {
|
||||
resps = append(resps, &peerHashKVResp{id: p.id, eps: p.eps, resp: nil, err: cerr})
|
||||
resps = append(resps, &peerHashKVResp{peerInfo: p, resp: nil, err: lastErr})
|
||||
}
|
||||
}
|
||||
return resps
|
||||
@@ -382,11 +383,11 @@ type applierV3Corrupt struct {
|
||||
|
||||
func newApplierV3Corrupt(a applierV3) *applierV3Corrupt { return &applierV3Corrupt{a} }
|
||||
|
||||
func (a *applierV3Corrupt) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, error) {
|
||||
return nil, ErrCorrupt
|
||||
func (a *applierV3Corrupt) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, *traceutil.Trace, error) {
|
||||
return nil, nil, ErrCorrupt
|
||||
}
|
||||
|
||||
func (a *applierV3Corrupt) Range(txn mvcc.TxnRead, p *pb.RangeRequest) (*pb.RangeResponse, error) {
|
||||
func (a *applierV3Corrupt) Range(ctx context.Context, txn mvcc.TxnRead, p *pb.RangeRequest) (*pb.RangeResponse, error) {
|
||||
return nil, ErrCorrupt
|
||||
}
|
||||
|
||||
@@ -398,8 +399,8 @@ func (a *applierV3Corrupt) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) {
|
||||
return nil, ErrCorrupt
|
||||
}
|
||||
|
||||
func (a *applierV3Corrupt) Compaction(compaction *pb.CompactionRequest) (*pb.CompactionResponse, <-chan struct{}, error) {
|
||||
return nil, nil, ErrCorrupt
|
||||
func (a *applierV3Corrupt) Compaction(compaction *pb.CompactionRequest) (*pb.CompactionResponse, <-chan struct{}, *traceutil.Trace, error) {
|
||||
return nil, nil, nil, ErrCorrupt
|
||||
}
|
||||
|
||||
func (a *applierV3Corrupt) LeaseGrant(lc *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error) {
|
||||
@@ -409,3 +410,112 @@ func (a *applierV3Corrupt) LeaseGrant(lc *pb.LeaseGrantRequest) (*pb.LeaseGrantR
|
||||
func (a *applierV3Corrupt) LeaseRevoke(lc *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error) {
|
||||
return nil, ErrCorrupt
|
||||
}
|
||||
|
||||
type ServerPeerV2 interface {
|
||||
ServerPeer
|
||||
HashKVHandler() http.Handler
|
||||
}
|
||||
|
||||
const PeerHashKVPath = "/members/hashkv"
|
||||
|
||||
type hashKVHandler struct {
|
||||
lg *zap.Logger
|
||||
server *EtcdServer
|
||||
}
|
||||
|
||||
func (s *EtcdServer) HashKVHandler() http.Handler {
|
||||
return &hashKVHandler{lg: s.getLogger(), server: s}
|
||||
}
|
||||
|
||||
func (h *hashKVHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||
if r.Method != http.MethodGet {
|
||||
w.Header().Set("Allow", http.MethodGet)
|
||||
http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
|
||||
return
|
||||
}
|
||||
if r.URL.Path != PeerHashKVPath {
|
||||
http.Error(w, "bad path", http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
|
||||
defer r.Body.Close()
|
||||
b, err := ioutil.ReadAll(r.Body)
|
||||
if err != nil {
|
||||
http.Error(w, "error reading body", http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
|
||||
req := &pb.HashKVRequest{}
|
||||
if err = json.Unmarshal(b, req); err != nil {
|
||||
h.lg.Warn("failed to unmarshal request", zap.Error(err))
|
||||
http.Error(w, "error unmarshalling request", http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
hash, rev, compactRev, err := h.server.KV().HashByRev(req.Revision)
|
||||
if err != nil {
|
||||
h.lg.Warn(
|
||||
"failed to get hashKV",
|
||||
zap.Int64("requested-revision", req.Revision),
|
||||
zap.Error(err),
|
||||
)
|
||||
http.Error(w, err.Error(), http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
resp := &pb.HashKVResponse{Header: &pb.ResponseHeader{Revision: rev}, Hash: hash, CompactRevision: compactRev}
|
||||
respBytes, err := json.Marshal(resp)
|
||||
if err != nil {
|
||||
h.lg.Warn("failed to marshal hashKV response", zap.Error(err))
|
||||
http.Error(w, err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
|
||||
w.Header().Set("X-Etcd-Cluster-ID", h.server.Cluster().ID().String())
|
||||
w.Header().Set("Content-Type", "application/json")
|
||||
w.Write(respBytes)
|
||||
}
|
||||
|
||||
// getPeerHashKVHTTP fetch hash of kv store at the given rev via http call to the given url
|
||||
func (s *EtcdServer) getPeerHashKVHTTP(ctx context.Context, url string, rev int64) (*pb.HashKVResponse, error) {
|
||||
cc := &http.Client{Transport: s.peerRt}
|
||||
hashReq := &pb.HashKVRequest{Revision: rev}
|
||||
hashReqBytes, err := json.Marshal(hashReq)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
requestUrl := url + PeerHashKVPath
|
||||
req, err := http.NewRequest(http.MethodGet, requestUrl, bytes.NewReader(hashReqBytes))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
req = req.WithContext(ctx)
|
||||
req.Header.Set("Content-Type", "application/json")
|
||||
req.Cancel = ctx.Done()
|
||||
|
||||
resp, err := cc.Do(req)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer resp.Body.Close()
|
||||
b, err := ioutil.ReadAll(resp.Body)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if resp.StatusCode == http.StatusBadRequest {
|
||||
if strings.Contains(string(b), mvcc.ErrCompacted.Error()) {
|
||||
return nil, rpctypes.ErrCompacted
|
||||
}
|
||||
if strings.Contains(string(b), mvcc.ErrFutureRev.Error()) {
|
||||
return nil, rpctypes.ErrFutureRev
|
||||
}
|
||||
}
|
||||
if resp.StatusCode != http.StatusOK {
|
||||
return nil, fmt.Errorf("unknown error: %s", string(b))
|
||||
}
|
||||
|
||||
hashResp := &pb.HashKVResponse{}
|
||||
if err := json.Unmarshal(b, hashResp); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return hashResp, nil
|
||||
}
|
||||
|
@@ -207,7 +207,7 @@ func monitorFileDescriptor(lg *zap.Logger, done <-chan struct{}) {
|
||||
}
|
||||
if used >= limit/5*4 {
|
||||
if lg != nil {
|
||||
lg.Warn("80%% of file descriptors are used", zap.Uint64("used", used), zap.Uint64("limit", limit))
|
||||
lg.Warn("80% of file descriptors are used", zap.Uint64("used", used), zap.Uint64("limit", limit))
|
||||
} else {
|
||||
plog.Warningf("80%% of the file descriptor limit is used [used = %d, limit = %d]", used, limit)
|
||||
}
|
||||
|
@@ -50,6 +50,7 @@ import (
|
||||
"go.etcd.io/etcd/pkg/pbutil"
|
||||
"go.etcd.io/etcd/pkg/runtime"
|
||||
"go.etcd.io/etcd/pkg/schedule"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
"go.etcd.io/etcd/pkg/types"
|
||||
"go.etcd.io/etcd/pkg/wait"
|
||||
"go.etcd.io/etcd/raft"
|
||||
@@ -785,7 +786,7 @@ func (s *EtcdServer) start() {
|
||||
} else {
|
||||
plog.Infof("starting server... [version: %v, cluster version: %v]", version.Version, version.Cluster(s.ClusterVersion().String()))
|
||||
}
|
||||
membership.ClusterVersionMetrics.With(prometheus.Labels{"cluster_version": s.ClusterVersion().String()}).Set(1)
|
||||
membership.ClusterVersionMetrics.With(prometheus.Labels{"cluster_version": version.Cluster(s.ClusterVersion().String())}).Set(1)
|
||||
} else {
|
||||
if lg != nil {
|
||||
lg.Info(
|
||||
@@ -806,12 +807,13 @@ func (s *EtcdServer) start() {
|
||||
|
||||
func (s *EtcdServer) purgeFile() {
|
||||
var dberrc, serrc, werrc <-chan error
|
||||
var dbdonec, sdonec, wdonec <-chan struct{}
|
||||
if s.Cfg.MaxSnapFiles > 0 {
|
||||
dberrc = fileutil.PurgeFile(s.getLogger(), s.Cfg.SnapDir(), "snap.db", s.Cfg.MaxSnapFiles, purgeFileInterval, s.done)
|
||||
serrc = fileutil.PurgeFile(s.getLogger(), s.Cfg.SnapDir(), "snap", s.Cfg.MaxSnapFiles, purgeFileInterval, s.done)
|
||||
dbdonec, dberrc = fileutil.PurgeFileWithDoneNotify(s.getLogger(), s.Cfg.SnapDir(), "snap.db", s.Cfg.MaxSnapFiles, purgeFileInterval, s.stopping)
|
||||
sdonec, serrc = fileutil.PurgeFileWithDoneNotify(s.getLogger(), s.Cfg.SnapDir(), "snap", s.Cfg.MaxSnapFiles, purgeFileInterval, s.stopping)
|
||||
}
|
||||
if s.Cfg.MaxWALFiles > 0 {
|
||||
werrc = fileutil.PurgeFile(s.getLogger(), s.Cfg.WALDir(), "wal", s.Cfg.MaxWALFiles, purgeFileInterval, s.done)
|
||||
wdonec, werrc = fileutil.PurgeFileWithDoneNotify(s.getLogger(), s.Cfg.WALDir(), "wal", s.Cfg.MaxWALFiles, purgeFileInterval, s.stopping)
|
||||
}
|
||||
|
||||
lg := s.getLogger()
|
||||
@@ -835,6 +837,15 @@ func (s *EtcdServer) purgeFile() {
|
||||
plog.Fatalf("failed to purge wal file %v", e)
|
||||
}
|
||||
case <-s.stopping:
|
||||
if dbdonec != nil {
|
||||
<-dbdonec
|
||||
}
|
||||
if sdonec != nil {
|
||||
<-sdonec
|
||||
}
|
||||
if wdonec != nil {
|
||||
<-wdonec
|
||||
}
|
||||
return
|
||||
}
|
||||
}
|
||||
@@ -1178,7 +1189,7 @@ func (s *EtcdServer) applySnapshot(ep *etcdProgress, apply *apply) {
|
||||
plog.Info("recovering lessor...")
|
||||
}
|
||||
|
||||
s.lessor.Recover(newbe, func() lease.TxnDelete { return s.kv.Write() })
|
||||
s.lessor.Recover(newbe, func() lease.TxnDelete { return s.kv.Write(traceutil.TODO()) })
|
||||
|
||||
if lg != nil {
|
||||
lg.Info("restored lease store")
|
||||
|
@@ -26,6 +26,7 @@ import (
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/lease/leasehttp"
|
||||
"go.etcd.io/etcd/mvcc"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
"go.etcd.io/etcd/raft"
|
||||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
@@ -38,6 +39,7 @@ const (
|
||||
// However, if the committed entries are very heavy to apply, the gap might grow.
|
||||
// We should stop accepting new proposals if the gap growing to a certain point.
|
||||
maxGapBetweenApplyAndCommitIndex = 5000
|
||||
traceThreshold = 100 * time.Millisecond
|
||||
)
|
||||
|
||||
type RaftKV interface {
|
||||
@@ -85,14 +87,29 @@ type Authenticator interface {
|
||||
}
|
||||
|
||||
func (s *EtcdServer) Range(ctx context.Context, r *pb.RangeRequest) (*pb.RangeResponse, error) {
|
||||
trace := traceutil.New("range",
|
||||
s.getLogger(),
|
||||
traceutil.Field{Key: "range_begin", Value: string(r.Key)},
|
||||
traceutil.Field{Key: "range_end", Value: string(r.RangeEnd)},
|
||||
)
|
||||
ctx = context.WithValue(ctx, traceutil.TraceKey, trace)
|
||||
|
||||
var resp *pb.RangeResponse
|
||||
var err error
|
||||
defer func(start time.Time) {
|
||||
warnOfExpensiveReadOnlyRangeRequest(s.getLogger(), start, r, resp, err)
|
||||
if resp != nil {
|
||||
trace.AddField(
|
||||
traceutil.Field{Key: "response_count", Value: len(resp.Kvs)},
|
||||
traceutil.Field{Key: "response_revision", Value: resp.Header.Revision},
|
||||
)
|
||||
}
|
||||
trace.LogIfLong(traceThreshold)
|
||||
}(time.Now())
|
||||
|
||||
if !r.Serializable {
|
||||
err = s.linearizableReadNotify(ctx)
|
||||
trace.Step("agreement among raft nodes before linearized reading")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -101,7 +118,7 @@ func (s *EtcdServer) Range(ctx context.Context, r *pb.RangeRequest) (*pb.RangeRe
|
||||
return s.authStore.IsRangePermitted(ai, r.Key, r.RangeEnd)
|
||||
}
|
||||
|
||||
get := func() { resp, err = s.applyV3Base.Range(nil, r) }
|
||||
get := func() { resp, err = s.applyV3Base.Range(ctx, nil, r) }
|
||||
if serr := s.doSerialize(ctx, chk, get); serr != nil {
|
||||
err = serr
|
||||
return nil, err
|
||||
@@ -110,6 +127,7 @@ func (s *EtcdServer) Range(ctx context.Context, r *pb.RangeRequest) (*pb.RangeRe
|
||||
}
|
||||
|
||||
func (s *EtcdServer) Put(ctx context.Context, r *pb.PutRequest) (*pb.PutResponse, error) {
|
||||
ctx = context.WithValue(ctx, traceutil.StartTimeKey, time.Now())
|
||||
resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{Put: r})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -186,7 +204,18 @@ func isTxnReadonly(r *pb.TxnRequest) bool {
|
||||
}
|
||||
|
||||
func (s *EtcdServer) Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.CompactionResponse, error) {
|
||||
startTime := time.Now()
|
||||
result, err := s.processInternalRaftRequestOnce(ctx, pb.InternalRaftRequest{Compaction: r})
|
||||
trace := traceutil.TODO()
|
||||
if result != nil && result.trace != nil {
|
||||
trace = result.trace
|
||||
defer func() {
|
||||
trace.LogIfLong(traceThreshold)
|
||||
}()
|
||||
applyStart := result.trace.GetStartTime()
|
||||
result.trace.SetStartTime(startTime)
|
||||
trace.InsertStep(0, applyStart, "process raft request")
|
||||
}
|
||||
if r.Physical && result != nil && result.physc != nil {
|
||||
<-result.physc
|
||||
// The compaction is done deleting keys; the hash is now settled
|
||||
@@ -195,6 +224,7 @@ func (s *EtcdServer) Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.
|
||||
// if the compaction resumes. Force the finished compaction to
|
||||
// commit so it won't resume following a crash.
|
||||
s.be.ForceCommit()
|
||||
trace.Step("physically apply compaction")
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -210,6 +240,7 @@ func (s *EtcdServer) Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.
|
||||
resp.Header = &pb.ResponseHeader{}
|
||||
}
|
||||
resp.Header.Revision = s.kv.Rev()
|
||||
trace.AddField(traceutil.Field{Key: "response_revision", Value: resp.Header.Revision})
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
@@ -533,20 +564,25 @@ func (s *EtcdServer) raftRequestOnce(ctx context.Context, r pb.InternalRaftReque
|
||||
if result.err != nil {
|
||||
return nil, result.err
|
||||
}
|
||||
if startTime, ok := ctx.Value(traceutil.StartTimeKey).(time.Time); ok && result.trace != nil {
|
||||
applyStart := result.trace.GetStartTime()
|
||||
// The trace object is created in apply. Here reset the start time to trace
|
||||
// the raft request time by the difference between the request start time
|
||||
// and apply start time
|
||||
result.trace.SetStartTime(startTime)
|
||||
result.trace.InsertStep(0, applyStart, "process raft request")
|
||||
result.trace.LogIfLong(traceThreshold)
|
||||
}
|
||||
return result.resp, nil
|
||||
}
|
||||
|
||||
func (s *EtcdServer) raftRequest(ctx context.Context, r pb.InternalRaftRequest) (proto.Message, error) {
|
||||
for {
|
||||
resp, err := s.raftRequestOnce(ctx, r)
|
||||
if err != auth.ErrAuthOldRevision {
|
||||
return resp, err
|
||||
}
|
||||
}
|
||||
return s.raftRequestOnce(ctx, r)
|
||||
}
|
||||
|
||||
// doSerialize handles the auth logic, with permissions checked by "chk", for a serialized request "get". Returns a non-nil error on authentication failure.
|
||||
func (s *EtcdServer) doSerialize(ctx context.Context, chk func(*auth.AuthInfo) error, get func()) error {
|
||||
trace := traceutil.Get(ctx)
|
||||
ai, err := s.AuthInfoFromCtx(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
@@ -558,6 +594,7 @@ func (s *EtcdServer) doSerialize(ctx context.Context, chk func(*auth.AuthInfo) e
|
||||
if err = chk(ai); err != nil {
|
||||
return err
|
||||
}
|
||||
trace.Step("get authentication metadata")
|
||||
// fetch response for serialized request
|
||||
get()
|
||||
// check for stale token revision in case the auth store was updated while
|
||||
|
@@ -13,7 +13,7 @@ if ! [[ "${0}" =~ "scripts/docker-local-agent.sh" ]]; then
|
||||
fi
|
||||
|
||||
if [[ -z "${GO_VERSION}" ]]; then
|
||||
GO_VERSION=1.12.9
|
||||
GO_VERSION=1.12.12
|
||||
fi
|
||||
echo "Running with GO_VERSION:" ${GO_VERSION}
|
||||
|
||||
|
@@ -6,7 +6,7 @@ if ! [[ "${0}" =~ "scripts/docker-local-tester.sh" ]]; then
|
||||
fi
|
||||
|
||||
if [[ -z "${GO_VERSION}" ]]; then
|
||||
GO_VERSION=1.12.9
|
||||
GO_VERSION=1.12.12
|
||||
fi
|
||||
echo "Running with GO_VERSION:" ${GO_VERSION}
|
||||
|
||||
|
5
go.mod
5
go.mod
@@ -40,9 +40,10 @@ require (
|
||||
go.uber.org/multierr v1.1.0 // indirect
|
||||
go.uber.org/zap v1.10.0
|
||||
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2
|
||||
golang.org/x/net v0.0.0-20190813000000-74dc4d7220e7
|
||||
golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7
|
||||
golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456 // indirect
|
||||
golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2
|
||||
google.golang.org/grpc v1.23.0
|
||||
google.golang.org/grpc v1.23.1
|
||||
gopkg.in/cheggaaa/pb.v1 v1.0.25
|
||||
gopkg.in/yaml.v2 v2.2.2
|
||||
sigs.k8s.io/yaml v1.1.0
|
||||
|
10
go.sum
10
go.sum
@@ -153,8 +153,8 @@ golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73r
|
||||
golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||
golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628=
|
||||
golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||
golang.org/x/net v0.0.0-20190813000000-74dc4d7220e7 h1:HOTjhHFecQCpFnEhQ4MAH6Gf7yGklZnqaHvtezTKqnQ=
|
||||
golang.org/x/net v0.0.0-20190813000000-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||
golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7 h1:fHDIZ2oxGnUZRN6WgWFCbYBjH9uqVPRCUVUDhs0wnbA=
|
||||
golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
|
||||
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
@@ -167,6 +167,8 @@ golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5 h1:mzjBh+S5frKOsOBobWIMAbXav
|
||||
golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a h1:1BGLXjeY4akVXGgbC9HugT3Jv3hCI0z56oJR5vAMgBU=
|
||||
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||
golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456 h1:ng0gs1AKnRRuEMZoTLLlbOd+C17zUDepwGQBb/n+JVg=
|
||||
golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg=
|
||||
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
||||
golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2 h1:+DCIGbF/swA92ohVg0//6X2IVY3KZs6p9mix0ziNYJM=
|
||||
@@ -179,8 +181,8 @@ google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9Ywl
|
||||
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8 h1:Nw54tB0rB7hY/N0NQvRW8DG4Yk3Q6T9cu9RcFQDu1tc=
|
||||
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
|
||||
google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
|
||||
google.golang.org/grpc v1.23.0 h1:AzbTB6ux+okLTzP8Ru1Xs41C303zdcfEht7MQnYJt5A=
|
||||
google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
|
||||
google.golang.org/grpc v1.23.1 h1:q4XQuHFC6I28BKZpo6IYyb3mNO+l7lSOxRuYTCiDfXk=
|
||||
google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
|
||||
gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
|
||||
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM=
|
||||
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||
|
@@ -1,22 +1,22 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIDrjCCApagAwIBAgIUM24Z44NdsHtDQisQRIH+mmhXLHYwDQYJKoZIhvcNAQEL
|
||||
MIIDrjCCApagAwIBAgIUOl7DCgSvqQKhiihYrZDiBKNpQX4wDQYJKoZIhvcNAQEL
|
||||
BQAwbzEMMAoGA1UEBhMDVVNBMRMwEQYDVQQIEwpDYWxpZm9ybmlhMRYwFAYDVQQH
|
||||
Ew1TYW4gRnJhbmNpc2NvMQ0wCwYDVQQKEwRldGNkMRYwFAYDVQQLEw1ldGNkIFNl
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTAxMjExNDQwMDBaFw0yOTAxMTgxNDQw
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTEwMDgyMTE5MDBaFw0yOTEwMDUyMTE5
|
||||
MDBaMG8xDDAKBgNVBAYTA1VTQTETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UE
|
||||
BxMNU2FuIEZyYW5jaXNjbzENMAsGA1UEChMEZXRjZDEWMBQGA1UECxMNZXRjZCBT
|
||||
ZWN1cml0eTELMAkGA1UEAxMCY2EwggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEK
|
||||
AoIBAQCsWG1qafiCwfmKEltvpmslNqOlWgMp9H7VIP7cExbhsW4P1L4Jlfcz7rFH
|
||||
2MFpwktbxppoYI/4umTj5r7dx/K6mttUBtiLY5VwSCo/asZvLaOLFN2QP4cwkpLI
|
||||
lFDy4Pez2Uu+NnmMF6SLq+M6mOaHSbURNvphP1zWX9SLKo1OV8GT6r/oHYmcR+xy
|
||||
skWd/+6B73S0pbG/d3ME/WoovZtOXqaZtJn8YIBXE6LGd4NBkSK3Jg9c4QzlErTM
|
||||
j6ItTs7t9aPjXd2kiq8IY6UN2TrLwssWkGM4Oop+mlp5zcKIDLhDrfsRga5hxx2Z
|
||||
i0coNWBKNjvVaCO0L7Qn1nIHA1KtAgMBAAGjQjBAMA4GA1UdDwEB/wQEAwIBBjAP
|
||||
BgNVHRMBAf8EBTADAQH/MB0GA1UdDgQWBBTdZTAnocY85lHhyR/A7lJkp3t2mjAN
|
||||
BgkqhkiG9w0BAQsFAAOCAQEABCTckIi6zoE7uSy71uNO93RC/Pcb+YzmRSNQzl60
|
||||
ngUlUrd+18bjp3O9u8jQ8ikhWT3jfn5e4I1nqLKFqKP6xyMPwk2ZJXF3WeBvtuHW
|
||||
BonDscbYwMpL6RDgcUU1+2ZtZYlo+NZkeXQdTO0Pa8qoo/EtNXb+Bg1FFqnrLrVI
|
||||
EhY3Bd5+jvC0WkjJFMFeOUkZDmtKLX24P/901ZP+6HN2bA+MIBKmIDKbctP54J73
|
||||
tncuOOFBfyWkckIMISM4D+Mi9Ezju2Hq4thV7XJeyWTRiXG8+LhVRWJaz7St1FIw
|
||||
ViEST3A84CBLjiPyGqzqQCtr+HNhr7su+Tmcq550xU11Ug==
|
||||
AoIBAQDBNhwKD8oqOwNSDMZR+K6l6ocyXZzZPAIbv7co34xtjt25c8PPKz8FiBSU
|
||||
M4YeZpzsSp7n7WSSSzVWqFTRBZzvjIrBzLu4CfxMKuUrQX1/BPYgbSxQO+5YKPzO
|
||||
yaBMhIAEtW+WYsaa6PpWyL65L4giKpVoLS/UFTEBsf+lO6pwFpX2EJnIylLbpwEd
|
||||
pAXIgVFsodHlP9Zc2tR1TqYetmJ6/A/p5sSZpgLy1y2+Mg4VTMKvs2kNAoh/+lEu
|
||||
WPe204eMpkBXhukulOiJkVKNdhnCkLslt8ZaMWWqBvD9d94lXycMQ9wnGakPNc4W
|
||||
5VX3rbLOGOX7xK37BCsh5HGodIrZAgMBAAGjQjBAMA4GA1UdDwEB/wQEAwIBBjAP
|
||||
BgNVHRMBAf8EBTADAQH/MB0GA1UdDgQWBBRlB76vjaZyFLrEUGm6DQfyjmN6PjAN
|
||||
BgkqhkiG9w0BAQsFAAOCAQEAD0cRNBQqOPNAUmKCH9xCr4TZFoE+P5aNePU39Jyp
|
||||
qpJ1HjKI93zBk9aN5udDGPFhm2/iaKx6DuABbxCz0LwNhLiKP6UbHV8F2fTJJ5bo
|
||||
crXvD0CEpor+Quh995lbq9bv29+zcDVw+Hw0QainBdHWkdw6RAgmbFnJxETDDz8z
|
||||
VQ0DET3T736oxpEZ4DKQlbzK5LSgZH2lyPEEvzci4QjTZf5X/nitdx7fAdMFFPQ0
|
||||
lI4l7nIuge5LTR0isEfWHx7Orx6l8dzkofG3fz5BjHCI4JInVlWq3MNNSybDI4pI
|
||||
GFxeuE/U8K6kIixT8qCAh6Naq9/xuxFkffLmMKfZXoYLCg==
|
||||
-----END CERTIFICATE-----
|
||||
|
@@ -43,6 +43,15 @@ cfssl gencert \
|
||||
mv server-ip.pem server-ip.crt
|
||||
mv server-ip-key.pem server-ip.key.insecure
|
||||
|
||||
# generate IPv6: [::1], CN: example.com certificates
|
||||
cfssl gencert \
|
||||
--ca ./ca.crt \
|
||||
--ca-key ./ca-key.pem \
|
||||
--config ./gencert.json \
|
||||
./server-ca-csr-ipv6.json | cfssljson --bare ./server-ip
|
||||
mv server-ip.pem server-ipv6.crt
|
||||
mv server-ip-key.pem server-ipv6.key.insecure
|
||||
|
||||
# generate DNS: localhost, IP: 127.0.0.1, CN: example2.com certificates
|
||||
cfssl gencert \
|
||||
--ca ./ca.crt \
|
||||
|
Binary file not shown.
19
integration/fixtures/server-ca-csr-ipv6.json
Normal file
19
integration/fixtures/server-ca-csr-ipv6.json
Normal file
@@ -0,0 +1,19 @@
|
||||
{
|
||||
"key": {
|
||||
"algo": "rsa",
|
||||
"size": 2048
|
||||
},
|
||||
"names": [
|
||||
{
|
||||
"O": "etcd",
|
||||
"OU": "etcd Security",
|
||||
"L": "San Francisco",
|
||||
"ST": "California",
|
||||
"C": "USA"
|
||||
}
|
||||
],
|
||||
"CN": "example.com",
|
||||
"hosts": [
|
||||
"::1"
|
||||
]
|
||||
}
|
@@ -1,20 +1,20 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIDRzCCAi+gAwIBAgIUMEJ5c+Tt0TyOcLL+dIUuE2nOuukwDQYJKoZIhvcNAQEL
|
||||
MIIDRzCCAi+gAwIBAgIUKgQJ/CMaFxc4JcwwGyiT/7KpedIwDQYJKoZIhvcNAQEL
|
||||
BQAwbzEMMAoGA1UEBhMDVVNBMRMwEQYDVQQIEwpDYWxpZm9ybmlhMRYwFAYDVQQH
|
||||
Ew1TYW4gRnJhbmNpc2NvMQ0wCwYDVQQKEwRldGNkMRYwFAYDVQQLEw1ldGNkIFNl
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTAxMjExNDQwMDBaFw0yOTAxMTgxNDQw
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTEwMDgyMTE5MDBaFw0yOTEwMDUyMTE5
|
||||
MDBaMHgxDDAKBgNVBAYTA1VTQTETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UE
|
||||
BxMNU2FuIEZyYW5jaXNjbzENMAsGA1UEChMEZXRjZDEWMBQGA1UECxMNZXRjZCBT
|
||||
ZWN1cml0eTEUMBIGA1UEAxMLZXhhbXBsZS5jb20wWTATBgcqhkjOPQIBBggqhkjO
|
||||
PQMBBwNCAAS/sa5Guqq+tML3vUYITeBQ7gURu5yJa0gKALVIpQ75AJXG9fp1dMD1
|
||||
+3M7HiT56p5omwDPqe8zFsCvPSm6TSEPo4GcMIGZMA4GA1UdDwEB/wQEAwIFoDAd
|
||||
PQMBBwNCAARXbc8naiFZ3Y2LujrnDCScVNRks/TR+aXPmnuPGjDxbuHxSSbC8Q2z
|
||||
iTvCkgsIcsifmUIEQcI4v3Kbkj3qMF1so4GcMIGZMA4GA1UdDwEB/wQEAwIFoDAd
|
||||
BgNVHSUEFjAUBggrBgEFBQcDAQYIKwYBBQUHAwIwDAYDVR0TAQH/BAIwADAdBgNV
|
||||
HQ4EFgQUOErhH7Ot6qYob29cUsijrKhSGawwHwYDVR0jBBgwFoAU3WUwJ6HGPOZR
|
||||
4ckfwO5SZKd7dpowGgYDVR0RBBMwEYIJbG9jYWxob3N0hwR/AAABMA0GCSqGSIb3
|
||||
DQEBCwUAA4IBAQBOjMqQ2AGHTOvHiG1eumKDaxGzXGb7znMcnpKYuz0OT97IoZSw
|
||||
EggwwUbqaK+9DotDcAWaqkReP18P3T9TgzZMfFDFctSKB5rM4EU2iPpAHdA6EEB8
|
||||
87HutlAeFphjsRlUMRLZ2YvTutR0jVeniEDTmTUB9crhGuUrCbg5H8jsVjvDKDut
|
||||
si3l6jsm598EWYa2P7ac5/MXQ5/Z9QCMogE/zOPzbnHNuAbf6ZdGFHNM6cgUgHvs
|
||||
C8L6hnuOCouFfcNDRK+7WjpIde18LNwLC0AwCKXbwFdWErRWJ8W978t6htdBYS9p
|
||||
cvvxQXBuMRmAykhjKaE+rvjdV3IJqQU8mGLY
|
||||
HQ4EFgQU3z1DifT82BfoU5DfMe08meeYmSUwHwYDVR0jBBgwFoAUZQe+r42mchS6
|
||||
xFBpug0H8o5jej4wGgYDVR0RBBMwEYIJbG9jYWxob3N0hwR/AAABMA0GCSqGSIb3
|
||||
DQEBCwUAA4IBAQAE3bhZcJuGrnMGMgebCFMuAXvoF9twYIHXpxNOg6u0HTIWOsMB
|
||||
njEJW/rfZFE/RAJ6JdOMNE2bq2LbJ8dUA25PX3uz6V4omm9B3EvEG9Hh3J+C77XQ
|
||||
P+ofiUd+j06SdewoxrmmQmjZZdotpFUQG3EEncs+v94jsamwGNLdq4yWDjFdmyuC
|
||||
hqzSkD48aGqP2Q93wfv8uIiCEmJS1vITTm2LxssCLfiYGortpCx32/DWme8nUlni
|
||||
1U/pRTx8Brx00dMeruTGjCCpwb8k453oNV6u0D1LsQ9y5DuyEwmZtBEHBN1kVPro
|
||||
yYW3/b1jcmZk8W9GXqcXy16LbWmpvJmTHPsj
|
||||
-----END CERTIFICATE-----
|
||||
|
@@ -1,5 +1,5 @@
|
||||
-----BEGIN EC PRIVATE KEY-----
|
||||
MHcCAQEEINToOjKwxXFyCQHkiWoL55IPdPoYhm1TFmDylAUIhJWZoAoGCCqGSM49
|
||||
AwEHoUQDQgAEv7GuRrqqvrTC971GCE3gUO4FEbuciWtICgC1SKUO+QCVxvX6dXTA
|
||||
9ftzOx4k+eqeaJsAz6nvMxbArz0puk0hDw==
|
||||
MHcCAQEEIK3K2gimOw2P0pZ4soFAopriuORuqpRptllFXNRhCRV0oAoGCCqGSM49
|
||||
AwEHoUQDQgAEV23PJ2ohWd2Ni7o65wwknFTUZLP00fmlz5p7jxow8W7h8UkmwvEN
|
||||
s4k7wpILCHLIn5lCBEHCOL9ym5I96jBdbA==
|
||||
-----END EC PRIVATE KEY-----
|
||||
|
@@ -1,24 +1,24 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIEBzCCAu+gAwIBAgIUGZReOLZEaMEZ2PfqR25XMrEdBIMwDQYJKoZIhvcNAQEL
|
||||
MIIEBzCCAu+gAwIBAgIUSvxuG1lgImYpnaK4sPaCiMAd0lgwDQYJKoZIhvcNAQEL
|
||||
BQAwbzEMMAoGA1UEBhMDVVNBMRMwEQYDVQQIEwpDYWxpZm9ybmlhMRYwFAYDVQQH
|
||||
Ew1TYW4gRnJhbmNpc2NvMQ0wCwYDVQQKEwRldGNkMRYwFAYDVQQLEw1ldGNkIFNl
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTAxMjExNDQwMDBaFw0yOTAxMTgxNDQw
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTEwMDgyMTE5MDBaFw0yOTEwMDUyMTE5
|
||||
MDBaMHgxDDAKBgNVBAYTA1VTQTETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UE
|
||||
BxMNU2FuIEZyYW5jaXNjbzENMAsGA1UEChMEZXRjZDEWMBQGA1UECxMNZXRjZCBT
|
||||
ZWN1cml0eTEUMBIGA1UEAxMLZXhhbXBsZS5jb20wggEiMA0GCSqGSIb3DQEBAQUA
|
||||
A4IBDwAwggEKAoIBAQDPXZtt7WyGbsRRzkKHTKqeqAGaFdVJpeLCiLpq15hGiOGl
|
||||
RJFFrQ/SxEH2y8CmoKLcY96uKYxzVWHpPStK0wa/3DMTE0sxhdWFixD/eRTNgA/o
|
||||
ovvSEPTX/ya//DfrgvrKNeSCG/E3hDXitVdiexeUiIB8DZHwdAg82Zg9eJ41ck+G
|
||||
WD9u//PwUqS8epqs15xXaHMQphjATnkLa/0mIjwo6JPddtGopBQRADaorjvpaoUu
|
||||
SzL7TQaHzCVuj47szr7BmNK1mwoHXJk7d+BlBJz6SiSGBLLq3h3SmoX9/yDLNU7t
|
||||
rCi9Yl55/ITJEfY56ZZ6L/BLm83b9r03lv1vYYr/AgMBAAGjgZEwgY4wDgYDVR0P
|
||||
A4IBDwAwggEKAoIBAQC7mJOiyqWfmNM5ptQZ22plotVfgoBf9fHTzMw/ap2Vl0/0
|
||||
4V3GEyYCdPt6V87GWzjBSO9GAmlISBQQybMieZTaTm8KKW2066iJDKseBCv9m4nS
|
||||
mHv0oDqp3SHsZQ2xHis4lbi7ws2thdqpmjw4Dv96SUiCJUjhcBX4kBMRcOGgk1RF
|
||||
ENIOInTSKlAiwNF1NSnhj8wMNw7mjw90jpAGAuPuuiQ7+AYHJBJqtT9mRikR8ppw
|
||||
isjEE6kslCCg2RC45AiF4LXNp7A7Xwm6P34XJ6T9PJUh/r3pa0xHRuI2zQLaW8Z/
|
||||
b6NYkUGMbHR7AY/+2JzOfnnnQcSB8EYC9bHadvHnAgMBAAGjgZEwgY4wDgYDVR0P
|
||||
AQH/BAQDAgWgMB0GA1UdJQQWMBQGCCsGAQUFBwMBBggrBgEFBQcDAjAMBgNVHRMB
|
||||
Af8EAjAAMB0GA1UdDgQWBBRn6Ksd8Ra9fiY7fCZnmq2OylkQmTAfBgNVHSMEGDAW
|
||||
gBTdZTAnocY85lHhyR/A7lJkp3t2mjAPBgNVHREECDAGhwR/AAABMA0GCSqGSIb3
|
||||
DQEBCwUAA4IBAQCj+s26SaG99nC/OAHJtXQqxyqDfoKNwO6iwK4UGGzwlKAK9+a7
|
||||
8ObVOIyAbFtHUFjFJ6cIBMg+Tw+9++bRPyliOcrIDiv9ytNEzMVIQq07oj0Kx7Qw
|
||||
sSYcIeFRF439ftiHC0LAULFEV/hDBveuTVfdt3t5RnEzp8PiTjXvhSpgFOhkuln8
|
||||
n3NK4UoolN6gJ/sSCP91Oka90l4xagPYK37mksYfzbTBNmPB88rMgioee5ZU1nCG
|
||||
09fHiNWg/U8c0R6Iflpjy3lsUlnst3+VZp7HZ1mO+hBp7p0lduAdJqZs0ev9gjhi
|
||||
odfA+/2O8LPUTTXz6lpdnou2kXl0B4I0jG2v
|
||||
Af8EAjAAMB0GA1UdDgQWBBSPaFA2Jh7s/IJN/Yw/QFFR4pO3nDAfBgNVHSMEGDAW
|
||||
gBRlB76vjaZyFLrEUGm6DQfyjmN6PjAPBgNVHREECDAGhwR/AAABMA0GCSqGSIb3
|
||||
DQEBCwUAA4IBAQAO2EnUXDlZAzOJLmkzQQF/d88PjvzspFtBfj/jCGzK6bpjeZwq
|
||||
oM1fQOkjuFeNvVLA3WHVT0XEpZEM8lwAr/YwnBWMFlNd3Vb2Cho5VaQq0nVfhYoB
|
||||
tpzoWcf0Qx4cALesQZ3y2EnXePpzky1R4MfHqulYrmZKSBQsERob/7YgSBk+ucV9
|
||||
OHLzYxm4OvYvDoR54REq+vgZ3ohoDmBrNNv9OmUHLIrUi+nBpBgnww85Dc7cKB27
|
||||
EEKxqIfCNTeHSemvzfK/1M6manQX6eyGe48nOwQMV/ocfY6SeA7RABT0l/UsbeMp
|
||||
g/b2RU+liZ3e8FziW4/1VTt1pmFAN/2hnb0v
|
||||
-----END CERTIFICATE-----
|
||||
|
@@ -1,27 +1,27 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIEpAIBAAKCAQEAz12bbe1shm7EUc5Ch0yqnqgBmhXVSaXiwoi6ateYRojhpUSR
|
||||
Ra0P0sRB9svApqCi3GPerimMc1Vh6T0rStMGv9wzExNLMYXVhYsQ/3kUzYAP6KL7
|
||||
0hD01/8mv/w364L6yjXkghvxN4Q14rVXYnsXlIiAfA2R8HQIPNmYPXieNXJPhlg/
|
||||
bv/z8FKkvHqarNecV2hzEKYYwE55C2v9JiI8KOiT3XbRqKQUEQA2qK476WqFLksy
|
||||
+00Gh8wlbo+O7M6+wZjStZsKB1yZO3fgZQSc+kokhgSy6t4d0pqF/f8gyzVO7awo
|
||||
vWJeefyEyRH2OemWei/wS5vN2/a9N5b9b2GK/wIDAQABAoIBABMnzIHdGtdYSB5e
|
||||
dVrWRDSfxHYdajSBdG8P/lh8Tf7GCkIgEWNkVz/bDVTlAYji8eh1+U4RXH4S9xZ6
|
||||
phMlZ0w15Snv8FREzrKlZA6Vesx49f9Bfw2qr1N8qHG3tNq2oMApNlCmkCPWvLuS
|
||||
kN3yDP2VlnjfMAoMTe6BE0UqbUL2f58h2LBo8x04FYRye5VtroO+lM/9cHUB0xAJ
|
||||
Lsp7hiu5wWsORyn6pkpC5B/Fw2sqOdEKtImcs9CkimkbZAfQXq97Rd/jAkAoVaEB
|
||||
j4lKri9bKwiTH7Xi9LyH8Ix445lqC3uT7728yltTiyPHME0O6RsP5AdfZRmGCrEg
|
||||
cMuplFECgYEA1gUmENbRx6zQdQ0CiwgRloRzceLd3l0lJCxaohqKfX28/keBjMJY
|
||||
BB/EM+Qv+aBwrOshSAAW6EttBdqKMs3/EIngCpvMQfAKN7flECqfPZpGJX5BV+E/
|
||||
51E4wjwCrxpmkYlQ8qj7lt8wcNPU55Tw9bfKelEgQ+dfyD6TpWV9pnsCgYEA+ApP
|
||||
XfGsSkeaFsx6LJIy3ptYxrcd4Xus7ELv6ifTmCKsgi83OeTzrEccHF1niw/9RU6P
|
||||
FYH3jxUzGe1AxBfcAcgA8u4L6+Z/uHtnfU+y8od0H1kHwzAR90Rgc0/sCLlHo93+
|
||||
4yDPpS3JsLLrhrn6e0uHwEOTyoecyIXcyEL16E0CgYEAiH5uIY0v633u0MgEWDFE
|
||||
Lk+45OhAgiG7n09eWkY9Dv3TPATUvbXwtmigFEwywKyvT8kBx86uzWXVWUdgnjg8
|
||||
tQqJxZpJccAqdBCnWWElf/9VP3I/MFHrFJb7cP0e5RgcVDNUWf6lvjoHxd2DylJ2
|
||||
PvABhXMZ9dSphKdMOM76jOMCgYEAzPlIKSwj4qZVEe4cMGUIoKjjriN5D/LyLbQL
|
||||
KweKdjiBMnvuOWuYao/BDTeq72JhPDr1RyLF/3nXZt+HHAVTjC1Ji3doZqNufHeO
|
||||
SCHqkT2amqUqIwTAdAQPaHttZLAoIaS8k9lzft7dw6W3uPhLpEQAhMPTiBSVXagx
|
||||
kVS0fikCgYAhgIEyMauBbFeEA4kR8AuvpGfB8nMxJV7wljxGzLV1FJxmMsB+nkcE
|
||||
BgUUkMLRmqqbqfyTj4p8zNtsnjeNd6ka4Un9uYJoxLpbHhe7NxqGiMGaUMPszaKV
|
||||
/Q7vs7YissxWZxhhoTCMkd//YNikdHBUeMTYIC3CS6ahgX7+ueydhg==
|
||||
MIIEowIBAAKCAQEAu5iTosqln5jTOabUGdtqZaLVX4KAX/Xx08zMP2qdlZdP9OFd
|
||||
xhMmAnT7elfOxls4wUjvRgJpSEgUEMmzInmU2k5vCilttOuoiQyrHgQr/ZuJ0ph7
|
||||
9KA6qd0h7GUNsR4rOJW4u8LNrYXaqZo8OA7/eklIgiVI4XAV+JATEXDhoJNURRDS
|
||||
DiJ00ipQIsDRdTUp4Y/MDDcO5o8PdI6QBgLj7rokO/gGByQSarU/ZkYpEfKacIrI
|
||||
xBOpLJQgoNkQuOQIheC1zaewO18Juj9+Fyek/TyVIf696WtMR0biNs0C2lvGf2+j
|
||||
WJFBjGx0ewGP/ticzn5550HEgfBGAvWx2nbx5wIDAQABAoIBAB0jBpM7TFwsfWov
|
||||
6jOV68Gbd+6cs1m0NnpCDdsvsQgh904+jrUMFlQ9XS3UY45Vbsw+isNh7n5Gi69L
|
||||
1KHfJmp90itO4fY+v++BYzaHSVnbhZ2LB32oQVROv00bKPRAjk/8mTO4fv+bkanU
|
||||
BdRjJ/UTWsq0BczV/uObZQrJcJHi6+sAMYw4b/kxzTALd+UuvmOP7Z/NoWW6x8Mm
|
||||
ahHgqaMwA0O1f4DsdKYnSUVMF9DNGsxKCUYSYR6RH93Bq/Eo0q1U2egmLIMcTVW9
|
||||
7QSWsJoZuXlzkq7Hb7mxGdppa6kSzA/VM26qPNE9Cjg4tCMu1RJSfgkcnv27Y8vZ
|
||||
fZSq3zkCgYEA68VjIqG6sj43SZSvD+Z+Dfuzc+lO4YBSI0Yru8B4ZZq0vfTVQdM/
|
||||
uf0Bpk/nMbqec/kfcPMHP8zznLe8rcmfZXNQFIaajOb6rzWhCRSgbX98MeGnUe/y
|
||||
9sG+zFSRrAPDaVRJZwSYILs6o6Hz4o6DBCvr8iKFfm26SLB7hIjwx8UCgYEAy7EL
|
||||
dIMdsGDzfmxAYqad3oy/N1KVp96zfdnHEiIC0oiXz3YfI7YLFj54yXxx5rHR2/AK
|
||||
wOo7b90Rc8R0PgtKedKrz5p/E0Bz723ToTxHjsqgVRZqYaEKUOp8wR2t2DJOF9b9
|
||||
0C/qp6iUy0IOTBYyu3BCMV0aB5kRW62jXJIsQbsCgYB6uO7mOurUFsBug38wNpjM
|
||||
rIR3RCz0Afg/NipTe1bwBDwqWEOdFNmp9QEj0ZmU7//EfBsajtXqJsNzgswqZbWb
|
||||
eA9p77qItz4rby3YbS0oceByknOmmdCNEsI+15JPyFGyBNaEUgbhmrNmM0mgVu/p
|
||||
fvc8vS1hZro9VeelUCaMxQKBgFDgnXHH1fQAqu4ZwX7qNWj2bb5jtjSPgqmH3Tlf
|
||||
88rwnYasmjStxb0xVPh7xyYYmQFBUKPE3ZDPMGzNJnK0PQAeHEY0TByyzNXWv98X
|
||||
djpGTl86pUbakKQMVzi+thZP8x4YKXOOcxfbIimKsu6XKdGvAzlihEFcD75dNa4+
|
||||
BACdAoGBAJevnrC7M/KyDDGW3ci4sFcn7MxRGqLBulwGoCuM+zecbG7NBvDynoaH
|
||||
NRGpASiboRJyCEoIQivvkZf+K7L/oB4bL/ThF2ZpJUe471tq0444xnXdHRDLG0Dw
|
||||
OnBl27e3iAiUctqR51ufXKOUaNEf4gcsS9duELMPBxM70GE2Q/2r
|
||||
-----END RSA PRIVATE KEY-----
|
||||
|
24
integration/fixtures/server-ipv6.crt
Normal file
24
integration/fixtures/server-ipv6.crt
Normal file
@@ -0,0 +1,24 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIEEzCCAvugAwIBAgIUYTkp3oUkde9wFRkJA1LlvwFrZ3MwDQYJKoZIhvcNAQEL
|
||||
BQAwbzEMMAoGA1UEBhMDVVNBMRMwEQYDVQQIEwpDYWxpZm9ybmlhMRYwFAYDVQQH
|
||||
Ew1TYW4gRnJhbmNpc2NvMQ0wCwYDVQQKEwRldGNkMRYwFAYDVQQLEw1ldGNkIFNl
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTEwMDgyMTE5MDBaFw0yOTEwMDUyMTE5
|
||||
MDBaMHgxDDAKBgNVBAYTA1VTQTETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UE
|
||||
BxMNU2FuIEZyYW5jaXNjbzENMAsGA1UEChMEZXRjZDEWMBQGA1UECxMNZXRjZCBT
|
||||
ZWN1cml0eTEUMBIGA1UEAxMLZXhhbXBsZS5jb20wggEiMA0GCSqGSIb3DQEBAQUA
|
||||
A4IBDwAwggEKAoIBAQDcRWZxskwCNXhprj8XCtkxj9GP4z9hVgUxgquSBync1hic
|
||||
or6qNgrUztv6nlALdQdf+TbPKyGEwCgAlKU/hnJK6lAG3+riyShnyM74/ulV1wYS
|
||||
F3Rkeh0nNCo95TPNq4GLB+sMfzwoSsT0srPX7KzCqpGy+G7sB0JBNwkTZLkCuMZf
|
||||
dkkmcZJ3zqIiOzJPlcQa4iBa0L1nV3Uuv49kLZLMCLMslg//IZxC09fnmjn+XLcV
|
||||
4+RpOKIn7AMN1kqPqmaB6gk2aCbYTZZ8aS9+cOJmTERbynyX4y4sRV18ED3dRNvs
|
||||
HCedgPOp53nqDneSOqOhhg+Mb95tnMQq1on0+TRDAgMBAAGjgZ0wgZowDgYDVR0P
|
||||
AQH/BAQDAgWgMB0GA1UdJQQWMBQGCCsGAQUFBwMBBggrBgEFBQcDAjAMBgNVHRMB
|
||||
Af8EAjAAMB0GA1UdDgQWBBTFoXLQVq+Yg2AlRIirXj5ho0PMrjAfBgNVHSMEGDAW
|
||||
gBRlB76vjaZyFLrEUGm6DQfyjmN6PjAbBgNVHREEFDAShxAAAAAAAAAAAAAAAAAA
|
||||
AAABMA0GCSqGSIb3DQEBCwUAA4IBAQB4bl4f8TI7k+nlHe4MhJuHP1BKHB5O5SeG
|
||||
wrgI2+qV38UrKvTag2Z3OVKw12ANGN1vcOUrDS7cCtIZ8Aar7JpBgWrYvVlhAtc5
|
||||
3syj74Iapg1Prc0PFRmMQTZ4mahRHEqUTm3rdzkwMjNDekBs9yyBsKa08Qrm9+Cz
|
||||
Z84k/cQTBc3Bg6Xw3vUiL4EmeRQudBQAvh/vdxj6X+fwKmvLbPpgogXuQS/lHhFQ
|
||||
/rZ+s22RHLlqzAMuordjxS4Nw91dqYFwdYVvEmsK89ZnSWqwLvFCJ4uNnAe8siS7
|
||||
53YTpGbpLdNkQKAQJdMQSyvcDbQoQ7FI19a1EtSwpg5qSMOTpQ/C
|
||||
-----END CERTIFICATE-----
|
27
integration/fixtures/server-ipv6.key.insecure
Normal file
27
integration/fixtures/server-ipv6.key.insecure
Normal file
@@ -0,0 +1,27 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIEpAIBAAKCAQEA3EVmcbJMAjV4aa4/FwrZMY/Rj+M/YVYFMYKrkgcp3NYYnKK+
|
||||
qjYK1M7b+p5QC3UHX/k2zyshhMAoAJSlP4ZySupQBt/q4skoZ8jO+P7pVdcGEhd0
|
||||
ZHodJzQqPeUzzauBiwfrDH88KErE9LKz1+yswqqRsvhu7AdCQTcJE2S5ArjGX3ZJ
|
||||
JnGSd86iIjsyT5XEGuIgWtC9Z1d1Lr+PZC2SzAizLJYP/yGcQtPX55o5/ly3FePk
|
||||
aTiiJ+wDDdZKj6pmgeoJNmgm2E2WfGkvfnDiZkxEW8p8l+MuLEVdfBA93UTb7Bwn
|
||||
nYDzqed56g53kjqjoYYPjG/ebZzEKtaJ9Pk0QwIDAQABAoIBABBdY5gM3BLJ8DFB
|
||||
zdQjbTF+ct5SztGnd2lPQPnvaE/M5DU27h1tOG7JE5TSEDZZsnuR412O4cWgFRi9
|
||||
8mz+yxz/vYRVPHku4r6bL61WGvXSrNPJRE92txXDjWPd1HRySoSOyQq7pTeFHo7j
|
||||
e/MN1WP9EigOxwboHycDNLxpHkmyV1DIlAgNkCZV56//liU/b+4vAVIJrgWfwfGH
|
||||
NkFd9nkm93oCFOroJ2f30E1wLPlC+ZhIn4ysau+zlWDLYeils0xHwS2GD7gjp/if
|
||||
i/ibVPgMVW/WPb67olm3nMUsan6CLmKWTiG+yklJT2djoam/iCZWE8/SAZj3qsxy
|
||||
6W9rafkCgYEA+D8tPM8h0oHlKriFDQZx37EH1dfGJRqxr+SgQiJ03d9pGYEsT+jC
|
||||
yr/l5ntzTwEEJjp/biIRwCwSWPYQtN4dNqn+11ICQzjhQbfWTfeT6vhSoBNxkeTT
|
||||
R8tUM0fmoUNrXhPbGZ9XdIxDFgD1pJL96KtyaQGjIRAhyG+khIT7oIUCgYEA4yaM
|
||||
Mw65KDonnKSVfMiOuG0QNYf70UcIiLSH8USnhbQhzT/c2LG7tNmru9UtQhZtmrpc
|
||||
vezuOYTkfcAIUjwqm12Ra8Px8WMzwHwKx3C2SrFCLFgjNFyoQ+VIGjtAL1lNKvEx
|
||||
MObSX7kVIf5+gaO9+KRBEdu55R16yQpW/UVAwCcCgYEA8XdqRkLoED2/Ln3LFW9W
|
||||
ZpJpH61BlCfR/FhzNcEUUhiUv3UxKA0tJE/ijP05nPhNE+5Es1i6UWXM9vFqMLP4
|
||||
UIqsUr73anGyUd1CvBX8sEqY/BHNn26nwKbboQHoKKZOknTX4qVmSPyB6K5IQaul
|
||||
BKN3pwIrreZmJfPKYAiGRY0CgYAYgEbtFvB321X8enA5ZnSmhfUSoRlTaIMOI9Lp
|
||||
/krHjDd9KR9MLFef2T7B4uufzkWCRAnO3qiPgbsXqUf8fsrluUD/S8JkFBw37elH
|
||||
u+udwOLvX45kjn4D3M5bLfrtYIeHUz7IFI2qj48s/INuvle2Yxk1sOqrQPPGjZv2
|
||||
c6rZTwKBgQCHSa+ToxicPJBZ5E7ezgue0LyRGWIMsr2OR16PBL2lPPiCWPH8Ez+l
|
||||
mTClHll4KVZyqc0VOZDbjMjZBnTiAq/1lb8ZvwsXLi0ue1obkkEYfXLWcxYD3Yne
|
||||
iBCGhjkqaUA4rESb22j7yqB8WGT83qV0kB9JwElzE9SxnyR9iw2FmA==
|
||||
-----END RSA PRIVATE KEY-----
|
@@ -1,24 +1,24 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIEEjCCAvqgAwIBAgIUKhJ1faf58PSD3a0vfyrIDojJ/kswDQYJKoZIhvcNAQEL
|
||||
MIIEEjCCAvqgAwIBAgIUBwoN2+J27JtT6IaqV9sWhsHii2IwDQYJKoZIhvcNAQEL
|
||||
BQAwbzEMMAoGA1UEBhMDVVNBMRMwEQYDVQQIEwpDYWxpZm9ybmlhMRYwFAYDVQQH
|
||||
Ew1TYW4gRnJhbmNpc2NvMQ0wCwYDVQQKEwRldGNkMRYwFAYDVQQLEw1ldGNkIFNl
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTAxMjExNDQwMDBaFw0yOTAxMTgxNDQw
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTEwMDgyMTE5MDBaFw0yOTEwMDUyMTE5
|
||||
MDBaMHgxDDAKBgNVBAYTA1VTQTETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UE
|
||||
BxMNU2FuIEZyYW5jaXNjbzENMAsGA1UEChMEZXRjZDEWMBQGA1UECxMNZXRjZCBT
|
||||
ZWN1cml0eTEUMBIGA1UEAxMLZXhhbXBsZS5jb20wggEiMA0GCSqGSIb3DQEBAQUA
|
||||
A4IBDwAwggEKAoIBAQCczqeCNBDCkgqgJuhfIKlo1uw6NAvAN8tBSoxmYH6t51bM
|
||||
+XDJzuw8iQm8UuarHI68O2qjUSiVNvEoilYi/xSu78Yj5vRsNc6i4+FBs09WdSH5
|
||||
fieQ8kGVKRJwBDDNiMUQnG3DrEXH/cd7NLegyFdnO17yjTeAt8Fl482QKIFDuInT
|
||||
tGLBR+NKYcSM347Vz0tmJIuD+1Ri1DPhOrnrS0KaLKVvaDYpFVdNy0xS2zX+p4Ti
|
||||
QyznXjqXOXiFIDOU06huNerWJEzr8Z/ylRPz3pA0bGqcJpIXxzJcGJWN6Jfh6izf
|
||||
ONszTSgchoN9sn7iah8cHitEdYQs4LTKjyqu+9QXAgMBAAGjgZwwgZkwDgYDVR0P
|
||||
A4IBDwAwggEKAoIBAQDHS/zscOjq013InbTlwsBVwasv8e5+ukZNGDQx5RNaXYxI
|
||||
NVUM/5Bai4R3CS+DTbr+jBDylKi55gPQ/UIDKlU/NQH+x6UJB050G+aLDWAuRmxi
|
||||
w8dq7kRw2QJvuMxI+quiZhWk2HYjtvZRZLCUGl//QTL/VCT1smXwXRBU19S2uOfy
|
||||
g9KgZL/DCkJ9VBUh3+bFVKXBDnIphY4N/0+B/sW71cvRj8zvW3iD0R5T1J+QVEFz
|
||||
sFRT99/OhV2kUEwMaAYOFv/mMIEO6qc7vf6pB91qdUfEP8AbsOlmiSuOOLuR6X/2
|
||||
FHUjc8JrFfMuOVHnedRR5quxXbP8o83ilat0tXeVAgMBAAGjgZwwgZkwDgYDVR0P
|
||||
AQH/BAQDAgWgMB0GA1UdJQQWMBQGCCsGAQUFBwMBBggrBgEFBQcDAjAMBgNVHRMB
|
||||
Af8EAjAAMB0GA1UdDgQWBBTdwPxrHxU94wYrwQBOjp6ma6EAajAfBgNVHSMEGDAW
|
||||
gBTdZTAnocY85lHhyR/A7lJkp3t2mjAaBgNVHREEEzARgglsb2NhbGhvc3SHBH8A
|
||||
AAEwDQYJKoZIhvcNAQELBQADggEBAJB6rG/9kScmQi1TQMFJa0ZAsG+/9m7Rczye
|
||||
RApBF6pG5nf8FJiCt7sNYT8r8i+kby2H0CLII16dXSZxPG3giRN4TviM++/YXW/j
|
||||
rW1SueyhS+bxajOQfRVLxTnBk7TVDvacwJdFy/VI28i6hoV8E12g9jslAMiWREWd
|
||||
nhgk3zIyXFlVuiHIRYqKFWeo75/cEyTZ5XWs06r5Odawzo2L094CT4uxgu8mRCwN
|
||||
sJKa408ev6CUEW7YXZVtJ8IwtFfJCWAbe5Tsq/9K/m1puHLOiVRwYBl6rCUTLjGO
|
||||
+iKZYsV3wVf75iENevyv7rQ9OkomJokdWxhi5e+VxC4x+zwbvKU=
|
||||
Af8EAjAAMB0GA1UdDgQWBBS7gBJSFrjAHryiQpe38OMTzCKH1TAfBgNVHSMEGDAW
|
||||
gBRlB76vjaZyFLrEUGm6DQfyjmN6PjAaBgNVHREEEzARgglsb2NhbGhvc3SHBH8A
|
||||
AAEwDQYJKoZIhvcNAQELBQADggEBAE2tsLHtgF1T3d/anKf543q9uh61tr46HHf0
|
||||
RrGZF+RJuJY5XIAiCN514Z/I7i2wG/x1TDTKwZUebajbk4GvaI4nEnCXs05jwm/n
|
||||
wpdyRE1EUy5PkVFfXKCNQd096mpZu6EYXBGnQ2fQjg5zFvZSDnYaIf0vBF1WxE4W
|
||||
0a4a9na3N77OSamPEljM1RJ1Sk+Zg5yI+nwyKcWWk3OlD0j668Vp6/m5VZKyQEkx
|
||||
crfSj7kgRJWZRhMeh6li3xa9vDmzdF6ojGkgRN3Qljrs36JnmsTono2ETF8GIc+g
|
||||
eNByAQNppLJjMn+zsaG9J5pr0gDLubFA7oa8aAJgYgJMM/GecAg=
|
||||
-----END CERTIFICATE-----
|
||||
|
@@ -1,27 +1,27 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIEowIBAAKCAQEAnM6ngjQQwpIKoCboXyCpaNbsOjQLwDfLQUqMZmB+redWzPlw
|
||||
yc7sPIkJvFLmqxyOvDtqo1EolTbxKIpWIv8Uru/GI+b0bDXOouPhQbNPVnUh+X4n
|
||||
kPJBlSkScAQwzYjFEJxtw6xFx/3HezS3oMhXZzte8o03gLfBZePNkCiBQ7iJ07Ri
|
||||
wUfjSmHEjN+O1c9LZiSLg/tUYtQz4Tq560tCmiylb2g2KRVXTctMUts1/qeE4kMs
|
||||
5146lzl4hSAzlNOobjXq1iRM6/Gf8pUT896QNGxqnCaSF8cyXBiVjeiX4eos3zjb
|
||||
M00oHIaDfbJ+4mofHB4rRHWELOC0yo8qrvvUFwIDAQABAoIBAE2/MNKf2gd+vYH2
|
||||
iJRR720p0upwz7q4bzCqM7627VhwMVtj+gY1cG/7SjZzrCb++85cb10WalbBbQS6
|
||||
wkaLzseF3uoCIwJcE140OoWG0Dl/zh6T7C4zz0yTlq6vhTsDKyI3TT0Nd3PXYka2
|
||||
Nq2jZ6Kpj0QXvIM3mM5aCpwFWUInwGTAPG2tskhPiIcbnJnajiGdupia4Gsl1DWY
|
||||
H+L7FGxI1FHtOczb57Z58BQoG+vXFbrnuRUBWCR3TF0bGG+6ywUFPnTqE9TfVrbf
|
||||
1kORUZzIGbZBOYl0n/ylq5nc/DUXThKY1G6C31kTvZu6hTsgx/A5MeKOxu6eojZ9
|
||||
3DierQECgYEAwD1EwiKmaxOX9XksmVUa8IvbI/WKhXy1h9cbfJFpjPKUXBzgWAaQ
|
||||
Si00PzrzKFkItGsuoI8DoglZMWEWswv+ipN+Q8DqT6+RposqpIbO/Kvr6ynMjkNI
|
||||
+SW5Opfo1ih2YaTTSsKVYOwxI0HyIMnIbqUo/MDsrx0xM5x6ZcWN+lcCgYEA0NDn
|
||||
huvKY+NDYaQn6G3JjD+uvAqfsTaGSyPZpWpvNb8QBGmB9gmEHfvpO+fs+ANpk1xM
|
||||
T70piT15SQM8XlX5yt1w9MtCliXNUEj4KIZ60r+fD+LU61a0os3cUfeuLvck090T
|
||||
/8hg0VIfh5kT4vW+/Qzj3ToGJlbgxPjy4vTIXEECgYBbHS73rKCf45uEdzhSPGoW
|
||||
VNM6yegruJ35klSux+T1LwxuVCloz0OYRLCcv338Q6bCjIJ/Cwm3xSKHKvMtTaQJ
|
||||
XjrcNhVYS2iHPTKq5FI6o+mffiI701xE6kJOqS1sQ6VIRmz25B8U3x2zDlCKhdj/
|
||||
yDwTAAWuQfHN+n5wuh++5QKBgQCij79u3el10dsHDJC09aNON9WENpETmOr7XDF4
|
||||
AWAKa2/Av5KZQ9dCsHZmWKu2eJGcUuJlwVGwsdbV7Fr25d/M+o+RQxvXYB5yPhxT
|
||||
ED5WCy99Yo13mMyPYcv/U96aWXSmKxoCqrAUxOYe3iEJM184COzrsNxQxJm/Pj94
|
||||
SDAjAQKBgHCW9ndreIIEHm7pugis6/WaoZTkdU00nQw+pYourCZ4/zaI9QiUnFwH
|
||||
HPLkUdpt2fJKKu9h5DMGl1hzYHF4QRuPWJK8xGTP7YTJmZAzLtadiAhRaAq8/nWv
|
||||
fZJPEXDteRoceSo/Odujl3mDMhRTiGi5QKfdGV6WZPicLbvbRkmk
|
||||
MIIEpAIBAAKCAQEAx0v87HDo6tNdyJ205cLAVcGrL/HufrpGTRg0MeUTWl2MSDVV
|
||||
DP+QWouEdwkvg026/owQ8pSoueYD0P1CAypVPzUB/selCQdOdBvmiw1gLkZsYsPH
|
||||
au5EcNkCb7jMSPqromYVpNh2I7b2UWSwlBpf/0Ey/1Qk9bJl8F0QVNfUtrjn8oPS
|
||||
oGS/wwpCfVQVId/mxVSlwQ5yKYWODf9Pgf7Fu9XL0Y/M71t4g9EeU9SfkFRBc7BU
|
||||
U/ffzoVdpFBMDGgGDhb/5jCBDuqnO73+qQfdanVHxD/AG7DpZokrjji7kel/9hR1
|
||||
I3PCaxXzLjlR53nUUearsV2z/KPN4pWrdLV3lQIDAQABAoIBAQC2y+TVvY51bJ81
|
||||
lilJIIMnZTauCDqXdCVtKwkcxp8koG89/+Tdwj7WPeenAv7YcWBVf4U/6siDkgzo
|
||||
EJMOsjJ0ghstZFLkYBY+eyTPX9pbN27MfAQZ+Sc/VlxcuuRs/7aTgwzRIVXi1jtB
|
||||
Vph7j2GDj3rGJJit3w6PE90Z5MkPOhXwbPD+T2OCIhO0OQCv9YNrdHmQzFZJ8vn/
|
||||
FuKUjZuoKKnwgXvBVBKsUPvvSdPTWpavNYdA7WQtjpVYVjVHgEHZWtxUwQ43JHzb
|
||||
pABWqYp/XJNiGhZ+cEXsw5dBBWp/BPxbu1P2iagZTmNr/8EfGCq04fEkKhv22x0y
|
||||
FbQa+2e1AoGBAOCztIuf1Magca9mFD+3YZHgBv2TA2XSujwYBr/664dLjL/9NQIK
|
||||
00IBykiNykiWZ0ixcaJI1j+af7fWr5OuSzBVwdXMUZraKUEwrKI3hh764yX8aUYt
|
||||
JsqpAFhyro7smp3LaUyMCW2ZFVxayp60h8fQXcNepFwmK5o5BnsTsFHHAoGBAOMO
|
||||
ZooI0Yz/fzBKOEMM1Vdf3PpUqYnjCyJSXag8OeZn/OPgiYkwXWL0idfC49B0ArVZ
|
||||
/j2zMXJduIrwa3UIfd6tjPf8O24YOiO2SenkVkcsUwJgsB1nM1QlOamGw8BB+nbT
|
||||
O5V44r7vy3HldHHQgbPvjs0z5de3b3eBBTZC/2vDAoGAVNroSnYAV0YNyIwHB4zL
|
||||
9tegLDBRbylmFP2JxwQN39ji/Tm0w+Gsp9efOUj6Y/EQbf48iGlzJy/EHXugcGe4
|
||||
kzc/bOqswoqyW6DzAItxRc++6gBpDQxOAuhRbhVY4DZvqTlAuZyEjvPpgifzLn3E
|
||||
bOu+DOJ3tSjg/Guei+oCgs8CgYAUfwxKkZk4/SdiGJETnGj1xjWQc2wKgnBS3NSP
|
||||
h0BCyEhP2ckQlUkY0bJPw8wE2TQVYtZMg4yHImayRBmvKuER5ODA0ggbXByDdMUf
|
||||
U/ll215y7H95aAN+KQ4Xe47YIByX9WF/kLYHPmZDFc95JrVOpOVjKLgqzOhHBWKP
|
||||
D2U3OQKBgQDbmwsNr0mopOYiAp60KsKJmICUQO27RyL87UfdFysDrTZ+K1Pc6X1e
|
||||
HOFtma4zNftDym9Xjzz2eOXT6flHeJNu1qZwvurNV1g0JZdXnY83q4C130bAFJLt
|
||||
I0+I3vDpJt9wznYnC3jDI24gCbEJ2D//8dpeDNUPKk94rjsEjXxDFg==
|
||||
-----END RSA PRIVATE KEY-----
|
||||
|
@@ -1,25 +1,25 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIELDCCAxSgAwIBAgIUHqSMAwCTIo4rAMwp1Dp8tBqn3eMwDQYJKoZIhvcNAQEL
|
||||
MIIELDCCAxSgAwIBAgIUEQuXXKtjueOgUpZjzr1ORDG/zHwwDQYJKoZIhvcNAQEL
|
||||
BQAwbzEMMAoGA1UEBhMDVVNBMRMwEQYDVQQIEwpDYWxpZm9ybmlhMRYwFAYDVQQH
|
||||
Ew1TYW4gRnJhbmNpc2NvMQ0wCwYDVQQKEwRldGNkMRYwFAYDVQQLEw1ldGNkIFNl
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTAxMjExNDQwMDBaFw0yOTAxMTgxNDQw
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTEwMDgyMTE5MDBaFw0yOTEwMDUyMTE5
|
||||
MDBaMHgxDDAKBgNVBAYTA1VTQTETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UE
|
||||
BxMNU2FuIEZyYW5jaXNjbzENMAsGA1UEChMEZXRjZDEWMBQGA1UECxMNZXRjZCBT
|
||||
ZWN1cml0eTEUMBIGA1UEAxMLZXhhbXBsZS5jb20wggEiMA0GCSqGSIb3DQEBAQUA
|
||||
A4IBDwAwggEKAoIBAQC8QxRZGCUocS7zTrqiOCIuM0HYoojgOZTK91GX8EooGHMs
|
||||
76bZrzTUncEvvdXxOOOAkSWzOz69937sP0G0QhPBw95UqZ4+vAcL+ByJ3PG7cKQ5
|
||||
HYEo5FyETLihD3mLXydXsSudv9sCGp8aexJ2Y6KOzFzY3v7oTc5evPeYKL7qnMri
|
||||
SJUUMr3W8cK1jpU/C+MddzYvC9mNe9QQ3vvRoNYLe6oKyHfV0XhR3pl8katZcf/8
|
||||
091KBzx35ksHiGJ2zmaR2IDs+HN+KMzKJiizPQC3VYBvqiSHJ/2ANDKk7pRwCvdc
|
||||
zyIafk679p2aInXEReOvfFbMqNHfpDAMIqG0As9TAgMBAAGjgbYwgbMwDgYDVR0P
|
||||
A4IBDwAwggEKAoIBAQDPAGzCcM5JSlitl/iHLYJ6eGO8MJ3S6R1qzAvmdB9+KsGD
|
||||
F99gWVTrJRzz/hJyo9Lt8GvNj9Ll3iT2QnXyyaSOX1+uT4cxBM2MFBBfERDh0WUY
|
||||
43uLQKY45H4zrS4tJTOuSGKM/LlK2ZMj++pQBqHsONrNG+nOhqe3qLqPDV3yBfmD
|
||||
PXfjASNvHINgxb9AwQWJydgjfGDiAwWHnKbnVScYBFgWfMG0Gm1wa8EfRfWD0NPd
|
||||
L61XwQwgb5VsYAs7XH7bxVbPm6E+/oQTOJXQHMzQYve9DFPy62KFSIkfvNwVRctL
|
||||
NE+k3HnyviDzbs387ys56ZjPG1/XpbFmeQuDRndJAgMBAAGjgbYwgbMwDgYDVR0P
|
||||
AQH/BAQDAgWgMB0GA1UdJQQWMBQGCCsGAQUFBwMBBggrBgEFBQcDAjAMBgNVHRMB
|
||||
Af8EAjAAMB0GA1UdDgQWBBSJjfpommIxJQ3q67KG/4NQe0g2yjAfBgNVHSMEGDAW
|
||||
gBTdZTAnocY85lHhyR/A7lJkp3t2mjA0BgNVHREELTArggwqLmV0Y2QubG9jYWyC
|
||||
Af8EAjAAMB0GA1UdDgQWBBRyAdJmiDFhC3CYXPLW8kufz4zp6DAfBgNVHSMEGDAW
|
||||
gBRlB76vjaZyFLrEUGm6DQfyjmN6PjA0BgNVHREELTArggwqLmV0Y2QubG9jYWyC
|
||||
CmV0Y2QubG9jYWyCCWxvY2FsaG9zdIcEfwAAATANBgkqhkiG9w0BAQsFAAOCAQEA
|
||||
d0BLO+3K4SfbHfn+rpNH2xCcXVyLZx2p/HhXo8ivzAQ/t+pkWBgxIkJ/qxHfQjZO
|
||||
O0/b5DC+No6UhKNNyunHxv/gwsmzz4rzPmCA0OD6om/yZoXtz3kf0ktmbKbO8RWc
|
||||
nQPZRC26D+cshebDAAm9XoCGErsdI1Xekx5Y66c1t1tUVZaijDB7FvnaRC5Rbzn3
|
||||
5WKRHkF+ahJDqkpTWUsFDuyRWF771riiA0kD5wwmaw5W5/knr1BiOePCUFaTZPXx
|
||||
r9LgDqIG7w1A1kbi5zLm43pN/S5qpif3DcSYYaCjQZC7yTVaRb1JBvcTY/g5whJx
|
||||
nyHrKafioCOq8CwHpbvZlg==
|
||||
sc0JnS1udx4HFStcPZXY+kyVwYPuRDv3GisO+TTrxzdupQQl4gPmuGa/cik4tKxQ
|
||||
o//XFgCBQCapO2cY+JerjMSOLHtt4YmdyYDSXeMjsRG0sP36njH3nHIYsFAoEyvg
|
||||
nJQX7iDWj/9bzMT4dU2ac3t9RgCtyABRoT4G/MNhWMlJt9XkwVTN2Pqf4TMV0GlV
|
||||
54GOScsWAIwoIDPOCoO8Q40jtFSSnehrlrW7x6B37gY/EbbYpZNrIDNckNfLJMvl
|
||||
dak9P/ovtjLk8GM11gE4s2ANWA3o5bIm17b1x7Fw122sB4Rxptbc/BDpv2GIY5Zm
|
||||
tqDculro2C7Ib5GyEBEl3Q==
|
||||
-----END CERTIFICATE-----
|
||||
|
@@ -1,27 +1,27 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIEpQIBAAKCAQEAvEMUWRglKHEu8066ojgiLjNB2KKI4DmUyvdRl/BKKBhzLO+m
|
||||
2a801J3BL73V8TjjgJElszs+vfd+7D9BtEITwcPeVKmePrwHC/gcidzxu3CkOR2B
|
||||
KORchEy4oQ95i18nV7Ernb/bAhqfGnsSdmOijsxc2N7+6E3OXrz3mCi+6pzK4kiV
|
||||
FDK91vHCtY6VPwvjHXc2LwvZjXvUEN770aDWC3uqCsh31dF4Ud6ZfJGrWXH//NPd
|
||||
Sgc8d+ZLB4hids5mkdiA7PhzfijMyiYosz0At1WAb6okhyf9gDQypO6UcAr3XM8i
|
||||
Gn5Ou/admiJ1xEXjr3xWzKjR36QwDCKhtALPUwIDAQABAoIBACE9KrOMSss4KJT2
|
||||
uob3qDlF/YM1Nzt3jyjCv9o2p7Eql/NAINJgV9CORdSbDNH5PFmvxts9Q5egwf0O
|
||||
MhiUbg6Q+YkzJWhoJEpdtRQ2OUvh2GzpI6vHnfoR3as6IziTRUv7UPsaK53Ue3De
|
||||
d+UFNGdH98CmppT3X89kR8bKsuKTHLd0bL3gPt1kDqyMGEEZ3XUB0l4PrakRv466
|
||||
WfYuT9suHjhdsNQn0bHzr93OyWT0KFffrwA6l4cKsGgf3eW6CKZCcqzTRHfe4Iip
|
||||
Oa2l0O4uM1Qp3Eo4KPBsxk2i6dlLrAEhDUUlnvg5PCIResWV9MbBERKA/+BEi3CB
|
||||
azUiyEECgYEA0Wk3AWWzPZnOVdGMltiArllkSODuwekuZN5VNQKYs5Qfi3fHcm9n
|
||||
rXN6/wSOai9Mt6oL8wGHODyfChmllgtU67jV7RKQUy0WqRN81KbHrQynqmG8DAgS
|
||||
aR/np8pIYoyeL1sELH9CvRSmLfGWxa4j6J2wuUzlCPFeYhmniSecxjMCgYEA5iVY
|
||||
5rUXY8uILYkVBme8YEPbiqfTmgaXcsVhB9H331o5BX8pFXc2n0+igbkQ1ZomDN/u
|
||||
5qJ3ZLk/ya77BixVV7U1+UPJM3knBBBbzB6zLGsXSpMuLf/cGOM8nnJ2EFfl0QnB
|
||||
nTcqwH59iigUHBLXnO/nijdZtnN5P/l/U/ZDcmECgYEAojt01axIVCzX4TxaQnKw
|
||||
HmI6gwtfbPKNcq+cK1k7m8PhPFcrOMh99a5FV1PcUP8b3B3s6/H/I94zB8wesENP
|
||||
It8rPGLpVMbVi18Bkm0yvCnVqvXUjS2jtbV17lOUCGUQF3fXn/FnbryUkXtcZwEl
|
||||
6Ixh4Oxlc6wqhq2BUYxStGkCgYEA0cKwaHYdP2O+VmJeu1vJIaQ3cQTNo9DmMEEd
|
||||
0tbYqMW+uvfPJjVln4Yhg0J9yXGZxJpFUAg78Z22Ocg2GsZFco9DBlF2DGgb22Rd
|
||||
holknNNugxXqPRq6LCTQl0mTugmi+Qd/ZB9n49Jl8Ynd6khyJCO0URFpvxU4Kcro
|
||||
9km554ECgYEAyqC8orPs7VkJJ4CCaJXGNQycFxlklv9vodV2b5dhOsmX3gzs2Ssn
|
||||
xhMphgGR35YEsMWxpKofiMbqpI/VZEpjH8Zcb8IklVCGKi61SQvBgJo7/545Gr4+
|
||||
ZdJ9TBLTq21LhleqLMFsWMGR3v+5Yuo1NoGf6jxJ29J1PjK2gooqIw0=
|
||||
MIIEpAIBAAKCAQEAzwBswnDOSUpYrZf4hy2CenhjvDCd0ukdaswL5nQffirBgxff
|
||||
YFlU6yUc8/4ScqPS7fBrzY/S5d4k9kJ18smkjl9frk+HMQTNjBQQXxEQ4dFlGON7
|
||||
i0CmOOR+M60uLSUzrkhijPy5StmTI/vqUAah7DjazRvpzoant6i6jw1d8gX5gz13
|
||||
4wEjbxyDYMW/QMEFicnYI3xg4gMFh5ym51UnGARYFnzBtBptcGvBH0X1g9DT3S+t
|
||||
V8EMIG+VbGALO1x+28VWz5uhPv6EEziV0BzM0GL3vQxT8utihUiJH7zcFUXLSzRP
|
||||
pNx58r4g827N/O8rOemYzxtf16WxZnkLg0Z3SQIDAQABAoIBAQCd5u0PxY0WSygq
|
||||
A2sJcqW9Vmh9/XfmkvxloxDQ0nPTgjnrDiLPFFW6qazUUlMwL9eOuX8CZ1uxDSuU
|
||||
zk26ziZAlHAgP3oY4lkJKaTzX8lI+Lntqllrd/1UGLhMIya+OUqa/4xtj7qoZh/f
|
||||
qyKpuOV7lEMTgt9vMzhs2MC2rrOjEZxcpuwpnZLKvpuwBMcxD1ccRdCA1zHvKdQ9
|
||||
ukPTRVjz9WUEOgANRkndHTZKWMz2p4QC9Id35HlksZi0/M6oboz2Eg1mtZEpbgUX
|
||||
loMv1CPtWP1uj9PFWiOmnBC2/v/2MVGg2fJ1Lf4c72ZVFEIU4l3YNiV4IqFb38F+
|
||||
GJVcmiGhAoGBAOvrTjYYl5vodK57gRRT0UsaU6x64/IK2i0vbBGTATywuijJ313X
|
||||
vwZBU9I2rLZqr7FZ27g5ANorw8dUKn92otr/TVS/c/VZOSw/+gTM9Rl4ZGji+qKt
|
||||
4zY/dA38jlDJNWmFwK/9KNOfXNS+WLsA2QJlONgUfkFPb3yXJUGLsU8nAoGBAOCf
|
||||
AUcyDHjGwtYsLc/4aiKtQUIdeX0v6jCWtl9EI9cZ/o414iapE05sOGb724itSFN/
|
||||
EI4biQGw8CaMcaqMaRJ8+xVQQJ7qkXItzZEFVGqz0PKwiwYAwFp6raXuio9y+cTw
|
||||
savJIM8IDijph9ezRCalef4Qj6I0zFI8H7PmiwwPAoGABWvY1kFmanzDAadw5eiv
|
||||
LIykU5hXWJ6LOPKYBydbpethu8I30c49Y4VoybHb8i0tcGPiOq+Ep37N9uymNVui
|
||||
jmnDeykTHxY3zB6EPkv/beBoXkio/cgFKp/2qMOe+ZhGE/Cw5tpob8R/u5vMKi/w
|
||||
zK9KyRxfclzC8RgAESuGnY0CgYEAlHC/+Xrbvx0rOTps9Blomo4AqF6uIMr/ayjO
|
||||
UNrJDKfDD9wQHhhyB8uA4p3ikMpjF7rLB/6uZg22RuNdYqXz8iHiFE26xsqhX+Fh
|
||||
DkuFZBZ9KUT+OvNYKvMTuqqPqwkCguHFqI78PZVHNkZOXX+8tAV7PylWoo1d0aKm
|
||||
GM9saIUCgYBXO9TtUTiaoxIVTe+r+Abt5iwasAwxai/RvymdykwYFwq2NOEl02oi
|
||||
fU3gbqDV3oirHAsAKJipnrASc70hTn4SM9hUKTQrD3fNIABch811ZDd8vaHEzLZG
|
||||
pp9yKam09sPvQo6O4E7TJPccrddV286jZq+qO9YNIsRlYJiCnjzihw==
|
||||
-----END RSA PRIVATE KEY-----
|
||||
|
@@ -1,24 +1,24 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIEEjCCAvqgAwIBAgIUDJK/Om7oMHhIrvfvj/NMLa2PbkIwDQYJKoZIhvcNAQEL
|
||||
MIIEEjCCAvqgAwIBAgIUdUecO/Un2wfrJXsZRGkX2zFkK7wwDQYJKoZIhvcNAQEL
|
||||
BQAwbzEMMAoGA1UEBhMDVVNBMRMwEQYDVQQIEwpDYWxpZm9ybmlhMRYwFAYDVQQH
|
||||
Ew1TYW4gRnJhbmNpc2NvMQ0wCwYDVQQKEwRldGNkMRYwFAYDVQQLEw1ldGNkIFNl
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTAxMjExNDQwMDBaFw0yOTAxMTgxNDQw
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTEwMDgyMTE5MDBaFw0yOTEwMDUyMTE5
|
||||
MDBaMHgxDDAKBgNVBAYTA1VTQTETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UE
|
||||
BxMNU2FuIEZyYW5jaXNjbzENMAsGA1UEChMEZXRjZDEWMBQGA1UECxMNZXRjZCBT
|
||||
ZWN1cml0eTEUMBIGA1UEAxMLZXhhbXBsZS5jb20wggEiMA0GCSqGSIb3DQEBAQUA
|
||||
A4IBDwAwggEKAoIBAQC20alHsyfXrOesg+du8zV5OyzCkwSfXrtDG+eXcPd/y6g8
|
||||
uw0MQq4A3aoV2mrpLA5BOJbfaXQEoTutM2xwsNubJl5jWUr7FtFDlnvv0uv1jRkU
|
||||
OG/hEwpam2otW6yFGT7YnVtfrCemvzk66siFooofmSWM2aw1r/U5irqtsR3S6dMY
|
||||
76YI0ZqdPYU82l7aRp+mEKPMjt7S4renIuK5Y9GUwJAWoCanq6z7rU3qZAWUkZJ0
|
||||
cHAimHhXG2bBSWbCS3AedmosWmBrowACeVh7TCwvxd9tz9WMFUbMHV9mSgViJIlS
|
||||
p4hdL24+5gX3QQJgswNlSdJVKBeYevHWRxp2pkElAgMBAAGjgZwwgZkwDgYDVR0P
|
||||
A4IBDwAwggEKAoIBAQCgCUX0Mjoo9cckyzK+OdFaUXWyB62pk+tRi74OCPeym56n
|
||||
IF+eQd0llbBJ4AfK6udhc76B+eGFu/tgaroZd3PXJqBCV5n+EcUSYxChWZwbbmVv
|
||||
mv6fr9kdhJNaF0LJ5IjrN7bVWHwIkEQsaxs+50M3yPxsjC+OcSGR+uSFz9gf2e6+
|
||||
bzYvumxkv0HuIHnPJ1fPJn5Dyu6odjPjoqKjY3opVIuMUyrtAlr/XsODXxWLhoQ8
|
||||
KkEGZA+ErDiVp29rNH5AfKytqF+iJPzKcHk5ftINYRx482NgJOWraY2j+KrKkv+T
|
||||
bv9ZVQ3GEEHvdtxljUPXeFwY/qDoxQFDWOvbS/8JAgMBAAGjgZwwgZkwDgYDVR0P
|
||||
AQH/BAQDAgWgMB0GA1UdJQQWMBQGCCsGAQUFBwMBBggrBgEFBQcDAjAMBgNVHRMB
|
||||
Af8EAjAAMB0GA1UdDgQWBBSFhLSU/LEPJNAzk1lir5Gekdm7sDAfBgNVHSMEGDAW
|
||||
gBTdZTAnocY85lHhyR/A7lJkp3t2mjAaBgNVHREEEzARgglsb2NhbGhvc3SHBH8A
|
||||
AAEwDQYJKoZIhvcNAQELBQADggEBACgZtM7ygZSzujOQdPy9lpC/9TJkGqNN1MBg
|
||||
+SJGalygPY5cSaVRS6ppWNnLgvy+mIGTBXfxH+2eVO5frSwzTCB+seXJPnzwxjhO
|
||||
yj89MKtPmy1YQAPlBvi4IOKJJS4cvYKhp3Dj1ca2JDOJ4Zb/FYRn+pjy5L6Ash1Q
|
||||
0pqlt5Xty7EU9EeUgf+2wHtrsCb4O9eR0qPr500ZS/7NA66vfu3vv1vnetIuAhFP
|
||||
zeJzL/9DBh73lwxoYroq/M9AslLLJUhBBRosxJGU8b3e2ylM6RRkAInI/rNYPlI+
|
||||
nRA658kmyYMM0W9hUB6mTYy19z9nZv/KzF+cYm7VyEmiGSHCD9I=
|
||||
Af8EAjAAMB0GA1UdDgQWBBQtNVqbrSLEx7TA7E4KhsH4h8NRhTAfBgNVHSMEGDAW
|
||||
gBRlB76vjaZyFLrEUGm6DQfyjmN6PjAaBgNVHREEEzARgglsb2NhbGhvc3SHBH8A
|
||||
AAEwDQYJKoZIhvcNAQELBQADggEBAFdeFLblwKDjQ/aTUe0XhsjI7IdKc+tQujrV
|
||||
uU15FtRzA977ntJCYhaYyNpTBjOo65C/UOyZK/ZD6vI6vH2ENRvCgZUIInmfKw2X
|
||||
w7TbTHKTDZqbnKVKqORx2+S8NBaE7KSPae+Q0zQbHLyv5btBcRGv8MECGsi+pjP8
|
||||
EiJM4VMQ9obziCtWl0j/BOY+YSgjIAR9huyJEA6RibCoNhjCaSSSzNqIEzpCh3zh
|
||||
6zR0NzQbC3z+B6IGWzvnPBbrc2LH+eqywSlswVy1Thp2btU8b21WcjIqlcuL4Q1u
|
||||
NmHmq8K2zLLvYQ3voLYzxbeMlsuZjCI2/UJSNKOSGlmlNswHy18=
|
||||
-----END CERTIFICATE-----
|
||||
|
@@ -1,27 +1,27 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIEogIBAAKCAQEAttGpR7Mn16znrIPnbvM1eTsswpMEn167Qxvnl3D3f8uoPLsN
|
||||
DEKuAN2qFdpq6SwOQTiW32l0BKE7rTNscLDbmyZeY1lK+xbRQ5Z779Lr9Y0ZFDhv
|
||||
4RMKWptqLVushRk+2J1bX6wnpr85OurIhaKKH5kljNmsNa/1OYq6rbEd0unTGO+m
|
||||
CNGanT2FPNpe2kafphCjzI7e0uK3pyLiuWPRlMCQFqAmp6us+61N6mQFlJGSdHBw
|
||||
Iph4VxtmwUlmwktwHnZqLFpga6MAAnlYe0wsL8Xfbc/VjBVGzB1fZkoFYiSJUqeI
|
||||
XS9uPuYF90ECYLMDZUnSVSgXmHrx1kcadqZBJQIDAQABAoIBAEehUmBDc+LvXj6b
|
||||
1/wC0vrTErCSBgejiF+8+Tq/ClpKI3rYFz6siZzRPSke0TDXECbB02tp2AlQWx97
|
||||
vItS4Fij6eunOteykv+WYK0QyV8Gj8L6ceuQGEh0CGAJQUlNIPCihwCvZUHSPsw0
|
||||
3ahujQvgZ3QOfYjzjnOjRY10jijqxxsOfGJUjgFsD1ltMMLCrHBLnBk/KXgWfEv0
|
||||
pJ79RDhQhZHcB8T5FmAc8cPnXj+uZG8rZJ+VHQq6CM8/clwTb0SBk0gdWuO0h8/L
|
||||
0k4qQzZTLnB3ASS0TMbyLtdnIuCnIVgXYT5cc0THPNI+/2gdklUl5kjUviNG9gXT
|
||||
PFujFcECgYEAz1tFvIs7AFS9Y1usxmzd5rLReninnyIiGORmFMDT5vg4eQW3yzT/
|
||||
4NLVn1L61WKxrckpb4C9J4rDVVrDbV4RkO3CM7Fkeko/bvPwJj5h/aM6pvguMD1v
|
||||
UqMZGr5QcaYakabFt1ELUTURGlkkvg6fsvoac8F96Axeu8QhEIPLZRkCgYEA4bTL
|
||||
ebP5I5rjiq921JWm8N6/qnxT2cnWIXpeXdYWnSvtYfQ9LrbVt17xyJ+niWmZMAD/
|
||||
dmdWbyrTTBGjsAUT+U+uX2sup+3pXSG4nFrRZ09n+vNQZa8ZliyTg4x1fyjJmoXj
|
||||
/yEuFjT5pwghE2DroWy30cIxWB+cG8h6/4gbEe0CgYAX0xreIPk0fogMJHpjihqs
|
||||
6RrcgYRw6lEUnxmDhOxT+20xqpCFjp9fy5mz5qrfXamgmB5kq1wQcQckhXsy074c
|
||||
8tR/cABldKZ5LxoGquBDbj/M83MLI+PokMjPo9JGXJls5saM24j7WaePaXKpS7DZ
|
||||
tZbYegDnapRv6ocKCLqT8QKBgEINMbrFA3T6//n3DXR68ybsWPTge31xxrt3XyRg
|
||||
4a9PSqHu4vTHxtVp8KIArvvUrmLQ9/HDnhTcWIebZea+JbFBM0tzR20Xf8Kkeq0v
|
||||
Grb8Evbqu612R6ueEfFeaogy/IS/CBECucT/7cuG4n8UTwCnm0fEZ8JdRccPrYcY
|
||||
YqulAoGAWyNwkmFOooU48FPFkt7OKcXD/fLzzcXTeW2fcje9+bX3Kmkg9BE2B1Wg
|
||||
MDCMpBOKkUB/COtqS17iC7zdViyDRO0rxybBLFNAVEq23K7/0zVsxuWCEIPnjeE4
|
||||
nbh4EH5L/heT1/HrkiJtXvNd3hpYDXiVD0b06iYBVO19p3uDuO0=
|
||||
MIIEowIBAAKCAQEAoAlF9DI6KPXHJMsyvjnRWlF1sgetqZPrUYu+Dgj3spuepyBf
|
||||
nkHdJZWwSeAHyurnYXO+gfnhhbv7YGq6GXdz1yagQleZ/hHFEmMQoVmcG25lb5r+
|
||||
n6/ZHYSTWhdCyeSI6ze21Vh8CJBELGsbPudDN8j8bIwvjnEhkfrkhc/YH9nuvm82
|
||||
L7psZL9B7iB5zydXzyZ+Q8ruqHYz46Kio2N6KVSLjFMq7QJa/17Dg18Vi4aEPCpB
|
||||
BmQPhKw4ladvazR+QHysrahfoiT8ynB5OX7SDWEcePNjYCTlq2mNo/iqypL/k27/
|
||||
WVUNxhBB73bcZY1D13hcGP6g6MUBQ1jr20v/CQIDAQABAoIBAD5l/zmMj/LCiehF
|
||||
tj5HauJtWpeUuNiizSDZfLwaMQIZ/U0qqT2abrCl4bucN02eM6Nirsgc9xrexc+9
|
||||
LVyan7cm31uernNK2G0n5ScUOnLTo4dVhqwas2v38kAxS6BOlDgqXAZpXssz/PDY
|
||||
viHTp/jLS+jC7BP89lrl2U53UMxx3em+FIdWmZ54wNcLrxD585Ibcu2ajRS2PQT9
|
||||
szh8W0fEkyKCk1MOhCMFvXOuO34BqO7kMev1QTQFQ5QeriQU/XB1MWFwKldX9C3b
|
||||
0l5c4zJ1msTXpm/IyD21JtYfCJiH3Ny3Ojl864Ud5pdG0xQ2d4qHE6fvSRvzz9aG
|
||||
wAntmFUCgYEAxW4BZctTwrNMZwrXV4kNKBhkh6zl4BMxMexB0ze0N3aJmPEAcJUb
|
||||
7uoT6YiN3XA7DBRYOf7MmcpcLyqJi5Yt9T24xGIbU50jgxMw1KMIHf0lbGKQxDJN
|
||||
mdvLKM56O137qvnhaRYOX73pq2+DWD4UT1C/SWF+dkMTuG5aRz94h78CgYEAz4Nh
|
||||
8IRZMET+7dg66qpGqZ37Y2igANrxvMwKmY9VlAbHRZu3fpt8WwQ2Vs4CFk/rfmbI
|
||||
5H6mAUVu8pKYvi910wWvLlPPh/34Sxoj4TJ6QRlP0pD3Um8O/PYa/0joNlf/rZjw
|
||||
6O/7UkXjzubOUHqcEnYWEm+oym2e0Av/8yLWazcCgYAvZgej3rrPRaiUHIAync6w
|
||||
z5pjEFloAHORHr127iqwHh9Ovp1yafn40+3P5V7ZyPYEImZEFi4cxf53vGilQHrs
|
||||
I9NWIo+Y9WLvNw5EHpf2Sy5O5SMIV2NWCvStaVTjJ98h3zgEuKzew4N0CyOnbdAG
|
||||
csZZ4bQwxE3Zu3SlIlHXlQKBgQCabEj79WFxvEaBtMHTU7eWDcy/o2I+gLAYMTdK
|
||||
IxIqQAkW0dRxUT/vc2kEm/WNqRe0TsT81QqwM31m4pTsIuFpkfdVYGU17FdTfDZr
|
||||
JWc4/p8aMWr7W04qDPL2Oskjd8T66K+OiNfb18q5c6Tg2v0998ZhHdrcGUtvwx5L
|
||||
TweFbQKBgFOC+y+2mVhlKyUxibiGmN8LAmnFYmaFyHgvQDCOOIpo0VAILf8LyEQf
|
||||
EYQteM3us92jBUOuZ8NMUaI2sBCzaLPpZTkL02iIr13Y0IYtu0LgIfI157kXjoow
|
||||
ByRl9TClp9RAkmbJVCxDsgR9nJVW1L1cCw8M1p5+n9SQMJFUQghW
|
||||
-----END RSA PRIVATE KEY-----
|
||||
|
@@ -1,24 +1,24 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIEEzCCAvugAwIBAgIUA4E1YyD1y228iRlwJgN2TpMYiXMwDQYJKoZIhvcNAQEL
|
||||
MIIEEzCCAvugAwIBAgIUCHfFkPZDhLT2oK0fhu2TP4xZtcwwDQYJKoZIhvcNAQEL
|
||||
BQAwbzEMMAoGA1UEBhMDVVNBMRMwEQYDVQQIEwpDYWxpZm9ybmlhMRYwFAYDVQQH
|
||||
Ew1TYW4gRnJhbmNpc2NvMQ0wCwYDVQQKEwRldGNkMRYwFAYDVQQLEw1ldGNkIFNl
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTAxMjExNDQwMDBaFw0yOTAxMTgxNDQw
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTEwMDgyMTE5MDBaFw0yOTEwMDUyMTE5
|
||||
MDBaMHkxDDAKBgNVBAYTA1VTQTETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UE
|
||||
BxMNU2FuIEZyYW5jaXNjbzENMAsGA1UEChMEZXRjZDEWMBQGA1UECxMNZXRjZCBT
|
||||
ZWN1cml0eTEVMBMGA1UEAxMMZXhhbXBsZTIuY29tMIIBIjANBgkqhkiG9w0BAQEF
|
||||
AAOCAQ8AMIIBCgKCAQEA3Jo8Uwuzyj5BU4Rx6eBA0bkIccRgruUWpZDQ1e0fO9Au
|
||||
WgYcRSE5O/EiJW6MpHKhlT3jhBw9bmzDrcAF+Hre/1hCfRPP3udDerYAEbpHEQ4a
|
||||
y1c/FlsRqaOPWS3b+Yp+RQCvyDQGiJ7TPifm0vU4opeeM+MUFmnFOdDxS0FxJOWD
|
||||
JCdCckr4Srghn56srwQPJISzS0q73YT5hv1lgg4yr3PHtEzBYvzw2yRxY8nEE6ZM
|
||||
8hGnOwjDn9FNRMPr13wd05w94hq8akDHqvRp4yHKXfsw11fW7jUj+1D2GsHL4VVT
|
||||
mafx+RJsHy33/Lou5eJ5HIOeYsjX3PDs2KS7MlZX8wIDAQABo4GcMIGZMA4GA1Ud
|
||||
AAOCAQ8AMIIBCgKCAQEAv6FwE/OeAwAmZutb+DTP+y7FKpONm0+aHtESlrmTRmDo
|
||||
iLGuoEXjYcFOCHr6gQJTYKEAivBK3TIN0dRirNBqLQxssDlsbU4ZXG++OciH/OoR
|
||||
8E+VmMsbqof/E0nhVFYDumnuoS+waX8elzuDjDX4u7F+d1/gIb8aYU1VDjtZxF2b
|
||||
vqaPyroOn5HvBs4MW+BpAB4guHfDXpK/oAnJDsq9JTUZqoG1xOZfHNhA/iVBSAJv
|
||||
hO6aBxDCjzwO1gT5kTbNELrBCJ0V2NXlHFcBOPhxyl4+DIPwa0Q8oExUEOzxnHFu
|
||||
U3GPoklbLp0RNNiqHHKfD4yaLI1rf8dH5AZG/QjK4wIDAQABo4GcMIGZMA4GA1Ud
|
||||
DwEB/wQEAwIFoDAdBgNVHSUEFjAUBggrBgEFBQcDAQYIKwYBBQUHAwIwDAYDVR0T
|
||||
AQH/BAIwADAdBgNVHQ4EFgQUj+ruFZbJNhjHRC90urHmkegftsgwHwYDVR0jBBgw
|
||||
FoAU3WUwJ6HGPOZR4ckfwO5SZKd7dpowGgYDVR0RBBMwEYIJbG9jYWxob3N0hwR/
|
||||
AAABMA0GCSqGSIb3DQEBCwUAA4IBAQCcn0Y/zym6tla+ku/sMiAMrvB1NZNVd1eE
|
||||
HxPns99H7qdlGQ9bnxf1D1WxKu9HztmG3DhfdYUxYA3Z772+rrx3AhVQYquZAvvK
|
||||
kdZ6ixHynY90Q5PX48BlgZvzd884RZFDjwSCoGkbdYqTtD9FfuGS/fdYcHZJvVAJ
|
||||
1oRXH0D0e3I1piUAzfFkLTXqgmBHh1zgNHqq3CVcjA/4CVGn2foI5r7yH9TIO+Dt
|
||||
hegJjObxJQJjobDJ5ijXbkPLt+LC7OFEDGbn8IKBIMmLYXULGIJR8wLKpiaBLV6G
|
||||
zoME6J9TZe2g2VQYcPxwshesIx8MAcqNVjOBI6pBey5t5Lqhbj4Z
|
||||
AQH/BAIwADAdBgNVHQ4EFgQUS6HUgF/GO55DtrxLN2dcyGACgnQwHwYDVR0jBBgw
|
||||
FoAUZQe+r42mchS6xFBpug0H8o5jej4wGgYDVR0RBBMwEYIJbG9jYWxob3N0hwR/
|
||||
AAABMA0GCSqGSIb3DQEBCwUAA4IBAQC2pcpQZVkvZO4TC1zluGhG6Yhz+fzZuNlp
|
||||
jHa0U+WIdTPerqZZ98iU1hCm3F6voNjAQ0eHRV3z5q/rDph8AR7cVl9cT2rLO/zA
|
||||
F6M4QDMetddj7EIUq2/B1CWzHkOvwcJFgc0OfBWzJYAShFv/B7Ir1WpdtixOvyOH
|
||||
kWsWoy1WmatQvBQ2jDrvdGRWhqsPmg2uGbJrUjABeYtc5whQr0zscy+jEIrDpqPT
|
||||
VVuUu19/ALvdv2kOC+ayhH+vTAvEA38P6wlavDlgsv/M902ORWahdLQ/H0XX+gVP
|
||||
QDe3MyrBR6QkjAfKJvnMv/5x8mj+AFWfnALnRb9j3/q1UOwITWN2
|
||||
-----END CERTIFICATE-----
|
||||
|
@@ -1,27 +1,27 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIEogIBAAKCAQEA3Jo8Uwuzyj5BU4Rx6eBA0bkIccRgruUWpZDQ1e0fO9AuWgYc
|
||||
RSE5O/EiJW6MpHKhlT3jhBw9bmzDrcAF+Hre/1hCfRPP3udDerYAEbpHEQ4ay1c/
|
||||
FlsRqaOPWS3b+Yp+RQCvyDQGiJ7TPifm0vU4opeeM+MUFmnFOdDxS0FxJOWDJCdC
|
||||
ckr4Srghn56srwQPJISzS0q73YT5hv1lgg4yr3PHtEzBYvzw2yRxY8nEE6ZM8hGn
|
||||
OwjDn9FNRMPr13wd05w94hq8akDHqvRp4yHKXfsw11fW7jUj+1D2GsHL4VVTmafx
|
||||
+RJsHy33/Lou5eJ5HIOeYsjX3PDs2KS7MlZX8wIDAQABAoIBAAurkSNnjBRX5MbR
|
||||
S+Fufp9ZpYu3MtItxlvt5E5c4/kbE0ip6Bb0If38zKykQ4Zzlf66Lm4PvGQ6FsP7
|
||||
U5WIkoF5ntLVGHsXSueT9z741sedMmetbuX03WNXBr9WALwbyoBGAAacUfgqRsyT
|
||||
+c8RL/TSBomCKs7gu5FKb2lmxeCItpeyxGPp35AnXR1B5Q9GK7IoFnYlpJO5Vqbs
|
||||
t1cpOikotKTFrylq6hxGbRBtzBnrOLh51UneXR0r83oGzSrpHm5+BLMG5orYnK10
|
||||
UKBge1HijLjG9EDpCLdR+T/IJyz3oQyFCPRVFrIq5/76C8LsMSSjKdM6apCMc9d7
|
||||
NMnHPYkCgYEA6AWosramFnuAjihEkaBgJdpFRHfco+FlpglcS5cvaORmkW7dLl6R
|
||||
SZaO/mRJuhtC3j+LBXdGaP6rMdW7it9wRXs160e5mfSrEb7H66bgD0FfIG+iaEPV
|
||||
8OXESgrSUI69neddRgL2QXZeY++hSzaMNGef2lPpFd+irgkud+bltSUCgYEA82Z1
|
||||
/VVubW/lcnz2jBehssXL5yqNa/ySVvEWHdcKG6b0RNsoQCmOUZnBIwqDLMT4+62v
|
||||
e3bYNi/y44oJi741ifT6prPPQqv+J3tsEAQ0dlcscs71q4TXJEpLIoSlM+lQAsYA
|
||||
IhOkeLA5YKr2RzjnD9m2d+awErK6GGtCQqGbqTcCgYBHzgiJVlFMP+hnFjsyPknD
|
||||
LSumptmXthe5LMhtdFptwdGkTIUS5p1cAsY6IFtYxzsIgO3LQUB/GeFtSNIDhma+
|
||||
egUTzVy5MqkGkt5YJYrN7dM8vI+saOH67YCz0WmJGMPB7GpHUn9XfwRzNSPbnQQC
|
||||
69biwHkwFcfIyHqjDMgmKQKBgFWsTwOouoHSzRSLX8zX1Ja0gJ0RoU3NTUVE/t/p
|
||||
/SjWj0xdR8Gt3uZiFFVdMebkPi94ZzgyENCh+ACXadzCSt4QWNmsmNuC5qbHDrZk
|
||||
hILTFFYk+twwmfmwHNo9jljDWuJfB1T3TQEeJlQcWSugn9Q4cb2qeXdbaZ2Gw3/o
|
||||
mn3pAoGAbrBBySSjr8AnCmW5ipJ+ys1f8Q5Sg3Skvvx72KNH640AFY9c94glumjZ
|
||||
7Ryz1BcVIVszaJt9q21xQZeCpQs6VBHQB04AKlJLq/wOccg2Qb4qD8++zNyK5Ndm
|
||||
Bht4nGrX+sGDD4EwLKUCTNgxBgQju6ZQrT3b05xlTsRG0fA7SQQ=
|
||||
MIIEpAIBAAKCAQEAv6FwE/OeAwAmZutb+DTP+y7FKpONm0+aHtESlrmTRmDoiLGu
|
||||
oEXjYcFOCHr6gQJTYKEAivBK3TIN0dRirNBqLQxssDlsbU4ZXG++OciH/OoR8E+V
|
||||
mMsbqof/E0nhVFYDumnuoS+waX8elzuDjDX4u7F+d1/gIb8aYU1VDjtZxF2bvqaP
|
||||
yroOn5HvBs4MW+BpAB4guHfDXpK/oAnJDsq9JTUZqoG1xOZfHNhA/iVBSAJvhO6a
|
||||
BxDCjzwO1gT5kTbNELrBCJ0V2NXlHFcBOPhxyl4+DIPwa0Q8oExUEOzxnHFuU3GP
|
||||
oklbLp0RNNiqHHKfD4yaLI1rf8dH5AZG/QjK4wIDAQABAoIBAE5QTHxq4BV71zXS
|
||||
U7ig5KpTV9JpkMJ7CpIzgTRFzNFDQ2SxsJrhVOabWCeREpTsfWSNB6rAPugcz5cE
|
||||
A/t6BRo57KUsIoqdEzI6nHQC5shOZFxgOdPClaDgiTa5x7Nun4FsT1BiK+dBQyAs
|
||||
+zqux+L0y6k/blp8Peyr7OmvCaV8osB4/JLLH/WHt2wWgqFWisyIT7/D/gQlQn81
|
||||
Hvv84BAL9y8iyCmCzWhQL0YisLPyaFkGkb7DK4wznWxfQn3jRAkZDQMH675o/OHj
|
||||
8nL0NSdCA/MGLEtPAXegM7kMPCf68JwZV3gPZDyEK0JES1oT1z+op7JHuatlhgdL
|
||||
WTA10fkCgYEA1QXRRpOeZvHzGVMzrXrrgS5GeaR+XgjUalBGgu0w8nSET195oXu6
|
||||
Y8dVco4FlEZ0Wq7evA4M9XVJyKQkkEGR7Nkv922p8RhG+U73ajODANAQURIwqOPJ
|
||||
01IrfMIK2mkXDZwzkAxPaOnny2OMZtUznmZnNdJ/vLd7U0sScNPVQp8CgYEA5krD
|
||||
ImQ8U9/S4VOK78i3FMWMoutffXpW71lEc9tsz1YWUPf170raujjF5mqtBBXup9ko
|
||||
37CmVk6mOO2TdXLg1feMaVBsoblL6iPoBZot/fLdzgmICccpimst2yrUZEHwJpdk
|
||||
9k95xEZUQhN73eY/Ih6b5HZZ/ygxfAVvhDFzNT0CgYEAru5aDvUGbU9e7HsQwvNg
|
||||
FfMkWJwmUZ46oRtO7BFP0qqwRGYJAf0S8QEuQCY0mrDIt/dGXXPEXIV2k9eHVxch
|
||||
eDhaVXuuxJfFINIiBwpKGA7Ed27Smr6EbI7bu1W1h+oozjppdW9GfscmXDVhhMir
|
||||
3PYG54H298hM8/eAKzsps80CgYEAzk5Vp76ySNVv3spv4kYmtaYQSnef8RIjRYLs
|
||||
DvqY7NmLXnf0y618a22m5LfWTZ20Uov50QM40ILe6Ir1GjeS8jw1frc8yljsiFIo
|
||||
brRj1We4ivcA9vmD3mwMBZbF9RcZJAlmuj4SsOHsY9F+mxjEoDVZpP7duvbv9dIM
|
||||
yBlgw2UCgYB27Zwd3ta3x9mx2wRHaFMlcnwh+ERRGO6yAKndsr3L80Gkb2HxHlR3
|
||||
cA3dIg/YAGocjROOIO8MO5rclZm30qM5rlv0SH6gQ4nW16PZgHSuakgeX5fVHb4v
|
||||
nT8TKt0PY9r5e5cdHzpwlrDD1Q8GiAHvn0oH6oioGxlIV190YAHvGQ==
|
||||
-----END RSA PRIVATE KEY-----
|
||||
|
@@ -1,24 +1,24 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIID/DCCAuSgAwIBAgIUXO0yz2xs67pa/mT7ro2mx5d9+8gwDQYJKoZIhvcNAQEL
|
||||
MIID/DCCAuSgAwIBAgIUMgfWSv3FrD0dRcHmPNxLr7p9BXcwDQYJKoZIhvcNAQEL
|
||||
BQAwbzEMMAoGA1UEBhMDVVNBMRMwEQYDVQQIEwpDYWxpZm9ybmlhMRYwFAYDVQQH
|
||||
Ew1TYW4gRnJhbmNpc2NvMQ0wCwYDVQQKEwRldGNkMRYwFAYDVQQLEw1ldGNkIFNl
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTAxMjExNDQwMDBaFw0yOTAxMTgxNDQw
|
||||
Y3VyaXR5MQswCQYDVQQDEwJjYTAeFw0xOTEwMDgyMTE5MDBaFw0yOTEwMDUyMTE5
|
||||
MDBaMGIxDDAKBgNVBAYTA1VTQTETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UE
|
||||
BxMNU2FuIEZyYW5jaXNjbzENMAsGA1UEChMEZXRjZDEWMBQGA1UECxMNZXRjZCBT
|
||||
ZWN1cml0eTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBALf9KzBo3JFZ
|
||||
1Zi8lH4t6MSCx/de1MyHSR4S0sD+rl/GcdkdPLOLGKikHZLMU5Z9u1y5HK23QWBy
|
||||
KcdLFDkQckoUBDxAR+sxdVX5RgVTkMR9zdv/4HVertakKNutrno0lUdqhCtV05YR
|
||||
HeU8/tPMciyE4cBbeSLlvxGe49xW2cl0iZ4Fqu9xhCS1hlSLDNNN1Azw1hwptGkP
|
||||
3voTVWPzU95EXhqtnDhf7QO4QpcIZ44Zm0pWMoyip9TE16bTwjltiAdOs2shK+9h
|
||||
oQ9b2OW69cWpvtwnWsWAACveQHFUZ7drIDc7cv3OPvo7aonx4CHKqxEqnmlIxipP
|
||||
t8nkKtyHr10CAwEAAaOBnDCBmTAOBgNVHQ8BAf8EBAMCBaAwHQYDVR0lBBYwFAYI
|
||||
KwYBBQUHAwEGCCsGAQUFBwMCMAwGA1UdEwEB/wQCMAAwHQYDVR0OBBYEFPc/GrBA
|
||||
B2mcJnSACyKgkNt5GWvrMB8GA1UdIwQYMBaAFN1lMCehxjzmUeHJH8DuUmSne3aa
|
||||
ZWN1cml0eTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBALKKf9zhwcgu
|
||||
S53Vd92PLqHtDNpTbl4nM/5qYy7nHV5FHW7uolQHFA0BjJwxz7/LC39G950PpSIs
|
||||
K7Y99P7aMPVqVH5FGL+uoDbHMzt/gIUGgWj51J38+x6zN/9vIvAVIBClBhzEuB2k
|
||||
WJU6KyB1V5G+1wnnKRXLB9QC0f/7vqd9f21O7sJmWeVhGJuEUwwAp1p5WDGM2Tn8
|
||||
Fjy57O9f1nT4WVqWhB5EbvYGDF2Z5DWyKz90EWOwVw30ThcQHF57X6WJdlNiQOrY
|
||||
KIWO475QKPwbUSpRkvw1jwvllU8s4l6pB1hKTAcUK3UsWrzpf2+m8v8ou35uFD/c
|
||||
pd2bBx5MVlkCAwEAAaOBnDCBmTAOBgNVHQ8BAf8EBAMCBaAwHQYDVR0lBBYwFAYI
|
||||
KwYBBQUHAwEGCCsGAQUFBwMCMAwGA1UdEwEB/wQCMAAwHQYDVR0OBBYEFAcMdmvX
|
||||
8NC4V4Z67Jb/pNUwai7cMB8GA1UdIwQYMBaAFGUHvq+NpnIUusRQaboNB/KOY3o+
|
||||
MBoGA1UdEQQTMBGCCWxvY2FsaG9zdIcEfwAAATANBgkqhkiG9w0BAQsFAAOCAQEA
|
||||
F30eIP+5u2gZItB9VpohOFjKoW2Ts+TIGT+nE2/b70kpX1JXuOHvJSFgYxn1N9KK
|
||||
tJ03T52ENwhFw7ES9sYmpAKjt/Z+2W1DnY1n29fWvGYKReOt3TjjPQcC26v9l+Kf
|
||||
oNSFQEac2rdzKZEzkPDRpvlwVQuACGNbLpbpFmV4G1lavt0ecRuhohHgCtO03nx+
|
||||
UOQ1Enla3gA002qw2OI71cmrxYzpS3tgZCGR3QsOOibls+/1kBQE/F4HIyhLIImo
|
||||
dGC98Q1hLYVS8oKwK+DifK4mDdRSbXE7ek/5+vMOsKWdcs2dTcn9WoijJV6YVX/e
|
||||
HU7ocaMm/c5SQ0yndcCU2w==
|
||||
btQjJEWHD/0gYsNLFg3tDxZ64U/HfNlh4USGOK02VL2LteMcV8AoYlZ3jwmp4+33
|
||||
D3HlqLclJNABax2pOvTHVnQlf25TSNwJRtmzOvcg+6xYbPdgRoeEVsWbmgbpX7Vi
|
||||
P8FYelYCiYTPezjqZgPG1gmq0Uf/drlTrjwsG2njEcuK7hip+LdJnIrtpIrabpIk
|
||||
lZRa7Y/JBM3gP/rR1fu9lhzJ97s3NabuHzPwyouSTTknaaiGwSV8F5frh9NGcFhd
|
||||
G7giCLZLKklQB4IUTOFcVFSZmeAGy6KBqyT10N2kkBrsrcWhyMKIU9X0+6hh3Tlc
|
||||
JEla9as6qFvt1dFGp+qeww==
|
||||
-----END CERTIFICATE-----
|
||||
|
@@ -1,27 +1,27 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIEpAIBAAKCAQEAt/0rMGjckVnVmLyUfi3oxILH917UzIdJHhLSwP6uX8Zx2R08
|
||||
s4sYqKQdksxTln27XLkcrbdBYHIpx0sUORByShQEPEBH6zF1VflGBVOQxH3N2//g
|
||||
dV6u1qQo262uejSVR2qEK1XTlhEd5Tz+08xyLIThwFt5IuW/EZ7j3FbZyXSJngWq
|
||||
73GEJLWGVIsM003UDPDWHCm0aQ/e+hNVY/NT3kReGq2cOF/tA7hClwhnjhmbSlYy
|
||||
jKKn1MTXptPCOW2IB06zayEr72GhD1vY5br1xam+3CdaxYAAK95AcVRnt2sgNzty
|
||||
/c4++jtqifHgIcqrESqeaUjGKk+3yeQq3IevXQIDAQABAoIBACc8fjFcq2zz03E3
|
||||
BhPChpkhqrM+LfNQBm+7Y+Z0aYtTLoQ2j/NZ/nA4T3Y2zLyTI5mqgEsrgW2n6vDt
|
||||
OerRbw4NJroSm0O00Gj9N8l7AKxmh7ga4SsmffKYH355k3PMMul/9Z+oNe8Sx2VT
|
||||
lKRmPRLxY5M57qNai1yencknXFpxetSzHwusfDFFNiStN8rTDeIgJtn9FKxhMAcJ
|
||||
P0D8rHH+x4/8XGSVtjANJr22cj1win0dce4tVVUQp60pBAFhGIY+uPOM8vmxkJYx
|
||||
xQTmR6bh7S5KrYT1BNK3cpo6gQGE6HJsPzTuijN6hqvJ7ufTl/lq2T0ETVEemHOf
|
||||
B5NKOwECgYEAxigkRZeaCsvvUcPWEYrnIhZt6SfCx5iKlkVQsG/uYps3aWohkBAY
|
||||
Ws6rr55qkH7hsye9ikeyhT9VDnWHeAbTr2SKIsa8CAc+gQoPzYfY4GVYzbNP+/rj
|
||||
d1jS/s5/LyVryi3ln3EJN1fUHv+ZzvEWORSa/WVHkBuxMxoeHX8PhSkCgYEA7bJK
|
||||
inBlLXtUy+c/q5DigYLIg0MquQ9slcfJapcwgswmbLnagtW9oQxlsM/uSiqR4br+
|
||||
9SKD7TNLNhwJmByG6HlPyjCF2FGPhUg4pIWuRckPqJ6o2SgONupD3mZdu/CyzsuO
|
||||
pyCqq0d2yeLXW77MxidMUeNfPsAA9lvRRulhCxUCgYEAwbM1ma2TD+DABP5ZQHa2
|
||||
b3TbZeHPHgr31eLV+FLCBTPTG8F6I3gIRqPl4dsKMktFVzqOpiBl2qjI/URX8zVB
|
||||
Mh8mhM4duf9S0xLB1dhoYRnQj+srUZazSdPTFO9IFg8PaegpoQz+xFGfcdnLQSYb
|
||||
4hpJU0/wf2cCdYCfVZgB1NkCgYB9K/a1EJs3aEsvVYfiAVpGeWi+NxC4g7ba6WrY
|
||||
BuY0+u0BNJ4taAGEXdLvWZBS3jgUdzTsQlDXCLwCsqEayWsB4WBzSToywECkH3Q0
|
||||
r3EmrsrgMS0Zrk5N/O/gnmeeIRMIc4mb2UgHCoszpZFjbwbHEsrOFL6DfPkEwzVh
|
||||
8mR4QQKBgQC88WRxSGnF0AyoM5tB8/x80O9a4CLsSwfFVzEyJcdh3uL9+JzfNVta
|
||||
YsXy5ah2SyBrI7xQd/LXhv3/7GY4eZIQuwcOH74hQNHdm2u3bWfmg39jH+HgWK8m
|
||||
mqYvWYE9qLXIVwBzsCzQtxq4z90KMiZ2qXV0YMFPGjZt+bBkbjJTOw==
|
||||
MIIEpAIBAAKCAQEAsop/3OHByC5LndV33Y8uoe0M2lNuXicz/mpjLucdXkUdbu6i
|
||||
VAcUDQGMnDHPv8sLf0b3nQ+lIiwrtj30/tow9WpUfkUYv66gNsczO3+AhQaBaPnU
|
||||
nfz7HrM3/28i8BUgEKUGHMS4HaRYlTorIHVXkb7XCecpFcsH1ALR//u+p31/bU7u
|
||||
wmZZ5WEYm4RTDACnWnlYMYzZOfwWPLns71/WdPhZWpaEHkRu9gYMXZnkNbIrP3QR
|
||||
Y7BXDfROFxAcXntfpYl2U2JA6tgohY7jvlAo/BtRKlGS/DWPC+WVTyziXqkHWEpM
|
||||
BxQrdSxavOl/b6by/yi7fm4UP9yl3ZsHHkxWWQIDAQABAoIBAQCVyfrCDqlsT+Li
|
||||
1UBOIp0l/uIEnXCAD3XgodL6e6249FVgR1brFlEtJDqapHO+XhQUQS7ml0ScqeA2
|
||||
cj6EPfxLOV0P3tqHnnMN4gvKhAsID9AsiUVnEuJ//C4j4FK4h5CyRjEdm7E4NTSY
|
||||
ZgfeoHPKdAinZ0eh4Ad+SKt0jvmCPDD1L+6bxpJ6E258xPDxB71rHTCgnwZZmXrN
|
||||
rHDg07tVVU6lYtXEsZAsyIBIxXV/RaCt4xSijM1C7kuSsUE3+CCw+pzQUvHDxkuk
|
||||
FPxE5hONzkUaCSBKTv4L6gVaiYa30Jo9THuTRWvzDJmcnNwlYgRLeIR8PJY0eHqv
|
||||
FYJbLdQBAoGBAOeiW1aHpNO+K4BqVMzm7fvz0a3DBkml/wIKSvw3dbTYnyIXFvDB
|
||||
Be7OZLhiWPwaE/58aQWh+/OGCr88yyCLAt2mOQ5aMalKpdSCaY5swEnzt923pY9z
|
||||
jt3DnhX8aXggPbqM6eJjaxJY7jIMSDNKUQZKJeSesr/EzFo0CRkkepyBAoGBAMVS
|
||||
Z2nIY+G2+P1VSUEbI3dbaZ3ciMEDLc+rZQs0fx8+xQCD71eCU9ggxf+O0R1/0smm
|
||||
9so65KrmKl3yOt8OQW3YgUpQqIQdJPHnfuTnsU7y/+zRl6k119gV89LLDlCM7nfW
|
||||
5/ey/iJLXQxfC7OoFF/hQM0odmorA8jBuqKDWy3ZAoGAZL6gq0njzpRvpzKYH2Zx
|
||||
K5woHkMsgOvJtcF0S65za2ysCc+xEpVhVzQ9alScD0noWE8T/nctdgVetz5huo27
|
||||
eVvKhQuFffQRnBP8hQ2XtJJj7fLp9zJzeNCT+UwHM1ASiQiw0N4cu6YiM3JUFLrF
|
||||
8s5dHMpJRE778l+fdWgATAECgYEAmJ3osE+2uUCtCjvpwbp8zvdcFCYbe7W6vBGj
|
||||
wGvlGsSQ2JozB2sc8GBA5C2RHhDcdu11meq9LFWDVVBiKl27S3uWXGVQQYbNKXDU
|
||||
m7V8VUTrnz5o4A5uGIq6IEK/mpu2YehNWC8QEnRZzpTA1z7cK2Bsn4F5PRpx/deh
|
||||
Q8r3PdkCgYBd2rT+S4/51C5AnIhgMF/PYl0+DYMFD8HAfsx7VTaIUmFQ4devHMOz
|
||||
J6lbqRyEITZtXgna1n35LkyBDcPwsEtntjJOP+xneCtKzozdzhXyoAw2xQR3gqvV
|
||||
7YtV3miYQiOTqjOefViMhR/XiOV2zng3OId1AQObfOUZODJPfSN26g==
|
||||
-----END RSA PRIVATE KEY-----
|
||||
|
@@ -27,6 +27,7 @@ import (
|
||||
"go.etcd.io/etcd/mvcc"
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/pkg/testutil"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
@@ -173,7 +174,7 @@ func TestV3CorruptAlarm(t *testing.T) {
|
||||
// NOTE: cluster_proxy mode with namespacing won't set 'k', but namespace/'k'.
|
||||
s.Put([]byte("abc"), []byte("def"), 0)
|
||||
s.Put([]byte("xyz"), []byte("123"), 0)
|
||||
s.Compact(5)
|
||||
s.Compact(traceutil.TODO(), 5)
|
||||
s.Commit()
|
||||
s.Close()
|
||||
be.Close()
|
||||
|
@@ -357,6 +357,7 @@ func TestV3AuthNonAuthorizedRPCs(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestV3AuthOldRevConcurrent(t *testing.T) {
|
||||
t.Skip() // TODO(jingyih): re-enable the test when #10408 is fixed.
|
||||
defer testutil.AfterTest(t)
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
@@ -369,13 +369,27 @@ func (b *backend) defrag() error {
|
||||
|
||||
b.batchTx.tx = nil
|
||||
|
||||
tmpdb, err := bolt.Open(b.db.Path()+".tmp", 0600, boltOpenOptions)
|
||||
// Create a temporary file to ensure we start with a clean slate.
|
||||
// Snapshotter.cleanupSnapdir cleans up any of these that are found during startup.
|
||||
dir := filepath.Dir(b.db.Path())
|
||||
temp, err := ioutil.TempFile(dir, "db.tmp.*")
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
options := bolt.Options{}
|
||||
if boltOpenOptions != nil {
|
||||
options = *boltOpenOptions
|
||||
}
|
||||
options.OpenFile = func(path string, i int, mode os.FileMode) (file *os.File, err error) {
|
||||
return temp, nil
|
||||
}
|
||||
tdbp := temp.Name()
|
||||
tmpdb, err := bolt.Open(tdbp, 0600, &options)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
dbp := b.db.Path()
|
||||
tdbp := tmpdb.Path()
|
||||
size1, sizeInUse1 := b.Size(), b.SizeInUse()
|
||||
if b.lg != nil {
|
||||
b.lg.Info(
|
||||
@@ -387,11 +401,17 @@ func (b *backend) defrag() error {
|
||||
zap.String("current-db-size-in-use", humanize.Bytes(uint64(sizeInUse1))),
|
||||
)
|
||||
}
|
||||
|
||||
// gofail: var defragBeforeCopy struct{}
|
||||
err = defragdb(b.db, tmpdb, defragLimit)
|
||||
if err != nil {
|
||||
tmpdb.Close()
|
||||
os.RemoveAll(tmpdb.Path())
|
||||
if rmErr := os.RemoveAll(tmpdb.Path()); rmErr != nil {
|
||||
if b.lg != nil {
|
||||
b.lg.Error("failed to remove db.tmp after defragmentation completed", zap.Error(rmErr))
|
||||
} else {
|
||||
plog.Fatalf("failed to remove db.tmp after defragmentation completed: %v", rmErr)
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
@@ -411,6 +431,7 @@ func (b *backend) defrag() error {
|
||||
plog.Fatalf("cannot close database (%s)", err)
|
||||
}
|
||||
}
|
||||
// gofail: var defragBeforeRename struct{}
|
||||
err = os.Rename(tdbp, dbp)
|
||||
if err != nil {
|
||||
if b.lg != nil {
|
||||
|
@@ -18,6 +18,7 @@ import (
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/mvcc/mvccpb"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
)
|
||||
|
||||
type RangeOptions struct {
|
||||
@@ -102,10 +103,10 @@ type KV interface {
|
||||
WriteView
|
||||
|
||||
// Read creates a read transaction.
|
||||
Read() TxnRead
|
||||
Read(trace *traceutil.Trace) TxnRead
|
||||
|
||||
// Write creates a write transaction.
|
||||
Write() TxnWrite
|
||||
Write(trace *traceutil.Trace) TxnWrite
|
||||
|
||||
// Hash computes the hash of the KV's backend.
|
||||
Hash() (hash uint32, revision int64, err error)
|
||||
@@ -114,7 +115,7 @@ type KV interface {
|
||||
HashByRev(rev int64) (hash uint32, revision int64, compactRev int64, err error)
|
||||
|
||||
// Compact frees all superseded keys with revisions less than rev.
|
||||
Compact(rev int64) (<-chan struct{}, error)
|
||||
Compact(trace *traceutil.Trace, rev int64) (<-chan struct{}, error)
|
||||
|
||||
// Commit commits outstanding txns into the underlying backend.
|
||||
Commit()
|
||||
|
@@ -25,6 +25,7 @@ import (
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/mvcc/mvccpb"
|
||||
"go.etcd.io/etcd/pkg/testutil"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
dto "github.com/prometheus/client_model/go"
|
||||
@@ -47,7 +48,7 @@ var (
|
||||
return kv.Range(key, end, ro)
|
||||
}
|
||||
txnRangeFunc = func(kv KV, key, end []byte, ro RangeOptions) (*RangeResult, error) {
|
||||
txn := kv.Read()
|
||||
txn := kv.Read(traceutil.TODO())
|
||||
defer txn.End()
|
||||
return txn.Range(key, end, ro)
|
||||
}
|
||||
@@ -56,7 +57,7 @@ var (
|
||||
return kv.Put(key, value, lease)
|
||||
}
|
||||
txnPutFunc = func(kv KV, key, value []byte, lease lease.LeaseID) int64 {
|
||||
txn := kv.Write()
|
||||
txn := kv.Write(traceutil.TODO())
|
||||
defer txn.End()
|
||||
return txn.Put(key, value, lease)
|
||||
}
|
||||
@@ -65,7 +66,7 @@ var (
|
||||
return kv.DeleteRange(key, end)
|
||||
}
|
||||
txnDeleteRangeFunc = func(kv KV, key, end []byte) (n, rev int64) {
|
||||
txn := kv.Write()
|
||||
txn := kv.Write(traceutil.TODO())
|
||||
defer txn.End()
|
||||
return txn.DeleteRange(key, end)
|
||||
}
|
||||
@@ -182,7 +183,7 @@ func testKVRangeBadRev(t *testing.T, f rangeFunc) {
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
put3TestKVs(s)
|
||||
if _, err := s.Compact(4); err != nil {
|
||||
if _, err := s.Compact(traceutil.TODO(), 4); err != nil {
|
||||
t.Fatalf("compact error (%v)", err)
|
||||
}
|
||||
|
||||
@@ -409,7 +410,7 @@ func TestKVTxnBlockWriteOperations(t *testing.T) {
|
||||
func() { s.DeleteRange([]byte("foo"), nil) },
|
||||
}
|
||||
for i, tt := range tests {
|
||||
txn := s.Write()
|
||||
txn := s.Write(traceutil.TODO())
|
||||
done := make(chan struct{}, 1)
|
||||
go func() {
|
||||
tt()
|
||||
@@ -438,7 +439,7 @@ func TestKVTxnNonBlockRange(t *testing.T) {
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
txn := s.Write()
|
||||
txn := s.Write(traceutil.TODO())
|
||||
defer txn.End()
|
||||
|
||||
donec := make(chan struct{})
|
||||
@@ -460,7 +461,7 @@ func TestKVTxnOperationInSequence(t *testing.T) {
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
txn := s.Write()
|
||||
txn := s.Write(traceutil.TODO())
|
||||
base := int64(i + 1)
|
||||
|
||||
// put foo
|
||||
@@ -544,7 +545,7 @@ func TestKVCompactReserveLastValue(t *testing.T) {
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
_, err := s.Compact(tt.rev)
|
||||
_, err := s.Compact(traceutil.TODO(), tt.rev)
|
||||
if err != nil {
|
||||
t.Errorf("#%d: unexpect compact error %v", i, err)
|
||||
}
|
||||
@@ -580,7 +581,7 @@ func TestKVCompactBad(t *testing.T) {
|
||||
{100, ErrFutureRev},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
_, err := s.Compact(tt.rev)
|
||||
_, err := s.Compact(traceutil.TODO(), tt.rev)
|
||||
if err != tt.werr {
|
||||
t.Errorf("#%d: compact error = %v, want %v", i, err, tt.werr)
|
||||
}
|
||||
@@ -626,7 +627,7 @@ func TestKVRestore(t *testing.T) {
|
||||
func(kv KV) {
|
||||
kv.Put([]byte("foo"), []byte("bar0"), 1)
|
||||
kv.Put([]byte("foo"), []byte("bar1"), 2)
|
||||
kv.Compact(1)
|
||||
kv.Compact(traceutil.TODO(), 1)
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
|
@@ -14,24 +14,27 @@
|
||||
|
||||
package mvcc
|
||||
|
||||
import "go.etcd.io/etcd/lease"
|
||||
import (
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
)
|
||||
|
||||
type readView struct{ kv KV }
|
||||
|
||||
func (rv *readView) FirstRev() int64 {
|
||||
tr := rv.kv.Read()
|
||||
tr := rv.kv.Read(traceutil.TODO())
|
||||
defer tr.End()
|
||||
return tr.FirstRev()
|
||||
}
|
||||
|
||||
func (rv *readView) Rev() int64 {
|
||||
tr := rv.kv.Read()
|
||||
tr := rv.kv.Read(traceutil.TODO())
|
||||
defer tr.End()
|
||||
return tr.Rev()
|
||||
}
|
||||
|
||||
func (rv *readView) Range(key, end []byte, ro RangeOptions) (r *RangeResult, err error) {
|
||||
tr := rv.kv.Read()
|
||||
tr := rv.kv.Read(traceutil.TODO())
|
||||
defer tr.End()
|
||||
return tr.Range(key, end, ro)
|
||||
}
|
||||
@@ -39,13 +42,13 @@ func (rv *readView) Range(key, end []byte, ro RangeOptions) (r *RangeResult, err
|
||||
type writeView struct{ kv KV }
|
||||
|
||||
func (wv *writeView) DeleteRange(key, end []byte) (n, rev int64) {
|
||||
tw := wv.kv.Write()
|
||||
tw := wv.kv.Write(traceutil.TODO())
|
||||
defer tw.End()
|
||||
return tw.DeleteRange(key, end)
|
||||
}
|
||||
|
||||
func (wv *writeView) Put(key, value []byte, lease lease.LeaseID) (rev int64) {
|
||||
tw := wv.kv.Write()
|
||||
tw := wv.kv.Write(traceutil.TODO())
|
||||
defer tw.End()
|
||||
return tw.Put(key, value, lease)
|
||||
}
|
||||
|
@@ -29,6 +29,7 @@ import (
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/mvcc/mvccpb"
|
||||
"go.etcd.io/etcd/pkg/schedule"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
|
||||
"github.com/coreos/pkg/capnslog"
|
||||
"go.uber.org/zap"
|
||||
@@ -140,7 +141,7 @@ func NewStore(lg *zap.Logger, b backend.Backend, le lease.Lessor, ig ConsistentI
|
||||
s.ReadView = &readView{s}
|
||||
s.WriteView = &writeView{s}
|
||||
if s.le != nil {
|
||||
s.le.SetRangeDeleter(func() lease.TxnDelete { return s.Write() })
|
||||
s.le.SetRangeDeleter(func() lease.TxnDelete { return s.Write(traceutil.TODO()) })
|
||||
}
|
||||
|
||||
tx := s.b.BatchTx()
|
||||
@@ -269,9 +270,10 @@ func (s *store) updateCompactRev(rev int64) (<-chan struct{}, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (s *store) compact(rev int64) (<-chan struct{}, error) {
|
||||
func (s *store) compact(trace *traceutil.Trace, rev int64) (<-chan struct{}, error) {
|
||||
start := time.Now()
|
||||
keep := s.kvindex.Compact(rev)
|
||||
trace.Step("compact in-memory index tree")
|
||||
ch := make(chan struct{})
|
||||
var j = func(ctx context.Context) {
|
||||
if ctx.Err() != nil {
|
||||
@@ -288,6 +290,7 @@ func (s *store) compact(rev int64) (<-chan struct{}, error) {
|
||||
s.fifoSched.Schedule(j)
|
||||
|
||||
indexCompactionPauseMs.Observe(float64(time.Since(start) / time.Millisecond))
|
||||
trace.Step("schedule compaction")
|
||||
return ch, nil
|
||||
}
|
||||
|
||||
@@ -297,21 +300,21 @@ func (s *store) compactLockfree(rev int64) (<-chan struct{}, error) {
|
||||
return ch, err
|
||||
}
|
||||
|
||||
return s.compact(rev)
|
||||
return s.compact(traceutil.TODO(), rev)
|
||||
}
|
||||
|
||||
func (s *store) Compact(rev int64) (<-chan struct{}, error) {
|
||||
func (s *store) Compact(trace *traceutil.Trace, rev int64) (<-chan struct{}, error) {
|
||||
s.mu.Lock()
|
||||
|
||||
ch, err := s.updateCompactRev(rev)
|
||||
|
||||
trace.Step("check and update compact revision")
|
||||
if err != nil {
|
||||
s.mu.Unlock()
|
||||
return ch, err
|
||||
}
|
||||
s.mu.Unlock()
|
||||
|
||||
return s.compact(rev)
|
||||
return s.compact(trace, rev)
|
||||
}
|
||||
|
||||
// DefaultIgnores is a map of keys to ignore in hash checking.
|
||||
@@ -355,19 +358,7 @@ func (s *store) Restore(b backend.Backend) error {
|
||||
}
|
||||
|
||||
func (s *store) restore() error {
|
||||
b := s.b
|
||||
reportDbTotalSizeInBytesMu.Lock()
|
||||
reportDbTotalSizeInBytes = func() float64 { return float64(b.Size()) }
|
||||
reportDbTotalSizeInBytesMu.Unlock()
|
||||
reportDbTotalSizeInBytesDebugMu.Lock()
|
||||
reportDbTotalSizeInBytesDebug = func() float64 { return float64(b.Size()) }
|
||||
reportDbTotalSizeInBytesDebugMu.Unlock()
|
||||
reportDbTotalSizeInUseInBytesMu.Lock()
|
||||
reportDbTotalSizeInUseInBytes = func() float64 { return float64(b.SizeInUse()) }
|
||||
reportDbTotalSizeInUseInBytesMu.Unlock()
|
||||
reportDbOpenReadTxNMu.Lock()
|
||||
reportDbOpenReadTxN = func() float64 { return float64(b.OpenReadTxN()) }
|
||||
reportDbOpenReadTxNMu.Unlock()
|
||||
s.setupMetricsReporter()
|
||||
|
||||
min, max := newRevBytes(), newRevBytes()
|
||||
revToBytes(revision{main: 1}, min)
|
||||
@@ -579,6 +570,36 @@ func (s *store) ConsistentIndex() uint64 {
|
||||
return v
|
||||
}
|
||||
|
||||
func (s *store) setupMetricsReporter() {
|
||||
b := s.b
|
||||
reportDbTotalSizeInBytesMu.Lock()
|
||||
reportDbTotalSizeInBytes = func() float64 { return float64(b.Size()) }
|
||||
reportDbTotalSizeInBytesMu.Unlock()
|
||||
reportDbTotalSizeInBytesDebugMu.Lock()
|
||||
reportDbTotalSizeInBytesDebug = func() float64 { return float64(b.Size()) }
|
||||
reportDbTotalSizeInBytesDebugMu.Unlock()
|
||||
reportDbTotalSizeInUseInBytesMu.Lock()
|
||||
reportDbTotalSizeInUseInBytes = func() float64 { return float64(b.SizeInUse()) }
|
||||
reportDbTotalSizeInUseInBytesMu.Unlock()
|
||||
reportDbOpenReadTxNMu.Lock()
|
||||
reportDbOpenReadTxN = func() float64 { return float64(b.OpenReadTxN()) }
|
||||
reportDbOpenReadTxNMu.Unlock()
|
||||
reportCurrentRevMu.Lock()
|
||||
reportCurrentRev = func() float64 {
|
||||
s.revMu.RLock()
|
||||
defer s.revMu.RUnlock()
|
||||
return float64(s.currentRev)
|
||||
}
|
||||
reportCurrentRevMu.Unlock()
|
||||
reportCompactRevMu.Lock()
|
||||
reportCompactRev = func() float64 {
|
||||
s.revMu.RLock()
|
||||
defer s.revMu.RUnlock()
|
||||
return float64(s.compactMainRev)
|
||||
}
|
||||
reportCompactRevMu.Unlock()
|
||||
}
|
||||
|
||||
// appendMarkTombstone appends tombstone mark to normal revision bytes.
|
||||
func appendMarkTombstone(lg *zap.Logger, b []byte) []byte {
|
||||
if len(b) != revBytesLen {
|
||||
|
@@ -20,6 +20,7 @@ import (
|
||||
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
@@ -130,7 +131,7 @@ func BenchmarkStoreTxnPut(b *testing.B) {
|
||||
b.ResetTimer()
|
||||
b.ReportAllocs()
|
||||
for i := 0; i < b.N; i++ {
|
||||
txn := s.Write()
|
||||
txn := s.Write(traceutil.TODO())
|
||||
txn.Put(keys[i], vals[i], lease.NoLease)
|
||||
txn.End()
|
||||
}
|
||||
@@ -151,7 +152,7 @@ func benchmarkStoreRestore(revsPerKey int, b *testing.B) {
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
for j := 0; j < revsPerKey; j++ {
|
||||
txn := s.Write()
|
||||
txn := s.Write(traceutil.TODO())
|
||||
txn.Put(keys[i], vals[i], lease.NoLease)
|
||||
txn.End()
|
||||
}
|
||||
|
@@ -43,6 +43,7 @@ func (s *store) scheduleCompaction(compactMainRev int64, keep map[revision]struc
|
||||
rev = bytesToRev(key)
|
||||
if _, ok := keep[rev]; !ok {
|
||||
tx.UnsafeDelete(keyBucketName, key)
|
||||
keyCompactions++
|
||||
}
|
||||
}
|
||||
|
||||
|
@@ -22,6 +22,7 @@ import (
|
||||
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
@@ -109,7 +110,7 @@ func TestCompactAllAndRestore(t *testing.T) {
|
||||
|
||||
rev := s0.Rev()
|
||||
// compact all keys
|
||||
done, err := s0.Compact(rev)
|
||||
done, err := s0.Compact(traceutil.TODO(), rev)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@@ -34,6 +34,7 @@ import (
|
||||
"go.etcd.io/etcd/mvcc/mvccpb"
|
||||
"go.etcd.io/etcd/pkg/schedule"
|
||||
"go.etcd.io/etcd/pkg/testutil"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
@@ -331,7 +332,7 @@ func TestStoreCompact(t *testing.T) {
|
||||
key2 := newTestKeyBytes(revision{2, 0}, false)
|
||||
b.tx.rangeRespc <- rangeResp{[][]byte{key1, key2}, nil}
|
||||
|
||||
s.Compact(3)
|
||||
s.Compact(traceutil.TODO(), 3)
|
||||
s.fifoSched.WaitFinish(1)
|
||||
|
||||
if s.compactMainRev != 3 {
|
||||
@@ -582,7 +583,7 @@ func TestHashKVWhenCompacting(t *testing.T) {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
for i := 100; i >= 0; i-- {
|
||||
_, err := s.Compact(int64(rev - 1 - i))
|
||||
_, err := s.Compact(traceutil.TODO(), int64(rev-1-i))
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
@@ -609,7 +610,7 @@ func TestHashKVZeroRevision(t *testing.T) {
|
||||
for i := 2; i <= rev; i++ {
|
||||
s.Put([]byte("foo"), []byte(fmt.Sprintf("bar%d", i)), lease.NoLease)
|
||||
}
|
||||
if _, err := s.Compact(int64(rev / 2)); err != nil {
|
||||
if _, err := s.Compact(traceutil.TODO(), int64(rev/2)); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
@@ -639,7 +640,7 @@ func TestTxnPut(t *testing.T) {
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
for i := 0; i < sliceN; i++ {
|
||||
txn := s.Write()
|
||||
txn := s.Write(traceutil.TODO())
|
||||
base := int64(i + 2)
|
||||
if rev := txn.Put(keys[i], vals[i], lease.NoLease); rev != base {
|
||||
t.Errorf("#%d: rev = %d, want %d", i, rev, base)
|
||||
@@ -658,7 +659,7 @@ func TestConcurrentReadNotBlockingWrite(t *testing.T) {
|
||||
s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
|
||||
|
||||
// readTx simulates a long read request
|
||||
readTx1 := s.Read()
|
||||
readTx1 := s.Read(traceutil.TODO())
|
||||
|
||||
// write should not be blocked by reads
|
||||
done := make(chan struct{})
|
||||
@@ -673,7 +674,7 @@ func TestConcurrentReadNotBlockingWrite(t *testing.T) {
|
||||
}
|
||||
|
||||
// readTx2 simulates a short read request
|
||||
readTx2 := s.Read()
|
||||
readTx2 := s.Read(traceutil.TODO())
|
||||
ro := RangeOptions{Limit: 1, Rev: 0, Count: false}
|
||||
ret, err := readTx2.Range([]byte("foo"), nil, ro)
|
||||
if err != nil {
|
||||
@@ -730,7 +731,7 @@ func TestConcurrentReadTxAndWrite(t *testing.T) {
|
||||
defer wg.Done()
|
||||
time.Sleep(time.Duration(mrand.Intn(100)) * time.Millisecond) // random starting time
|
||||
|
||||
tx := s.Write()
|
||||
tx := s.Write(traceutil.TODO())
|
||||
numOfPuts := mrand.Intn(maxNumOfPutsPerWrite) + 1
|
||||
var pendingKvs kvs
|
||||
for j := 0; j < numOfPuts; j++ {
|
||||
@@ -756,7 +757,7 @@ func TestConcurrentReadTxAndWrite(t *testing.T) {
|
||||
mu.Lock()
|
||||
wKVs := make(kvs, len(committedKVs))
|
||||
copy(wKVs, committedKVs)
|
||||
tx := s.Read()
|
||||
tx := s.Read(traceutil.TODO())
|
||||
mu.Unlock()
|
||||
// get all keys in backend store, and compare with wKVs
|
||||
ret, err := tx.Range([]byte("\x00000000"), []byte("\xffffffff"), RangeOptions{})
|
||||
|
@@ -18,6 +18,7 @@ import (
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/mvcc/mvccpb"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
@@ -27,9 +28,11 @@ type storeTxnRead struct {
|
||||
|
||||
firstRev int64
|
||||
rev int64
|
||||
|
||||
trace *traceutil.Trace
|
||||
}
|
||||
|
||||
func (s *store) Read() TxnRead {
|
||||
func (s *store) Read(trace *traceutil.Trace) TxnRead {
|
||||
s.mu.RLock()
|
||||
s.revMu.RLock()
|
||||
// backend holds b.readTx.RLock() only when creating the concurrentReadTx. After
|
||||
@@ -38,7 +41,7 @@ func (s *store) Read() TxnRead {
|
||||
tx.RLock() // RLock is no-op. concurrentReadTx does not need to be locked after it is created.
|
||||
firstRev, rev := s.compactMainRev, s.currentRev
|
||||
s.revMu.RUnlock()
|
||||
return newMetricsTxnRead(&storeTxnRead{s, tx, firstRev, rev})
|
||||
return newMetricsTxnRead(&storeTxnRead{s, tx, firstRev, rev, trace})
|
||||
}
|
||||
|
||||
func (tr *storeTxnRead) FirstRev() int64 { return tr.firstRev }
|
||||
@@ -61,12 +64,12 @@ type storeTxnWrite struct {
|
||||
changes []mvccpb.KeyValue
|
||||
}
|
||||
|
||||
func (s *store) Write() TxnWrite {
|
||||
func (s *store) Write(trace *traceutil.Trace) TxnWrite {
|
||||
s.mu.RLock()
|
||||
tx := s.b.BatchTx()
|
||||
tx.Lock()
|
||||
tw := &storeTxnWrite{
|
||||
storeTxnRead: storeTxnRead{s, tx, 0, 0},
|
||||
storeTxnRead: storeTxnRead{s, tx, 0, 0, trace},
|
||||
tx: tx,
|
||||
beginRev: s.currentRev,
|
||||
changes: make([]mvccpb.KeyValue, 0, 4),
|
||||
@@ -124,6 +127,7 @@ func (tr *storeTxnRead) rangeKeys(key, end []byte, curRev int64, ro RangeOptions
|
||||
}
|
||||
|
||||
revpairs := tr.s.kvindex.Revisions(key, end, rev)
|
||||
tr.trace.Step("range keys from in-memory index tree")
|
||||
if len(revpairs) == 0 {
|
||||
return &RangeResult{KVs: nil, Count: 0, Rev: curRev}, nil
|
||||
}
|
||||
@@ -163,6 +167,7 @@ func (tr *storeTxnRead) rangeKeys(key, end []byte, curRev int64, ro RangeOptions
|
||||
}
|
||||
}
|
||||
}
|
||||
tr.trace.Step("range keys from bolt db")
|
||||
return &RangeResult{KVs: kvs, Count: len(revpairs), Rev: curRev}, nil
|
||||
}
|
||||
|
||||
@@ -178,7 +183,7 @@ func (tw *storeTxnWrite) put(key, value []byte, leaseID lease.LeaseID) {
|
||||
c = created.main
|
||||
oldLease = tw.s.le.GetLease(lease.LeaseItem{Key: string(key)})
|
||||
}
|
||||
|
||||
tw.trace.Step("get key's previous created_revision and leaseID")
|
||||
ibytes := newRevBytes()
|
||||
idxRev := revision{main: rev, sub: int64(len(tw.changes))}
|
||||
revToBytes(idxRev, ibytes)
|
||||
@@ -205,9 +210,11 @@ func (tw *storeTxnWrite) put(key, value []byte, leaseID lease.LeaseID) {
|
||||
}
|
||||
}
|
||||
|
||||
tw.trace.Step("marshal mvccpb.KeyValue")
|
||||
tw.tx.UnsafeSeqPut(keyBucketName, ibytes, d)
|
||||
tw.s.kvindex.Put(key, idxRev)
|
||||
tw.changes = append(tw.changes, kv)
|
||||
tw.trace.Step("store kv pair into bolt db")
|
||||
|
||||
if oldLease != lease.NoLease {
|
||||
if tw.s.le == nil {
|
||||
@@ -234,6 +241,7 @@ func (tw *storeTxnWrite) put(key, value []byte, leaseID lease.LeaseID) {
|
||||
panic("unexpected error from lease Attach")
|
||||
}
|
||||
}
|
||||
tw.trace.Step("attach lease to kv pair")
|
||||
}
|
||||
|
||||
func (tw *storeTxnWrite) deleteRange(key, end []byte) int64 {
|
||||
|
@@ -264,6 +264,46 @@ var (
|
||||
// highest bucket start of 0.01 sec * 2^14 == 163.84 sec
|
||||
Buckets: prometheus.ExponentialBuckets(.01, 2, 15),
|
||||
})
|
||||
|
||||
currentRev = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
|
||||
Namespace: "etcd_debugging",
|
||||
Subsystem: "mvcc",
|
||||
Name: "current_revision",
|
||||
Help: "The current revision of store.",
|
||||
},
|
||||
func() float64 {
|
||||
reportCurrentRevMu.RLock()
|
||||
defer reportCurrentRevMu.RUnlock()
|
||||
return reportCurrentRev()
|
||||
},
|
||||
)
|
||||
// overridden by mvcc initialization
|
||||
reportCurrentRevMu sync.RWMutex
|
||||
reportCurrentRev = func() float64 { return 0 }
|
||||
|
||||
compactRev = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
|
||||
Namespace: "etcd_debugging",
|
||||
Subsystem: "mvcc",
|
||||
Name: "compact_revision",
|
||||
Help: "The revision of the last compaction in store.",
|
||||
},
|
||||
func() float64 {
|
||||
reportCompactRevMu.RLock()
|
||||
defer reportCompactRevMu.RUnlock()
|
||||
return reportCompactRev()
|
||||
},
|
||||
)
|
||||
// overridden by mvcc initialization
|
||||
reportCompactRevMu sync.RWMutex
|
||||
reportCompactRev = func() float64 { return 0 }
|
||||
|
||||
totalPutSizeGauge = prometheus.NewGauge(
|
||||
prometheus.GaugeOpts{
|
||||
Namespace: "etcd_debugging",
|
||||
Subsystem: "mvcc",
|
||||
Name: "total_put_size_in_bytes",
|
||||
Help: "The total size of put kv pairs seen by this member.",
|
||||
})
|
||||
)
|
||||
|
||||
func init() {
|
||||
@@ -291,6 +331,9 @@ func init() {
|
||||
prometheus.MustRegister(dbOpenReadTxN)
|
||||
prometheus.MustRegister(hashSec)
|
||||
prometheus.MustRegister(hashRevSec)
|
||||
prometheus.MustRegister(currentRev)
|
||||
prometheus.MustRegister(compactRev)
|
||||
prometheus.MustRegister(totalPutSizeGauge)
|
||||
}
|
||||
|
||||
// ReportEventReceived reports that an event is received.
|
||||
|
@@ -21,14 +21,15 @@ type metricsTxnWrite struct {
|
||||
ranges uint
|
||||
puts uint
|
||||
deletes uint
|
||||
putSize int64
|
||||
}
|
||||
|
||||
func newMetricsTxnRead(tr TxnRead) TxnRead {
|
||||
return &metricsTxnWrite{&txnReadWrite{tr}, 0, 0, 0}
|
||||
return &metricsTxnWrite{&txnReadWrite{tr}, 0, 0, 0, 0}
|
||||
}
|
||||
|
||||
func newMetricsTxnWrite(tw TxnWrite) TxnWrite {
|
||||
return &metricsTxnWrite{tw, 0, 0, 0}
|
||||
return &metricsTxnWrite{tw, 0, 0, 0, 0}
|
||||
}
|
||||
|
||||
func (tw *metricsTxnWrite) Range(key, end []byte, ro RangeOptions) (*RangeResult, error) {
|
||||
@@ -43,6 +44,8 @@ func (tw *metricsTxnWrite) DeleteRange(key, end []byte) (n, rev int64) {
|
||||
|
||||
func (tw *metricsTxnWrite) Put(key, value []byte, lease lease.LeaseID) (rev int64) {
|
||||
tw.puts++
|
||||
size := int64(len(key) + len(value))
|
||||
tw.putSize += size
|
||||
return tw.TxnWrite.Put(key, value, lease)
|
||||
}
|
||||
|
||||
@@ -60,6 +63,7 @@ func (tw *metricsTxnWrite) End() {
|
||||
puts := float64(tw.puts)
|
||||
putCounter.Add(puts)
|
||||
putCounterDebug.Add(puts) // TODO: remove in 3.5 release
|
||||
totalPutSizeGauge.Add(float64(tw.putSize))
|
||||
|
||||
deletes := float64(tw.deletes)
|
||||
deleteCounter.Add(deletes)
|
||||
|
@@ -21,6 +21,7 @@ import (
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/mvcc/mvccpb"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
@@ -84,7 +85,7 @@ func newWatchableStore(lg *zap.Logger, b backend.Backend, le lease.Lessor, ig Co
|
||||
s.store.WriteView = &writeView{s}
|
||||
if s.le != nil {
|
||||
// use this store as the deleter so revokes trigger watch events
|
||||
s.le.SetRangeDeleter(func() lease.TxnDelete { return s.Write() })
|
||||
s.le.SetRangeDeleter(func() lease.TxnDelete { return s.Write(traceutil.TODO()) })
|
||||
}
|
||||
s.wg.Add(2)
|
||||
go s.syncWatchersLoop()
|
||||
|
@@ -21,6 +21,7 @@ import (
|
||||
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
@@ -59,7 +60,7 @@ func BenchmarkWatchableStoreTxnPut(b *testing.B) {
|
||||
b.ResetTimer()
|
||||
b.ReportAllocs()
|
||||
for i := 0; i < b.N; i++ {
|
||||
txn := s.Write()
|
||||
txn := s.Write(traceutil.TODO())
|
||||
txn.Put(keys[i], vals[i], lease.NoLease)
|
||||
txn.End()
|
||||
}
|
||||
|
@@ -26,6 +26,7 @@ import (
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/mvcc/backend"
|
||||
"go.etcd.io/etcd/mvcc/mvccpb"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
@@ -237,7 +238,7 @@ func TestWatchCompacted(t *testing.T) {
|
||||
for i := 0; i < maxRev; i++ {
|
||||
s.Put(testKey, testValue, lease.NoLease)
|
||||
}
|
||||
_, err := s.Compact(compactRev)
|
||||
_, err := s.Compact(traceutil.TODO(), compactRev)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to compact kv (%v)", err)
|
||||
}
|
||||
|
@@ -14,7 +14,10 @@
|
||||
|
||||
package mvcc
|
||||
|
||||
import "go.etcd.io/etcd/mvcc/mvccpb"
|
||||
import (
|
||||
"go.etcd.io/etcd/mvcc/mvccpb"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
)
|
||||
|
||||
func (tw *watchableStoreTxnWrite) End() {
|
||||
changes := tw.Changes()
|
||||
@@ -48,4 +51,6 @@ type watchableStoreTxnWrite struct {
|
||||
s *watchableStore
|
||||
}
|
||||
|
||||
func (s *watchableStore) Write() TxnWrite { return &watchableStoreTxnWrite{s.store.Write(), s} }
|
||||
func (s *watchableStore) Write(trace *traceutil.Trace) TxnWrite {
|
||||
return &watchableStoreTxnWrite{s.store.Write(trace), s}
|
||||
}
|
||||
|
@@ -25,13 +25,23 @@ import (
|
||||
)
|
||||
|
||||
func PurgeFile(lg *zap.Logger, dirname string, suffix string, max uint, interval time.Duration, stop <-chan struct{}) <-chan error {
|
||||
return purgeFile(lg, dirname, suffix, max, interval, stop, nil)
|
||||
return purgeFile(lg, dirname, suffix, max, interval, stop, nil, nil)
|
||||
}
|
||||
|
||||
func PurgeFileWithDoneNotify(lg *zap.Logger, dirname string, suffix string, max uint, interval time.Duration, stop <-chan struct{}) (<-chan struct{}, <-chan error) {
|
||||
doneC := make(chan struct{})
|
||||
errC := purgeFile(lg, dirname, suffix, max, interval, stop, nil, doneC)
|
||||
return doneC, errC
|
||||
}
|
||||
|
||||
// purgeFile is the internal implementation for PurgeFile which can post purged files to purgec if non-nil.
|
||||
func purgeFile(lg *zap.Logger, dirname string, suffix string, max uint, interval time.Duration, stop <-chan struct{}, purgec chan<- string) <-chan error {
|
||||
// if donec is non-nil, the function closes it to notify its exit.
|
||||
func purgeFile(lg *zap.Logger, dirname string, suffix string, max uint, interval time.Duration, stop <-chan struct{}, purgec chan<- string, donec chan<- struct{}) <-chan error {
|
||||
errC := make(chan error, 1)
|
||||
go func() {
|
||||
if donec != nil {
|
||||
defer close(donec)
|
||||
}
|
||||
for {
|
||||
fnames, err := ReadDir(dirname)
|
||||
if err != nil {
|
||||
|
@@ -45,7 +45,7 @@ func TestPurgeFile(t *testing.T) {
|
||||
stop, purgec := make(chan struct{}), make(chan string, 10)
|
||||
|
||||
// keep 3 most recent files
|
||||
errch := purgeFile(zap.NewExample(), dir, "test", 3, time.Millisecond, stop, purgec)
|
||||
errch := purgeFile(zap.NewExample(), dir, "test", 3, time.Millisecond, stop, purgec, nil)
|
||||
select {
|
||||
case f := <-purgec:
|
||||
t.Errorf("unexpected purge on %q", f)
|
||||
@@ -116,7 +116,7 @@ func TestPurgeFileHoldingLockFile(t *testing.T) {
|
||||
}
|
||||
|
||||
stop, purgec := make(chan struct{}), make(chan string, 10)
|
||||
errch := purgeFile(zap.NewExample(), dir, "test", 3, time.Millisecond, stop, purgec)
|
||||
errch := purgeFile(zap.NewExample(), dir, "test", 3, time.Millisecond, stop, purgec, nil)
|
||||
|
||||
for i := 0; i < 5; i++ {
|
||||
select {
|
||||
|
172
pkg/traceutil/trace.go
Normal file
172
pkg/traceutil/trace.go
Normal file
@@ -0,0 +1,172 @@
|
||||
// Copyright 2019 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 traceutil implements tracing utilities using "context".
|
||||
package traceutil
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
const (
|
||||
TraceKey = "trace"
|
||||
StartTimeKey = "startTime"
|
||||
)
|
||||
|
||||
// Field is a kv pair to record additional details of the trace.
|
||||
type Field struct {
|
||||
Key string
|
||||
Value interface{}
|
||||
}
|
||||
|
||||
func (f *Field) format() string {
|
||||
return fmt.Sprintf("%s:%v; ", f.Key, f.Value)
|
||||
}
|
||||
|
||||
func writeFields(fields []Field) string {
|
||||
if len(fields) == 0 {
|
||||
return ""
|
||||
}
|
||||
var buf bytes.Buffer
|
||||
buf.WriteString("{")
|
||||
for _, f := range fields {
|
||||
buf.WriteString(f.format())
|
||||
}
|
||||
buf.WriteString("}")
|
||||
return buf.String()
|
||||
}
|
||||
|
||||
type Trace struct {
|
||||
operation string
|
||||
lg *zap.Logger
|
||||
fields []Field
|
||||
startTime time.Time
|
||||
steps []step
|
||||
stepDisabled bool
|
||||
}
|
||||
|
||||
type step struct {
|
||||
time time.Time
|
||||
msg string
|
||||
fields []Field
|
||||
}
|
||||
|
||||
func New(op string, lg *zap.Logger, fields ...Field) *Trace {
|
||||
return &Trace{operation: op, lg: lg, startTime: time.Now(), fields: fields}
|
||||
}
|
||||
|
||||
// TODO returns a non-nil, empty Trace
|
||||
func TODO() *Trace {
|
||||
return &Trace{}
|
||||
}
|
||||
|
||||
func Get(ctx context.Context) *Trace {
|
||||
if trace, ok := ctx.Value(TraceKey).(*Trace); ok && trace != nil {
|
||||
return trace
|
||||
}
|
||||
return TODO()
|
||||
}
|
||||
|
||||
func (t *Trace) GetStartTime() time.Time {
|
||||
return t.startTime
|
||||
}
|
||||
|
||||
func (t *Trace) SetStartTime(time time.Time) {
|
||||
t.startTime = time
|
||||
}
|
||||
|
||||
func (t *Trace) InsertStep(at int, time time.Time, msg string, fields ...Field) {
|
||||
newStep := step{time, msg, fields}
|
||||
if at < len(t.steps) {
|
||||
t.steps = append(t.steps[:at+1], t.steps[at:]...)
|
||||
t.steps[at] = newStep
|
||||
} else {
|
||||
t.steps = append(t.steps, newStep)
|
||||
}
|
||||
}
|
||||
|
||||
// Step adds step to trace
|
||||
func (t *Trace) Step(msg string, fields ...Field) {
|
||||
if !t.stepDisabled {
|
||||
t.steps = append(t.steps, step{time: time.Now(), msg: msg, fields: fields})
|
||||
}
|
||||
}
|
||||
|
||||
// DisableStep sets the flag to prevent the trace from adding steps
|
||||
func (t *Trace) DisableStep() {
|
||||
t.stepDisabled = true
|
||||
}
|
||||
|
||||
// EnableStep re-enable the trace to add steps
|
||||
func (t *Trace) EnableStep() {
|
||||
t.stepDisabled = false
|
||||
}
|
||||
|
||||
func (t *Trace) AddField(fields ...Field) {
|
||||
for _, f := range fields {
|
||||
t.fields = append(t.fields, f)
|
||||
}
|
||||
}
|
||||
|
||||
// Log dumps all steps in the Trace
|
||||
func (t *Trace) Log() {
|
||||
t.LogWithStepThreshold(0)
|
||||
}
|
||||
|
||||
// LogIfLong dumps logs if the duration is longer than threshold
|
||||
func (t *Trace) LogIfLong(threshold time.Duration) {
|
||||
if time.Since(t.startTime) > threshold {
|
||||
stepThreshold := threshold / time.Duration(len(t.steps)+1)
|
||||
t.LogWithStepThreshold(stepThreshold)
|
||||
}
|
||||
}
|
||||
|
||||
// LogWithStepThreshold only dumps step whose duration is longer than step threshold
|
||||
func (t *Trace) LogWithStepThreshold(threshold time.Duration) {
|
||||
msg, fs := t.logInfo(threshold)
|
||||
if t.lg != nil {
|
||||
t.lg.Info(msg, fs...)
|
||||
}
|
||||
}
|
||||
|
||||
func (t *Trace) logInfo(threshold time.Duration) (string, []zap.Field) {
|
||||
endTime := time.Now()
|
||||
totalDuration := endTime.Sub(t.startTime)
|
||||
traceNum := rand.Int31()
|
||||
msg := fmt.Sprintf("trace[%d] %s", traceNum, t.operation)
|
||||
|
||||
var steps []string
|
||||
lastStepTime := t.startTime
|
||||
for _, step := range t.steps {
|
||||
stepDuration := step.time.Sub(lastStepTime)
|
||||
if stepDuration > threshold {
|
||||
steps = append(steps, fmt.Sprintf("trace[%d] '%v' %s (duration: %v)",
|
||||
traceNum, step.msg, writeFields(step.fields), stepDuration))
|
||||
}
|
||||
lastStepTime = step.time
|
||||
}
|
||||
|
||||
fs := []zap.Field{zap.String("detail", writeFields(t.fields)),
|
||||
zap.Duration("duration", totalDuration),
|
||||
zap.Time("start", t.startTime),
|
||||
zap.Time("end", endTime),
|
||||
zap.Strings("steps", steps)}
|
||||
return msg, fs
|
||||
}
|
262
pkg/traceutil/trace_test.go
Normal file
262
pkg/traceutil/trace_test.go
Normal file
@@ -0,0 +1,262 @@
|
||||
// Copyright 2019 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 traceutil
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
func TestGet(t *testing.T) {
|
||||
traceForTest := &Trace{operation: "test"}
|
||||
tests := []struct {
|
||||
name string
|
||||
inputCtx context.Context
|
||||
outputTrace *Trace
|
||||
}{
|
||||
{
|
||||
name: "When the context does not have trace",
|
||||
inputCtx: context.TODO(),
|
||||
outputTrace: TODO(),
|
||||
},
|
||||
{
|
||||
name: "When the context has trace",
|
||||
inputCtx: context.WithValue(context.Background(), TraceKey, traceForTest),
|
||||
outputTrace: traceForTest,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
trace := Get(tt.inputCtx)
|
||||
if trace == nil {
|
||||
t.Errorf("Expected %v; Got nil", tt.outputTrace)
|
||||
}
|
||||
if trace.operation != tt.outputTrace.operation {
|
||||
t.Errorf("Expected %v; Got %v", tt.outputTrace, trace)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestCreate(t *testing.T) {
|
||||
var (
|
||||
op = "Test"
|
||||
steps = []string{"Step1, Step2"}
|
||||
fields = []Field{
|
||||
{"traceKey1", "traceValue1"},
|
||||
{"traceKey2", "traceValue2"},
|
||||
}
|
||||
stepFields = []Field{
|
||||
{"stepKey1", "stepValue2"},
|
||||
{"stepKey2", "stepValue2"},
|
||||
}
|
||||
)
|
||||
|
||||
trace := New(op, nil, fields[0], fields[1])
|
||||
if trace.operation != op {
|
||||
t.Errorf("Expected %v; Got %v", op, trace.operation)
|
||||
}
|
||||
for i, f := range trace.fields {
|
||||
if f.Key != fields[i].Key {
|
||||
t.Errorf("Expected %v; Got %v", fields[i].Key, f.Key)
|
||||
}
|
||||
if f.Value != fields[i].Value {
|
||||
t.Errorf("Expected %v; Got %v", fields[i].Value, f.Value)
|
||||
}
|
||||
}
|
||||
|
||||
for i, v := range steps {
|
||||
trace.Step(v, stepFields[i])
|
||||
}
|
||||
|
||||
for i, v := range trace.steps {
|
||||
if steps[i] != v.msg {
|
||||
t.Errorf("Expected %v; Got %v", steps[i], v.msg)
|
||||
}
|
||||
if stepFields[i].Key != v.fields[0].Key {
|
||||
t.Errorf("Expected %v; Got %v", stepFields[i].Key, v.fields[0].Key)
|
||||
}
|
||||
if stepFields[i].Value != v.fields[0].Value {
|
||||
t.Errorf("Expected %v; Got %v", stepFields[i].Value, v.fields[0].Value)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestLog(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
trace *Trace
|
||||
fields []Field
|
||||
expectedMsg []string
|
||||
}{
|
||||
{
|
||||
name: "When dump all logs",
|
||||
trace: &Trace{
|
||||
operation: "Test",
|
||||
startTime: time.Now().Add(-100 * time.Millisecond),
|
||||
steps: []step{
|
||||
{time: time.Now().Add(-80 * time.Millisecond), msg: "msg1"},
|
||||
{time: time.Now().Add(-50 * time.Millisecond), msg: "msg2"},
|
||||
},
|
||||
},
|
||||
expectedMsg: []string{
|
||||
"msg1", "msg2",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "When trace has fields",
|
||||
trace: &Trace{
|
||||
operation: "Test",
|
||||
startTime: time.Now().Add(-100 * time.Millisecond),
|
||||
steps: []step{
|
||||
{
|
||||
time: time.Now().Add(-80 * time.Millisecond),
|
||||
msg: "msg1",
|
||||
fields: []Field{{"stepKey1", "stepValue1"}},
|
||||
},
|
||||
{
|
||||
time: time.Now().Add(-50 * time.Millisecond),
|
||||
msg: "msg2",
|
||||
fields: []Field{{"stepKey2", "stepValue2"}},
|
||||
},
|
||||
},
|
||||
},
|
||||
fields: []Field{
|
||||
{"traceKey1", "traceValue1"},
|
||||
{"count", 1},
|
||||
},
|
||||
expectedMsg: []string{
|
||||
"Test",
|
||||
"msg1", "msg2",
|
||||
"traceKey1:traceValue1", "count:1",
|
||||
"stepKey1:stepValue1", "stepKey2:stepValue2",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
logPath := filepath.Join(os.TempDir(), fmt.Sprintf("test-log-%d", time.Now().UnixNano()))
|
||||
defer os.RemoveAll(logPath)
|
||||
|
||||
lcfg := zap.NewProductionConfig()
|
||||
lcfg.OutputPaths = []string{logPath}
|
||||
lcfg.ErrorOutputPaths = []string{logPath}
|
||||
lg, _ := lcfg.Build()
|
||||
|
||||
for _, f := range tt.fields {
|
||||
tt.trace.AddField(f)
|
||||
}
|
||||
tt.trace.lg = lg
|
||||
tt.trace.Log()
|
||||
data, err := ioutil.ReadFile(logPath)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
for _, msg := range tt.expectedMsg {
|
||||
if !bytes.Contains(data, []byte(msg)) {
|
||||
t.Errorf("Expected to find %v in log", msg)
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestLogIfLong(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
threshold time.Duration
|
||||
trace *Trace
|
||||
expectedMsg []string
|
||||
}{
|
||||
{
|
||||
name: "When the duration is smaller than threshold",
|
||||
threshold: time.Duration(200 * time.Millisecond),
|
||||
trace: &Trace{
|
||||
operation: "Test",
|
||||
startTime: time.Now().Add(-100 * time.Millisecond),
|
||||
steps: []step{
|
||||
{time: time.Now().Add(-50 * time.Millisecond), msg: "msg1"},
|
||||
{time: time.Now(), msg: "msg2"},
|
||||
},
|
||||
},
|
||||
expectedMsg: []string{},
|
||||
},
|
||||
{
|
||||
name: "When the duration is longer than threshold",
|
||||
threshold: time.Duration(50 * time.Millisecond),
|
||||
trace: &Trace{
|
||||
operation: "Test",
|
||||
startTime: time.Now().Add(-100 * time.Millisecond),
|
||||
steps: []step{
|
||||
{time: time.Now().Add(-50 * time.Millisecond), msg: "msg1"},
|
||||
{time: time.Now(), msg: "msg2"},
|
||||
},
|
||||
},
|
||||
expectedMsg: []string{
|
||||
"msg1", "msg2",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "When not all steps are longer than step threshold",
|
||||
threshold: time.Duration(50 * time.Millisecond),
|
||||
trace: &Trace{
|
||||
operation: "Test",
|
||||
startTime: time.Now().Add(-100 * time.Millisecond),
|
||||
steps: []step{
|
||||
{time: time.Now(), msg: "msg1"},
|
||||
{time: time.Now(), msg: "msg2"},
|
||||
},
|
||||
},
|
||||
expectedMsg: []string{
|
||||
"msg1",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
logPath := filepath.Join(os.TempDir(), fmt.Sprintf("test-log-%d", time.Now().UnixNano()))
|
||||
defer os.RemoveAll(logPath)
|
||||
|
||||
lcfg := zap.NewProductionConfig()
|
||||
lcfg.OutputPaths = []string{logPath}
|
||||
lcfg.ErrorOutputPaths = []string{logPath}
|
||||
lg, _ := lcfg.Build()
|
||||
|
||||
tt.trace.lg = lg
|
||||
tt.trace.LogIfLong(tt.threshold)
|
||||
data, err := ioutil.ReadFile(logPath)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
for _, msg := range tt.expectedMsg {
|
||||
if !bytes.Contains(data, []byte(msg)) {
|
||||
t.Errorf("Expected to find %v in log", msg)
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
@@ -89,6 +89,7 @@ func HandleMetrics(mux *http.ServeMux, c *http.Client, eps []string) {
|
||||
resp, err := c.Get(target)
|
||||
if err != nil {
|
||||
http.Error(w, "Internal server error", http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
defer resp.Body.Close()
|
||||
w.Header().Set("Content-Type", "text/plain; version=0.0.4")
|
||||
|
@@ -57,6 +57,12 @@ function main {
|
||||
cd release
|
||||
setup_env "${PROJ}" "${VER}"
|
||||
|
||||
tarcmd=tar
|
||||
if [[ $(go env GOOS) == "darwin" ]]; then
|
||||
echo "Please use linux machine for release builds."
|
||||
exit 1
|
||||
fi
|
||||
|
||||
for os in darwin windows linux; do
|
||||
export GOOS=${os}
|
||||
TARGET_ARCHS=("amd64")
|
||||
@@ -78,7 +84,7 @@ function main {
|
||||
package "${TARGET}" "${PROJ}"
|
||||
|
||||
if [ ${GOOS} == "linux" ]; then
|
||||
tar cfz "${TARGET}.tar.gz" "${TARGET}"
|
||||
${tarcmd} cfz "${TARGET}.tar.gz" "${TARGET}"
|
||||
echo "Wrote release/${TARGET}.tar.gz"
|
||||
else
|
||||
zip -qr "${TARGET}.zip" "${TARGET}"
|
||||
|
128
scripts/release
128
scripts/release
@@ -5,7 +5,7 @@ set -o nounset
|
||||
set -o pipefail
|
||||
|
||||
help() {
|
||||
echo "$(basename $0) [version]"
|
||||
echo "$(basename "$0") [version]"
|
||||
echo "Release etcd using the same approach as the etcd-release-runbook (https://goo.gl/Gxwysq)"
|
||||
echo ""
|
||||
echo "WARNING: This does not perform the 'Add API capabilities', 'Performance testing' "
|
||||
@@ -37,12 +37,6 @@ main() {
|
||||
exit 1
|
||||
fi
|
||||
|
||||
KEYID=$(gpg --list-keys --with-colons| awk -F: '/^pub:/ { print $5 }')
|
||||
if [[ -z "${KEYID}" ]]; then
|
||||
echo "Failed to load gpg key. Is gpg set up correctly for etcd releases?"
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# Expected umask for etcd release artifacts
|
||||
umask 022
|
||||
|
||||
@@ -51,30 +45,33 @@ main() {
|
||||
if [ ! -d "${reldir}/etcd" ]; then
|
||||
mkdir -p "${reldir}"
|
||||
cd "${reldir}"
|
||||
git clone git@github.com:etcd-io/etcd.git --branch "${BRANCH}"
|
||||
git clone https://github.com/etcd-io/etcd.git --branch "${BRANCH}"
|
||||
fi
|
||||
cd "${reldir}/etcd"
|
||||
|
||||
# If a release version tag already exists, use it.
|
||||
local remote_tag_exists=$(git ls-remote origin "refs/tags/${RELEASE_VERSION}" | grep -c "${RELEASE_VERSION}")
|
||||
if [ ${remote_tag_exists} -gt 0 ]; then
|
||||
local remote_tag_exists
|
||||
remote_tag_exists=$(git ls-remote origin "refs/tags/${RELEASE_VERSION}" | grep -c "${RELEASE_VERSION}")
|
||||
if [ "${remote_tag_exists}" -gt 0 ]; then
|
||||
echo "Release version tag exists on remote. Checking out refs/tags/${RELEASE_VERSION}"
|
||||
git checkout -q "tags/${RELEASE_VERSION}"
|
||||
fi
|
||||
|
||||
# Check go version.
|
||||
# download "yq" from https://github.com/mikefarah/yq
|
||||
local go_version="go$(yq read .travis.yml "go[0]")"
|
||||
local current_go_version=$(go version | awk '{ print $3 }')
|
||||
local go_version current_go_version
|
||||
go_version="go$(yq read .travis.yml "go[0]")"
|
||||
current_go_version=$(go version | awk '{ print $3 }')
|
||||
if [[ "${current_go_version}" != "${go_version}" ]]; then
|
||||
echo "Current go version is ${current_go_version}, but etcd ${RELEASE_VERSION} requires ${go_version} (see .travis.yml)."
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# If the release tag does not already exist remotely, create it.
|
||||
if [ ${remote_tag_exists} -eq 0 ]; then
|
||||
if [ "${remote_tag_exists}" -eq 0 ]; then
|
||||
# Bump version/version.go to release version.
|
||||
local source_version=$(egrep "\s+Version\s*=" version/version.go | sed -e "s/.*\"\(.*\)\".*/\1/g")
|
||||
local source_version
|
||||
source_version=$(grep -E "\s+Version\s*=" version/version.go | sed -e "s/.*\"\(.*\)\".*/\1/g")
|
||||
if [[ "${source_version}" != "${VERSION}" ]]; then
|
||||
source_minor_version=$(echo "${source_version}" | cut -d. -f 1-2)
|
||||
if [[ "${source_minor_version}" != "${MINOR_VERSION}" ]]; then
|
||||
@@ -87,13 +84,14 @@ main() {
|
||||
|
||||
echo "Building etcd and checking --version output"
|
||||
./build
|
||||
local etcd_version=$(bin/etcd --version | grep "etcd Version" | awk '{ print $3 }')
|
||||
local etcd_version
|
||||
etcd_version=$(bin/etcd --version | grep "etcd Version" | awk '{ print $3 }')
|
||||
if [[ "${etcd_version}" != "${VERSION}" ]]; then
|
||||
echo "Wrong etcd version in version/version.go. Expected ${etcd_version} but got ${VERSION}. Aborting."
|
||||
exit 1
|
||||
fi
|
||||
|
||||
if [[ ! -z $(git status -s) ]]; then
|
||||
if [[ -n $(git status -s) ]]; then
|
||||
echo "Committing version/version.go update."
|
||||
git add version/version.go
|
||||
git commit -m "version: bump up to ${VERSION}"
|
||||
@@ -101,22 +99,41 @@ main() {
|
||||
fi
|
||||
|
||||
# Push the version change if it's not already been pushed.
|
||||
if [ $(git rev-list --count "origin/${BRANCH}..${BRANCH}") -gt 0 ]; then
|
||||
read -p "Push version bump up to ${VERSION} to github.com/etcd-io/etcd [y/N]? " confirm
|
||||
if [ "$(git rev-list --count "origin/${BRANCH}..${BRANCH}")" -gt 0 ]; then
|
||||
read -p "Push version bump up to ${VERSION} to github.com/etcd-io/etcd [y/N]? " -r confirm
|
||||
[[ "${confirm,,}" == "y" ]] || exit 1
|
||||
git push
|
||||
fi
|
||||
|
||||
# Tag release.
|
||||
if [ $(git tag --list | grep -c "${RELEASE_VERSION}") -gt 0 ]; then
|
||||
if [ "$(git tag --list | grep -c "${RELEASE_VERSION}")" -gt 0 ]; then
|
||||
echo "Skipping tag step. git tag ${RELEASE_VERSION} already exists."
|
||||
else
|
||||
echo "Tagging release..."
|
||||
KEYID=$(gpg --list-keys --with-colons| awk -F: '/^pub:/ { print $5 }')
|
||||
if [[ -z "${KEYID}" ]]; then
|
||||
echo "Failed to load gpg key. Is gpg set up correctly for etcd releases?"
|
||||
exit 1
|
||||
fi
|
||||
git tag --local-user "${KEYID}" --sign "${RELEASE_VERSION}" --message "${RELEASE_VERSION}"
|
||||
fi
|
||||
|
||||
# Verify the latest commit has the version tag
|
||||
local tag="$(git describe --exact-match HEAD)"
|
||||
if [ "${tag}" != "${RELEASE_VERSION}" ]; then
|
||||
echo "Error: Expected HEAD to be tagged with ${RELEASE_VERSION}, but 'git describe --exact-match HEAD' reported: ${tag}"
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# Verify the version tag is on the right branch
|
||||
local branch=$(git branch --contains "${RELEASE_VERSION}")
|
||||
if [ "${branch}" != "release-${MINOR_VERSION}" ]; then
|
||||
echo "Error: Git tag ${RELEASE_VERSION} should be on branch release-${MINOR_VERSION} but is on ${branch}"
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# Push the tag change if it's not already been pushed.
|
||||
read -p "Push etcd ${RELEASE_VERSION} tag [y/N]? " confirm
|
||||
read -p "Push etcd ${RELEASE_VERSION} tag [y/N]? " -r confirm
|
||||
[[ "${confirm,,}" == "y" ]] || exit 1
|
||||
git push origin "tags/${RELEASE_VERSION}"
|
||||
fi
|
||||
@@ -137,16 +154,18 @@ main() {
|
||||
fi
|
||||
|
||||
# Sanity checks.
|
||||
./release/etcd-${RELEASE_VERSION}-$(go env GOOS)-amd64/etcd --version | grep -q "etcd Version: ${VERSION}" || true
|
||||
./release/etcd-${RELEASE_VERSION}-$(go env GOOS)-amd64/etcdctl version | grep -q "etcdctl version: ${VERSION}" || true
|
||||
"./release/etcd-${RELEASE_VERSION}-$(go env GOOS)-amd64/etcd" --version | grep -q "etcd Version: ${VERSION}" || true
|
||||
"./release/etcd-${RELEASE_VERSION}-$(go env GOOS)-amd64/etcdctl" version | grep -q "etcdctl version: ${VERSION}" || true
|
||||
|
||||
# Generate SHA256SUM
|
||||
echo -e "Generating sha256sum of release artifacts.\n"
|
||||
ls ./release | grep -E '\.tar.gz$|\.zip$' | xargs shasum -a 256; > ./release/SHA256SUM
|
||||
if [ -s ./release/SHA256SUM ]; then
|
||||
cat ./release/SHA256SUM
|
||||
# Generate SHA256SUMS
|
||||
echo -e "Generating sha256sums of release artifacts.\n"
|
||||
pushd ./release
|
||||
ls . | grep -E '\.tar.gz$|\.zip$' | xargs shasum -a 256 > ./SHA256SUMS
|
||||
popd
|
||||
if [ -s ./release/SHA256SUMS ]; then
|
||||
cat ./release/SHA256SUMS
|
||||
else
|
||||
echo "sha256sum is not valid. Aborting."
|
||||
echo "sha256sums is not valid. Aborting."
|
||||
exit 1
|
||||
fi
|
||||
|
||||
@@ -154,44 +173,67 @@ main() {
|
||||
if [ "${NO_UPLOAD}" == 1 ]; then
|
||||
echo "Skipping artifact upload to gs://etcd. --no-upload flat is set."
|
||||
else
|
||||
read -p "Upload etcd ${RELEASE_VERSION} release artifacts to gs://etcd [y/N]? " confirm
|
||||
read -p "Upload etcd ${RELEASE_VERSION} release artifacts to gs://etcd [y/N]? " -r confirm
|
||||
[[ "${confirm,,}" == "y" ]] || exit 1
|
||||
gsutil -m cp ./release/SHA256SUM gs://etcd/${RELEASE_VERSION}/
|
||||
gsutil -m cp ./release/*.zip gs://etcd/${RELEASE_VERSION}/
|
||||
gsutil -m cp ./release/*.tar.gz gs://etcd/${RELEASE_VERSION}/
|
||||
gsutil -m acl ch -u allUsers:R -r gs://etcd/${RELEASE_VERSION}/
|
||||
gsutil -m cp ./release/SHA256SUMS "gs://etcd/${RELEASE_VERSION}/"
|
||||
gsutil -m cp ./release/*.zip "gs://etcd/${RELEASE_VERSION}/"
|
||||
gsutil -m cp ./release/*.tar.gz "gs://etcd/${RELEASE_VERSION}/"
|
||||
gsutil -m acl ch -u allUsers:R -r "gs://etcd/${RELEASE_VERSION}/"
|
||||
fi
|
||||
|
||||
# Push images.
|
||||
if [ "${NO_DOCKER_PUSH}" == 1 ]; then
|
||||
echo "Skipping docker push. --no-docker-push flat is set."
|
||||
else
|
||||
read -p "Publish etcd ${RELEASE_VERSION} docker images to quay.io [y/N]? " confirm
|
||||
read -p "Publish etcd ${RELEASE_VERSION} docker images to quay.io [y/N]? " -r confirm
|
||||
[[ "${confirm,,}" == "y" ]] || exit 1
|
||||
# shellcheck disable=SC2034
|
||||
for i in {1..5}; do
|
||||
docker login quay.io && break
|
||||
echo "login failed, retrying"
|
||||
done
|
||||
gcloud docker -- login -u _json_key -p "$(cat /etc/gcp-key-etcd-development.json)" https://gcr.io
|
||||
|
||||
echo "Pushing container images to quay.io" ${RELEASE_VERSION}
|
||||
docker push quay.io/coreos/etcd:${RELEASE_VERSION}
|
||||
echo "Pushing container images to quay.io ${RELEASE_VERSION}"
|
||||
docker push "quay.io/coreos/etcd:${RELEASE_VERSION}"
|
||||
|
||||
echo "Pushing container images to gcr.io" ${RELEASE_VERSION}
|
||||
gcloud docker -- push gcr.io/etcd-development/etcd:${RELEASE_VERSION}
|
||||
echo "Pushing container images to gcr.io ${RELEASE_VERSION}"
|
||||
gcloud docker -- push "gcr.io/etcd-development/etcd:${RELEASE_VERSION}"
|
||||
|
||||
for TARGET_ARCH in "-arm64" "-ppc64le"; do
|
||||
echo "Pushing container images to quay.io" ${RELEASE_VERSION}${TARGET_ARCH}
|
||||
docker push quay.io/coreos/etcd:${RELEASE_VERSION}${TARGET_ARCH}
|
||||
echo "Pushing container images to quay.io ${RELEASE_VERSION}${TARGET_ARCH}"
|
||||
docker push "quay.io/coreos/etcd:${RELEASE_VERSION}${TARGET_ARCH}"
|
||||
|
||||
echo "Pushing container images to gcr.io" ${RELEASE_VERSION}${TARGET_ARCH}
|
||||
gcloud docker -- push gcr.io/etcd-development/etcd:${RELEASE_VERSION}${TARGET_ARCH}
|
||||
echo "Pushing container images to gcr.io ${RELEASE_VERSION}${TARGET_ARCH}"
|
||||
gcloud docker -- push "gcr.io/etcd-development/etcd:${RELEASE_VERSION}${TARGET_ARCH}"
|
||||
done
|
||||
|
||||
echo "Setting permissions using gsutil..."
|
||||
gsutil -m acl ch -u allUsers:R -r gs://artifacts.etcd-development.appspot.com
|
||||
fi
|
||||
|
||||
### Release validation
|
||||
mkdir -p downloads
|
||||
|
||||
# Check image versions
|
||||
for IMAGE in "quay.io/coreos/etcd:${RELEASE_VERSION}" "gcr.io/etcd-development/etcd:${RELEASE_VERSION}"; do
|
||||
local image_version=$(docker run --rm "${IMAGE}" etcd --version | grep "etcd Version" | awk -F: '{print $2}' | tr -d '[:space:]')
|
||||
if [ "${image_version}" != "${VERSION}" ]; then
|
||||
echo "Check failed: etcd --version output for ${IMAGE} is incorrect: ${image_version}"
|
||||
exit 1
|
||||
fi
|
||||
done
|
||||
|
||||
# Check gsutil binary versions
|
||||
local BINARY_TGZ="etcd-${RELEASE_VERSION}-$(go env GOOS)-amd64.tar.gz"
|
||||
gsutil cp "gs://etcd/${RELEASE_VERSION}/${BINARY_TGZ}" downloads
|
||||
tar -zx -C downloads -f "downloads/${BINARY_TGZ}"
|
||||
local binary_version=$("./downloads/etcd-${RELEASE_VERSION}-$(go env GOOS)-amd64/etcd" --version | grep "etcd Version" | awk -F: '{print $2}' | tr -d '[:space:]')
|
||||
if [ "${binary_version}" != "${VERSION}" ]; then
|
||||
echo "Check failed: etcd --version output for ${BINARY_TGZ} from gs://etcd/${RELEASE_VERSION} is incorrect: ${binary_version}"
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# TODO: signing process
|
||||
echo ""
|
||||
echo "WARNING: The release has not been signed and published to github. This must be done manually."
|
||||
@@ -232,4 +274,4 @@ if [[ ! $# -eq 1 ]]; then
|
||||
exit 1
|
||||
fi
|
||||
|
||||
main $1
|
||||
main "$1"
|
||||
|
@@ -16,7 +16,6 @@ package e2e
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/version"
|
||||
@@ -37,15 +36,9 @@ func TestV3MetricsInsecure(t *testing.T) {
|
||||
}
|
||||
|
||||
func metricsTest(cx ctlCtx) {
|
||||
cx.t.Skip()
|
||||
|
||||
if err := ctlV3Put(cx, "k", "v", ""); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
ver := version.Version
|
||||
if strings.HasSuffix(ver, "-pre") {
|
||||
ver = strings.Replace(ver, "-pre", "", 1)
|
||||
}
|
||||
|
||||
i := 0
|
||||
for _, test := range []struct {
|
||||
@@ -55,7 +48,7 @@ func metricsTest(cx ctlCtx) {
|
||||
{"/metrics", fmt.Sprintf("etcd_debugging_mvcc_keys_total 1")},
|
||||
{"/metrics", fmt.Sprintf("etcd_mvcc_delete_total 3")},
|
||||
{"/metrics", fmt.Sprintf(`etcd_server_version{server_version="%s"} 1`, version.Version)},
|
||||
{"/metrics", fmt.Sprintf(`etcd_cluster_version{cluster_version="%s"} 1`, ver)},
|
||||
{"/metrics", fmt.Sprintf(`etcd_cluster_version{cluster_version="%s"} 1`, version.Cluster(version.Version))},
|
||||
{"/health", `{"health":"true"}`},
|
||||
} {
|
||||
i++
|
||||
|
@@ -186,21 +186,18 @@ func testV3CurlTxn(cx ctlCtx) {
|
||||
|
||||
func testV3CurlAuth(cx ctlCtx) {
|
||||
p := cx.apiPrefix
|
||||
usernames := []string{"root", "nonroot", "nooption"}
|
||||
pwds := []string{"toor", "pass", "pass"}
|
||||
options := []*authpb.UserAddOptions{{NoPassword: false}, {NoPassword: false}, nil}
|
||||
|
||||
// create root user
|
||||
rootuser, err := json.Marshal(&pb.AuthUserAddRequest{Name: string("root"), Password: string("toor"), Options: &authpb.UserAddOptions{NoPassword: false}})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
// create users
|
||||
for i := 0; i < len(usernames); i++ {
|
||||
user, err := json.Marshal(&pb.AuthUserAddRequest{Name: usernames[i], Password: pwds[i], Options: options[i]})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/auth/user/add"), value: string(rootuser), expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth add user with curl (%v)", err)
|
||||
}
|
||||
|
||||
// create non root user
|
||||
nonrootuser, err := json.Marshal(&pb.AuthUserAddRequest{Name: string("example.com"), Password: string("example"), Options: &authpb.UserAddOptions{NoPassword: false}})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/auth/user/add"), value: string(nonrootuser), expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth add user with curl (%v)", err)
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/auth/user/add"), value: string(user), expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth add user %v with curl (%v)", usernames[i], err)
|
||||
}
|
||||
}
|
||||
|
||||
// create root role
|
||||
@@ -211,20 +208,14 @@ func testV3CurlAuth(cx ctlCtx) {
|
||||
cx.t.Fatalf("failed testV3CurlAuth create role with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
|
||||
// grant root role
|
||||
grantroleroot, err := json.Marshal(&pb.AuthUserGrantRoleRequest{User: string("root"), Role: string("root")})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
//grant root role
|
||||
for i := 0; i < len(usernames); i++ {
|
||||
grantroleroot, err := json.Marshal(&pb.AuthUserGrantRoleRequest{User: usernames[i], Role: "root"})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/auth/user/grant"), value: string(grantroleroot), expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth grant role with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
|
||||
// grant non root user root role
|
||||
grantrole, err := json.Marshal(&pb.AuthUserGrantRoleRequest{User: string("example.com"), Role: string("root")})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/auth/user/grant"), value: string(grantrole), expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth grant role with curl using prefix (%s) (%v)", p, err)
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/auth/user/grant"), value: string(grantroleroot), expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth grant role with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
}
|
||||
|
||||
// enable auth
|
||||
@@ -232,45 +223,47 @@ func testV3CurlAuth(cx ctlCtx) {
|
||||
cx.t.Fatalf("failed testV3CurlAuth enable auth with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
|
||||
// put "bar" into "foo"
|
||||
putreq, err := json.Marshal(&pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
for i := 0; i < len(usernames); i++ {
|
||||
// put "bar[i]" into "foo[i]"
|
||||
putreq, err := json.Marshal(&pb.PutRequest{Key: []byte(fmt.Sprintf("foo%d", i)), Value: []byte(fmt.Sprintf("bar%d", i))})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
// fail put no auth
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/put"), value: string(putreq), expected: "error"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth no auth put with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
// fail put no auth
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/put"), value: string(putreq), expected: "error"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth no auth put with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
|
||||
// auth request
|
||||
authreq, err := json.Marshal(&pb.AuthenticateRequest{Name: string("root"), Password: string("toor")})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
// auth request
|
||||
authreq, err := json.Marshal(&pb.AuthenticateRequest{Name: usernames[i], Password: pwds[i]})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
var (
|
||||
authHeader string
|
||||
cmdArgs []string
|
||||
lineFunc = func(txt string) bool { return true }
|
||||
)
|
||||
var (
|
||||
authHeader string
|
||||
cmdArgs []string
|
||||
lineFunc = func(txt string) bool { return true }
|
||||
)
|
||||
|
||||
cmdArgs = cURLPrefixArgs(cx.epc, "POST", cURLReq{endpoint: path.Join(p, "/auth/authenticate"), value: string(authreq)})
|
||||
proc, err := spawnCmd(cmdArgs)
|
||||
testutil.AssertNil(cx.t, err)
|
||||
cmdArgs = cURLPrefixArgs(cx.epc, "POST", cURLReq{endpoint: path.Join(p, "/auth/authenticate"), value: string(authreq)})
|
||||
proc, err := spawnCmd(cmdArgs)
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
cURLRes, err := proc.ExpectFunc(lineFunc)
|
||||
testutil.AssertNil(cx.t, err)
|
||||
cURLRes, err := proc.ExpectFunc(lineFunc)
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
authRes := make(map[string]interface{})
|
||||
testutil.AssertNil(cx.t, json.Unmarshal([]byte(cURLRes), &authRes))
|
||||
authRes := make(map[string]interface{})
|
||||
testutil.AssertNil(cx.t, json.Unmarshal([]byte(cURLRes), &authRes))
|
||||
|
||||
token, ok := authRes[rpctypes.TokenFieldNameGRPC].(string)
|
||||
if !ok {
|
||||
cx.t.Fatalf("failed invalid token in authenticate response with curl")
|
||||
}
|
||||
token, ok := authRes[rpctypes.TokenFieldNameGRPC].(string)
|
||||
if !ok {
|
||||
cx.t.Fatalf("failed invalid token in authenticate response with curl using user (%v)", usernames[i])
|
||||
}
|
||||
|
||||
authHeader = "Authorization: " + token
|
||||
authHeader = "Authorization: " + token
|
||||
|
||||
// put with auth
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/put"), value: string(putreq), header: authHeader, expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth auth put with curl using prefix (%s) (%v)", p, err)
|
||||
// put with auth
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/put"), value: string(putreq), header: authHeader, expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth auth put with curl using prefix (%s) and user (%v) (%v)", p, usernames[i], err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@@ -23,6 +23,7 @@ import (
|
||||
|
||||
"go.etcd.io/etcd/lease"
|
||||
"go.etcd.io/etcd/pkg/report"
|
||||
"go.etcd.io/etcd/pkg/traceutil"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
@@ -114,7 +115,7 @@ func mvccPutFunc(cmd *cobra.Command, args []string) {
|
||||
for i := 0; i < mvccTotalRequests; i++ {
|
||||
st := time.Now()
|
||||
|
||||
tw := s.Write()
|
||||
tw := s.Write(traceutil.TODO())
|
||||
for j := i; j < i+nrTxnOps; j++ {
|
||||
tw.Put(keys[j], vals[j], lease.NoLease)
|
||||
}
|
||||
|
10
vendor/github.com/creack/pty/types.go
generated
vendored
10
vendor/github.com/creack/pty/types.go
generated
vendored
@@ -1,10 +0,0 @@
|
||||
// +build ignore
|
||||
|
||||
package pty
|
||||
|
||||
import "C"
|
||||
|
||||
type (
|
||||
_C_int C.int
|
||||
_C_uint C.uint
|
||||
)
|
17
vendor/github.com/creack/pty/types_dragonfly.go
generated
vendored
17
vendor/github.com/creack/pty/types_dragonfly.go
generated
vendored
@@ -1,17 +0,0 @@
|
||||
// +build ignore
|
||||
|
||||
package pty
|
||||
|
||||
/*
|
||||
#define _KERNEL
|
||||
#include <sys/conf.h>
|
||||
#include <sys/param.h>
|
||||
#include <sys/filio.h>
|
||||
*/
|
||||
import "C"
|
||||
|
||||
const (
|
||||
_C_SPECNAMELEN = C.SPECNAMELEN /* max length of devicename */
|
||||
)
|
||||
|
||||
type fiodgnameArg C.struct_fiodname_args
|
15
vendor/github.com/creack/pty/types_freebsd.go
generated
vendored
15
vendor/github.com/creack/pty/types_freebsd.go
generated
vendored
@@ -1,15 +0,0 @@
|
||||
// +build ignore
|
||||
|
||||
package pty
|
||||
|
||||
/*
|
||||
#include <sys/param.h>
|
||||
#include <sys/filio.h>
|
||||
*/
|
||||
import "C"
|
||||
|
||||
const (
|
||||
_C_SPECNAMELEN = C.SPECNAMELEN /* max length of devicename */
|
||||
)
|
||||
|
||||
type fiodgnameArg C.struct_fiodgname_arg
|
14
vendor/github.com/creack/pty/types_openbsd.go
generated
vendored
14
vendor/github.com/creack/pty/types_openbsd.go
generated
vendored
@@ -1,14 +0,0 @@
|
||||
// +build ignore
|
||||
|
||||
package pty
|
||||
|
||||
/*
|
||||
#include <sys/time.h>
|
||||
#include <stdlib.h>
|
||||
#include <sys/tty.h>
|
||||
*/
|
||||
import "C"
|
||||
|
||||
type ptmget C.struct_ptmget
|
||||
|
||||
var ioctl_PTMGET = C.PTMGET
|
76
vendor/github.com/google/btree/btree_mem.go
generated
vendored
76
vendor/github.com/google/btree/btree_mem.go
generated
vendored
@@ -1,76 +0,0 @@
|
||||
// Copyright 2014 Google Inc.
|
||||
//
|
||||
// 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 ignore
|
||||
|
||||
// This binary compares memory usage between btree and gollrb.
|
||||
package main
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"runtime"
|
||||
"time"
|
||||
|
||||
"github.com/google/btree"
|
||||
"github.com/petar/GoLLRB/llrb"
|
||||
)
|
||||
|
||||
var (
|
||||
size = flag.Int("size", 1000000, "size of the tree to build")
|
||||
degree = flag.Int("degree", 8, "degree of btree")
|
||||
gollrb = flag.Bool("llrb", false, "use llrb instead of btree")
|
||||
)
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
vals := rand.Perm(*size)
|
||||
var t, v interface{}
|
||||
v = vals
|
||||
var stats runtime.MemStats
|
||||
for i := 0; i < 10; i++ {
|
||||
runtime.GC()
|
||||
}
|
||||
fmt.Println("-------- BEFORE ----------")
|
||||
runtime.ReadMemStats(&stats)
|
||||
fmt.Printf("%+v\n", stats)
|
||||
start := time.Now()
|
||||
if *gollrb {
|
||||
tr := llrb.New()
|
||||
for _, v := range vals {
|
||||
tr.ReplaceOrInsert(llrb.Int(v))
|
||||
}
|
||||
t = tr // keep it around
|
||||
} else {
|
||||
tr := btree.New(*degree)
|
||||
for _, v := range vals {
|
||||
tr.ReplaceOrInsert(btree.Int(v))
|
||||
}
|
||||
t = tr // keep it around
|
||||
}
|
||||
fmt.Printf("%v inserts in %v\n", *size, time.Since(start))
|
||||
fmt.Println("-------- AFTER ----------")
|
||||
runtime.ReadMemStats(&stats)
|
||||
fmt.Printf("%+v\n", stats)
|
||||
for i := 0; i < 10; i++ {
|
||||
runtime.GC()
|
||||
}
|
||||
fmt.Println("-------- AFTER GC ----------")
|
||||
runtime.ReadMemStats(&stats)
|
||||
fmt.Printf("%+v\n", stats)
|
||||
if t == v {
|
||||
fmt.Println("to make sure vals and tree aren't GC'd")
|
||||
}
|
||||
}
|
8
vendor/golang.org/x/sys/unix/affinity_linux.go
generated
vendored
8
vendor/golang.org/x/sys/unix/affinity_linux.go
generated
vendored
@@ -91,9 +91,13 @@ func onesCount64(x uint64) int {
|
||||
const m0 = 0x5555555555555555 // 01010101 ...
|
||||
const m1 = 0x3333333333333333 // 00110011 ...
|
||||
const m2 = 0x0f0f0f0f0f0f0f0f // 00001111 ...
|
||||
const m3 = 0x00ff00ff00ff00ff // etc.
|
||||
const m4 = 0x0000ffff0000ffff
|
||||
|
||||
// Unused in this function, but definitions preserved for
|
||||
// documentation purposes:
|
||||
//
|
||||
// const m3 = 0x00ff00ff00ff00ff // etc.
|
||||
// const m4 = 0x0000ffff0000ffff
|
||||
//
|
||||
// Implementation: Parallel summing of adjacent bits.
|
||||
// See "Hacker's Delight", Chap. 5: Counting Bits.
|
||||
// The following pattern shows the general approach:
|
||||
|
54
vendor/golang.org/x/sys/unix/asm_linux_riscv64.s
generated
vendored
Normal file
54
vendor/golang.org/x/sys/unix/asm_linux_riscv64.s
generated
vendored
Normal file
@@ -0,0 +1,54 @@
|
||||
// Copyright 2019 The Go Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
// +build riscv64,!gccgo
|
||||
|
||||
#include "textflag.h"
|
||||
|
||||
//
|
||||
// System calls for linux/riscv64.
|
||||
//
|
||||
// Where available, just jump to package syscall's implementation of
|
||||
// these functions.
|
||||
|
||||
TEXT ·Syscall(SB),NOSPLIT,$0-56
|
||||
JMP syscall·Syscall(SB)
|
||||
|
||||
TEXT ·Syscall6(SB),NOSPLIT,$0-80
|
||||
JMP syscall·Syscall6(SB)
|
||||
|
||||
TEXT ·SyscallNoError(SB),NOSPLIT,$0-48
|
||||
CALL runtime·entersyscall(SB)
|
||||
MOV a1+8(FP), A0
|
||||
MOV a2+16(FP), A1
|
||||
MOV a3+24(FP), A2
|
||||
MOV $0, A3
|
||||
MOV $0, A4
|
||||
MOV $0, A5
|
||||
MOV $0, A6
|
||||
MOV trap+0(FP), A7 // syscall entry
|
||||
ECALL
|
||||
MOV A0, r1+32(FP) // r1
|
||||
MOV A1, r2+40(FP) // r2
|
||||
CALL runtime·exitsyscall(SB)
|
||||
RET
|
||||
|
||||
TEXT ·RawSyscall(SB),NOSPLIT,$0-56
|
||||
JMP syscall·RawSyscall(SB)
|
||||
|
||||
TEXT ·RawSyscall6(SB),NOSPLIT,$0-80
|
||||
JMP syscall·RawSyscall6(SB)
|
||||
|
||||
TEXT ·RawSyscallNoError(SB),NOSPLIT,$0-48
|
||||
MOV a1+8(FP), A0
|
||||
MOV a2+16(FP), A1
|
||||
MOV a3+24(FP), A2
|
||||
MOV ZERO, A3
|
||||
MOV ZERO, A4
|
||||
MOV ZERO, A5
|
||||
MOV trap+0(FP), A7 // syscall entry
|
||||
ECALL
|
||||
MOV A0, r1+32(FP)
|
||||
MOV A1, r2+40(FP)
|
||||
RET
|
29
vendor/golang.org/x/sys/unix/asm_openbsd_arm64.s
generated
vendored
Normal file
29
vendor/golang.org/x/sys/unix/asm_openbsd_arm64.s
generated
vendored
Normal file
@@ -0,0 +1,29 @@
|
||||
// Copyright 2019 The Go Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
// +build !gccgo
|
||||
|
||||
#include "textflag.h"
|
||||
|
||||
//
|
||||
// System call support for arm64, OpenBSD
|
||||
//
|
||||
|
||||
// Just jump to package syscall's implementation for all these functions.
|
||||
// The runtime may know about them.
|
||||
|
||||
TEXT ·Syscall(SB),NOSPLIT,$0-56
|
||||
JMP syscall·Syscall(SB)
|
||||
|
||||
TEXT ·Syscall6(SB),NOSPLIT,$0-80
|
||||
JMP syscall·Syscall6(SB)
|
||||
|
||||
TEXT ·Syscall9(SB),NOSPLIT,$0-104
|
||||
JMP syscall·Syscall9(SB)
|
||||
|
||||
TEXT ·RawSyscall(SB),NOSPLIT,$0-56
|
||||
JMP syscall·RawSyscall(SB)
|
||||
|
||||
TEXT ·RawSyscall6(SB),NOSPLIT,$0-80
|
||||
JMP syscall·RawSyscall6(SB)
|
91
vendor/golang.org/x/sys/unix/dirent.go
generated
vendored
91
vendor/golang.org/x/sys/unix/dirent.go
generated
vendored
@@ -2,16 +2,101 @@
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
// +build aix darwin dragonfly freebsd linux nacl netbsd openbsd solaris
|
||||
// +build aix darwin dragonfly freebsd linux netbsd openbsd solaris
|
||||
|
||||
package unix
|
||||
|
||||
import "syscall"
|
||||
import "unsafe"
|
||||
|
||||
// readInt returns the size-bytes unsigned integer in native byte order at offset off.
|
||||
func readInt(b []byte, off, size uintptr) (u uint64, ok bool) {
|
||||
if len(b) < int(off+size) {
|
||||
return 0, false
|
||||
}
|
||||
if isBigEndian {
|
||||
return readIntBE(b[off:], size), true
|
||||
}
|
||||
return readIntLE(b[off:], size), true
|
||||
}
|
||||
|
||||
func readIntBE(b []byte, size uintptr) uint64 {
|
||||
switch size {
|
||||
case 1:
|
||||
return uint64(b[0])
|
||||
case 2:
|
||||
_ = b[1] // bounds check hint to compiler; see golang.org/issue/14808
|
||||
return uint64(b[1]) | uint64(b[0])<<8
|
||||
case 4:
|
||||
_ = b[3] // bounds check hint to compiler; see golang.org/issue/14808
|
||||
return uint64(b[3]) | uint64(b[2])<<8 | uint64(b[1])<<16 | uint64(b[0])<<24
|
||||
case 8:
|
||||
_ = b[7] // bounds check hint to compiler; see golang.org/issue/14808
|
||||
return uint64(b[7]) | uint64(b[6])<<8 | uint64(b[5])<<16 | uint64(b[4])<<24 |
|
||||
uint64(b[3])<<32 | uint64(b[2])<<40 | uint64(b[1])<<48 | uint64(b[0])<<56
|
||||
default:
|
||||
panic("syscall: readInt with unsupported size")
|
||||
}
|
||||
}
|
||||
|
||||
func readIntLE(b []byte, size uintptr) uint64 {
|
||||
switch size {
|
||||
case 1:
|
||||
return uint64(b[0])
|
||||
case 2:
|
||||
_ = b[1] // bounds check hint to compiler; see golang.org/issue/14808
|
||||
return uint64(b[0]) | uint64(b[1])<<8
|
||||
case 4:
|
||||
_ = b[3] // bounds check hint to compiler; see golang.org/issue/14808
|
||||
return uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24
|
||||
case 8:
|
||||
_ = b[7] // bounds check hint to compiler; see golang.org/issue/14808
|
||||
return uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 |
|
||||
uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56
|
||||
default:
|
||||
panic("syscall: readInt with unsupported size")
|
||||
}
|
||||
}
|
||||
|
||||
// ParseDirent parses up to max directory entries in buf,
|
||||
// appending the names to names. It returns the number of
|
||||
// bytes consumed from buf, the number of entries added
|
||||
// to names, and the new names slice.
|
||||
func ParseDirent(buf []byte, max int, names []string) (consumed int, count int, newnames []string) {
|
||||
return syscall.ParseDirent(buf, max, names)
|
||||
origlen := len(buf)
|
||||
count = 0
|
||||
for max != 0 && len(buf) > 0 {
|
||||
reclen, ok := direntReclen(buf)
|
||||
if !ok || reclen > uint64(len(buf)) {
|
||||
return origlen, count, names
|
||||
}
|
||||
rec := buf[:reclen]
|
||||
buf = buf[reclen:]
|
||||
ino, ok := direntIno(rec)
|
||||
if !ok {
|
||||
break
|
||||
}
|
||||
if ino == 0 { // File absent in directory.
|
||||
continue
|
||||
}
|
||||
const namoff = uint64(unsafe.Offsetof(Dirent{}.Name))
|
||||
namlen, ok := direntNamlen(rec)
|
||||
if !ok || namoff+namlen > uint64(len(rec)) {
|
||||
break
|
||||
}
|
||||
name := rec[namoff : namoff+namlen]
|
||||
for i, c := range name {
|
||||
if c == 0 {
|
||||
name = name[:i]
|
||||
break
|
||||
}
|
||||
}
|
||||
// Check for useless names before allocating a string.
|
||||
if string(name) == "." || string(name) == ".." {
|
||||
continue
|
||||
}
|
||||
max--
|
||||
count++
|
||||
names = append(names, string(name))
|
||||
}
|
||||
return origlen - len(buf), count, names
|
||||
}
|
||||
|
2
vendor/golang.org/x/sys/unix/endian_little.go
generated
vendored
2
vendor/golang.org/x/sys/unix/endian_little.go
generated
vendored
@@ -2,7 +2,7 @@
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
//
|
||||
// +build 386 amd64 amd64p32 arm arm64 ppc64le mipsle mips64le
|
||||
// +build 386 amd64 amd64p32 arm arm64 ppc64le mipsle mips64le riscv64
|
||||
|
||||
package unix
|
||||
|
||||
|
61
vendor/golang.org/x/sys/unix/mkasm_darwin.go
generated
vendored
61
vendor/golang.org/x/sys/unix/mkasm_darwin.go
generated
vendored
@@ -1,61 +0,0 @@
|
||||
// Copyright 2018 The Go Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
// +build ignore
|
||||
|
||||
// mkasm_darwin.go generates assembly trampolines to call libSystem routines from Go.
|
||||
//This program must be run after mksyscall.go.
|
||||
package main
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"log"
|
||||
"os"
|
||||
"strings"
|
||||
)
|
||||
|
||||
func main() {
|
||||
in1, err := ioutil.ReadFile("syscall_darwin.go")
|
||||
if err != nil {
|
||||
log.Fatalf("can't open syscall_darwin.go: %s", err)
|
||||
}
|
||||
arch := os.Args[1]
|
||||
in2, err := ioutil.ReadFile(fmt.Sprintf("syscall_darwin_%s.go", arch))
|
||||
if err != nil {
|
||||
log.Fatalf("can't open syscall_darwin_%s.go: %s", arch, err)
|
||||
}
|
||||
in3, err := ioutil.ReadFile(fmt.Sprintf("zsyscall_darwin_%s.go", arch))
|
||||
if err != nil {
|
||||
log.Fatalf("can't open zsyscall_darwin_%s.go: %s", arch, err)
|
||||
}
|
||||
in := string(in1) + string(in2) + string(in3)
|
||||
|
||||
trampolines := map[string]bool{}
|
||||
|
||||
var out bytes.Buffer
|
||||
|
||||
fmt.Fprintf(&out, "// go run mkasm_darwin.go %s\n", strings.Join(os.Args[1:], " "))
|
||||
fmt.Fprintf(&out, "// Code generated by the command above; DO NOT EDIT.\n")
|
||||
fmt.Fprintf(&out, "\n")
|
||||
fmt.Fprintf(&out, "// +build go1.12\n")
|
||||
fmt.Fprintf(&out, "\n")
|
||||
fmt.Fprintf(&out, "#include \"textflag.h\"\n")
|
||||
for _, line := range strings.Split(in, "\n") {
|
||||
if !strings.HasPrefix(line, "func ") || !strings.HasSuffix(line, "_trampoline()") {
|
||||
continue
|
||||
}
|
||||
fn := line[5 : len(line)-13]
|
||||
if !trampolines[fn] {
|
||||
trampolines[fn] = true
|
||||
fmt.Fprintf(&out, "TEXT ·%s_trampoline(SB),NOSPLIT,$0-0\n", fn)
|
||||
fmt.Fprintf(&out, "\tJMP\t%s(SB)\n", fn)
|
||||
}
|
||||
}
|
||||
err = ioutil.WriteFile(fmt.Sprintf("zsyscall_darwin_%s.s", arch), out.Bytes(), 0644)
|
||||
if err != nil {
|
||||
log.Fatalf("can't write zsyscall_darwin_%s.s: %s", arch, err)
|
||||
}
|
||||
}
|
106
vendor/golang.org/x/sys/unix/mkpost.go
generated
vendored
106
vendor/golang.org/x/sys/unix/mkpost.go
generated
vendored
@@ -1,106 +0,0 @@
|
||||
// Copyright 2016 The Go Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
// +build ignore
|
||||
|
||||
// mkpost processes the output of cgo -godefs to
|
||||
// modify the generated types. It is used to clean up
|
||||
// the sys API in an architecture specific manner.
|
||||
//
|
||||
// mkpost is run after cgo -godefs; see README.md.
|
||||
package main
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"go/format"
|
||||
"io/ioutil"
|
||||
"log"
|
||||
"os"
|
||||
"regexp"
|
||||
)
|
||||
|
||||
func main() {
|
||||
// Get the OS and architecture (using GOARCH_TARGET if it exists)
|
||||
goos := os.Getenv("GOOS")
|
||||
goarch := os.Getenv("GOARCH_TARGET")
|
||||
if goarch == "" {
|
||||
goarch = os.Getenv("GOARCH")
|
||||
}
|
||||
// Check that we are using the Docker-based build system if we should be.
|
||||
if goos == "linux" {
|
||||
if os.Getenv("GOLANG_SYS_BUILD") != "docker" {
|
||||
os.Stderr.WriteString("In the Docker-based build system, mkpost should not be called directly.\n")
|
||||
os.Stderr.WriteString("See README.md\n")
|
||||
os.Exit(1)
|
||||
}
|
||||
}
|
||||
|
||||
b, err := ioutil.ReadAll(os.Stdin)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
// Intentionally export __val fields in Fsid and Sigset_t
|
||||
valRegex := regexp.MustCompile(`type (Fsid|Sigset_t) struct {(\s+)X__val(\s+\S+\s+)}`)
|
||||
b = valRegex.ReplaceAll(b, []byte("type $1 struct {${2}Val$3}"))
|
||||
|
||||
// Intentionally export __fds_bits field in FdSet
|
||||
fdSetRegex := regexp.MustCompile(`type (FdSet) struct {(\s+)X__fds_bits(\s+\S+\s+)}`)
|
||||
b = fdSetRegex.ReplaceAll(b, []byte("type $1 struct {${2}Bits$3}"))
|
||||
|
||||
// If we have empty Ptrace structs, we should delete them. Only s390x emits
|
||||
// nonempty Ptrace structs.
|
||||
ptraceRexexp := regexp.MustCompile(`type Ptrace((Psw|Fpregs|Per) struct {\s*})`)
|
||||
b = ptraceRexexp.ReplaceAll(b, nil)
|
||||
|
||||
// Replace the control_regs union with a blank identifier for now.
|
||||
controlRegsRegex := regexp.MustCompile(`(Control_regs)\s+\[0\]uint64`)
|
||||
b = controlRegsRegex.ReplaceAll(b, []byte("_ [0]uint64"))
|
||||
|
||||
// Remove fields that are added by glibc
|
||||
// Note that this is unstable as the identifers are private.
|
||||
removeFieldsRegex := regexp.MustCompile(`X__glibc\S*`)
|
||||
b = removeFieldsRegex.ReplaceAll(b, []byte("_"))
|
||||
|
||||
// Convert [65]int8 to [65]byte in Utsname members to simplify
|
||||
// conversion to string; see golang.org/issue/20753
|
||||
convertUtsnameRegex := regexp.MustCompile(`((Sys|Node|Domain)name|Release|Version|Machine)(\s+)\[(\d+)\]u?int8`)
|
||||
b = convertUtsnameRegex.ReplaceAll(b, []byte("$1$3[$4]byte"))
|
||||
|
||||
// Convert [1024]int8 to [1024]byte in Ptmget members
|
||||
convertPtmget := regexp.MustCompile(`([SC]n)(\s+)\[(\d+)\]u?int8`)
|
||||
b = convertPtmget.ReplaceAll(b, []byte("$1[$3]byte"))
|
||||
|
||||
// Remove spare fields (e.g. in Statx_t)
|
||||
spareFieldsRegex := regexp.MustCompile(`X__spare\S*`)
|
||||
b = spareFieldsRegex.ReplaceAll(b, []byte("_"))
|
||||
|
||||
// Remove cgo padding fields
|
||||
removePaddingFieldsRegex := regexp.MustCompile(`Pad_cgo_\d+`)
|
||||
b = removePaddingFieldsRegex.ReplaceAll(b, []byte("_"))
|
||||
|
||||
// Remove padding, hidden, or unused fields
|
||||
removeFieldsRegex = regexp.MustCompile(`\b(X_\S+|Padding)`)
|
||||
b = removeFieldsRegex.ReplaceAll(b, []byte("_"))
|
||||
|
||||
// Remove the first line of warning from cgo
|
||||
b = b[bytes.IndexByte(b, '\n')+1:]
|
||||
// Modify the command in the header to include:
|
||||
// mkpost, our own warning, and a build tag.
|
||||
replacement := fmt.Sprintf(`$1 | go run mkpost.go
|
||||
// Code generated by the command above; see README.md. DO NOT EDIT.
|
||||
|
||||
// +build %s,%s`, goarch, goos)
|
||||
cgoCommandRegex := regexp.MustCompile(`(cgo -godefs .*)`)
|
||||
b = cgoCommandRegex.ReplaceAll(b, []byte(replacement))
|
||||
|
||||
// gofmt
|
||||
b, err = format.Source(b)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
os.Stdout.Write(b)
|
||||
}
|
402
vendor/golang.org/x/sys/unix/mksyscall.go
generated
vendored
402
vendor/golang.org/x/sys/unix/mksyscall.go
generated
vendored
@@ -1,402 +0,0 @@
|
||||
// Copyright 2018 The Go Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
// +build ignore
|
||||
|
||||
/*
|
||||
This program reads a file containing function prototypes
|
||||
(like syscall_darwin.go) and generates system call bodies.
|
||||
The prototypes are marked by lines beginning with "//sys"
|
||||
and read like func declarations if //sys is replaced by func, but:
|
||||
* The parameter lists must give a name for each argument.
|
||||
This includes return parameters.
|
||||
* The parameter lists must give a type for each argument:
|
||||
the (x, y, z int) shorthand is not allowed.
|
||||
* If the return parameter is an error number, it must be named errno.
|
||||
|
||||
A line beginning with //sysnb is like //sys, except that the
|
||||
goroutine will not be suspended during the execution of the system
|
||||
call. This must only be used for system calls which can never
|
||||
block, as otherwise the system call could cause all goroutines to
|
||||
hang.
|
||||
*/
|
||||
package main
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"flag"
|
||||
"fmt"
|
||||
"os"
|
||||
"regexp"
|
||||
"strings"
|
||||
)
|
||||
|
||||
var (
|
||||
b32 = flag.Bool("b32", false, "32bit big-endian")
|
||||
l32 = flag.Bool("l32", false, "32bit little-endian")
|
||||
plan9 = flag.Bool("plan9", false, "plan9")
|
||||
openbsd = flag.Bool("openbsd", false, "openbsd")
|
||||
netbsd = flag.Bool("netbsd", false, "netbsd")
|
||||
dragonfly = flag.Bool("dragonfly", false, "dragonfly")
|
||||
arm = flag.Bool("arm", false, "arm") // 64-bit value should use (even, odd)-pair
|
||||
tags = flag.String("tags", "", "build tags")
|
||||
filename = flag.String("output", "", "output file name (standard output if omitted)")
|
||||
)
|
||||
|
||||
// cmdLine returns this programs's commandline arguments
|
||||
func cmdLine() string {
|
||||
return "go run mksyscall.go " + strings.Join(os.Args[1:], " ")
|
||||
}
|
||||
|
||||
// buildTags returns build tags
|
||||
func buildTags() string {
|
||||
return *tags
|
||||
}
|
||||
|
||||
// Param is function parameter
|
||||
type Param struct {
|
||||
Name string
|
||||
Type string
|
||||
}
|
||||
|
||||
// usage prints the program usage
|
||||
func usage() {
|
||||
fmt.Fprintf(os.Stderr, "usage: go run mksyscall.go [-b32 | -l32] [-tags x,y] [file ...]\n")
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
// parseParamList parses parameter list and returns a slice of parameters
|
||||
func parseParamList(list string) []string {
|
||||
list = strings.TrimSpace(list)
|
||||
if list == "" {
|
||||
return []string{}
|
||||
}
|
||||
return regexp.MustCompile(`\s*,\s*`).Split(list, -1)
|
||||
}
|
||||
|
||||
// parseParam splits a parameter into name and type
|
||||
func parseParam(p string) Param {
|
||||
ps := regexp.MustCompile(`^(\S*) (\S*)$`).FindStringSubmatch(p)
|
||||
if ps == nil {
|
||||
fmt.Fprintf(os.Stderr, "malformed parameter: %s\n", p)
|
||||
os.Exit(1)
|
||||
}
|
||||
return Param{ps[1], ps[2]}
|
||||
}
|
||||
|
||||
func main() {
|
||||
// Get the OS and architecture (using GOARCH_TARGET if it exists)
|
||||
goos := os.Getenv("GOOS")
|
||||
if goos == "" {
|
||||
fmt.Fprintln(os.Stderr, "GOOS not defined in environment")
|
||||
os.Exit(1)
|
||||
}
|
||||
goarch := os.Getenv("GOARCH_TARGET")
|
||||
if goarch == "" {
|
||||
goarch = os.Getenv("GOARCH")
|
||||
}
|
||||
|
||||
// Check that we are using the Docker-based build system if we should
|
||||
if goos == "linux" {
|
||||
if os.Getenv("GOLANG_SYS_BUILD") != "docker" {
|
||||
fmt.Fprintf(os.Stderr, "In the Docker-based build system, mksyscall should not be called directly.\n")
|
||||
fmt.Fprintf(os.Stderr, "See README.md\n")
|
||||
os.Exit(1)
|
||||
}
|
||||
}
|
||||
|
||||
flag.Usage = usage
|
||||
flag.Parse()
|
||||
if len(flag.Args()) <= 0 {
|
||||
fmt.Fprintf(os.Stderr, "no files to parse provided\n")
|
||||
usage()
|
||||
}
|
||||
|
||||
endianness := ""
|
||||
if *b32 {
|
||||
endianness = "big-endian"
|
||||
} else if *l32 {
|
||||
endianness = "little-endian"
|
||||
}
|
||||
|
||||
libc := false
|
||||
if goos == "darwin" && strings.Contains(buildTags(), ",go1.12") {
|
||||
libc = true
|
||||
}
|
||||
trampolines := map[string]bool{}
|
||||
|
||||
text := ""
|
||||
for _, path := range flag.Args() {
|
||||
file, err := os.Open(path)
|
||||
if err != nil {
|
||||
fmt.Fprintf(os.Stderr, err.Error())
|
||||
os.Exit(1)
|
||||
}
|
||||
s := bufio.NewScanner(file)
|
||||
for s.Scan() {
|
||||
t := s.Text()
|
||||
t = strings.TrimSpace(t)
|
||||
t = regexp.MustCompile(`\s+`).ReplaceAllString(t, ` `)
|
||||
nonblock := regexp.MustCompile(`^\/\/sysnb `).FindStringSubmatch(t)
|
||||
if regexp.MustCompile(`^\/\/sys `).FindStringSubmatch(t) == nil && nonblock == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// Line must be of the form
|
||||
// func Open(path string, mode int, perm int) (fd int, errno error)
|
||||
// Split into name, in params, out params.
|
||||
f := regexp.MustCompile(`^\/\/sys(nb)? (\w+)\(([^()]*)\)\s*(?:\(([^()]+)\))?\s*(?:=\s*((?i)SYS_[A-Z0-9_]+))?$`).FindStringSubmatch(t)
|
||||
if f == nil {
|
||||
fmt.Fprintf(os.Stderr, "%s:%s\nmalformed //sys declaration\n", path, t)
|
||||
os.Exit(1)
|
||||
}
|
||||
funct, inps, outps, sysname := f[2], f[3], f[4], f[5]
|
||||
|
||||
// Split argument lists on comma.
|
||||
in := parseParamList(inps)
|
||||
out := parseParamList(outps)
|
||||
|
||||
// Try in vain to keep people from editing this file.
|
||||
// The theory is that they jump into the middle of the file
|
||||
// without reading the header.
|
||||
text += "// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT\n\n"
|
||||
|
||||
// Go function header.
|
||||
outDecl := ""
|
||||
if len(out) > 0 {
|
||||
outDecl = fmt.Sprintf(" (%s)", strings.Join(out, ", "))
|
||||
}
|
||||
text += fmt.Sprintf("func %s(%s)%s {\n", funct, strings.Join(in, ", "), outDecl)
|
||||
|
||||
// Check if err return available
|
||||
errvar := ""
|
||||
for _, param := range out {
|
||||
p := parseParam(param)
|
||||
if p.Type == "error" {
|
||||
errvar = p.Name
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// Prepare arguments to Syscall.
|
||||
var args []string
|
||||
n := 0
|
||||
for _, param := range in {
|
||||
p := parseParam(param)
|
||||
if regexp.MustCompile(`^\*`).FindStringSubmatch(p.Type) != nil {
|
||||
args = append(args, "uintptr(unsafe.Pointer("+p.Name+"))")
|
||||
} else if p.Type == "string" && errvar != "" {
|
||||
text += fmt.Sprintf("\tvar _p%d *byte\n", n)
|
||||
text += fmt.Sprintf("\t_p%d, %s = BytePtrFromString(%s)\n", n, errvar, p.Name)
|
||||
text += fmt.Sprintf("\tif %s != nil {\n\t\treturn\n\t}\n", errvar)
|
||||
args = append(args, fmt.Sprintf("uintptr(unsafe.Pointer(_p%d))", n))
|
||||
n++
|
||||
} else if p.Type == "string" {
|
||||
fmt.Fprintf(os.Stderr, path+":"+funct+" uses string arguments, but has no error return\n")
|
||||
text += fmt.Sprintf("\tvar _p%d *byte\n", n)
|
||||
text += fmt.Sprintf("\t_p%d, _ = BytePtrFromString(%s)\n", n, p.Name)
|
||||
args = append(args, fmt.Sprintf("uintptr(unsafe.Pointer(_p%d))", n))
|
||||
n++
|
||||
} else if regexp.MustCompile(`^\[\](.*)`).FindStringSubmatch(p.Type) != nil {
|
||||
// Convert slice into pointer, length.
|
||||
// Have to be careful not to take address of &a[0] if len == 0:
|
||||
// pass dummy pointer in that case.
|
||||
// Used to pass nil, but some OSes or simulators reject write(fd, nil, 0).
|
||||
text += fmt.Sprintf("\tvar _p%d unsafe.Pointer\n", n)
|
||||
text += fmt.Sprintf("\tif len(%s) > 0 {\n\t\t_p%d = unsafe.Pointer(&%s[0])\n\t}", p.Name, n, p.Name)
|
||||
text += fmt.Sprintf(" else {\n\t\t_p%d = unsafe.Pointer(&_zero)\n\t}\n", n)
|
||||
args = append(args, fmt.Sprintf("uintptr(_p%d)", n), fmt.Sprintf("uintptr(len(%s))", p.Name))
|
||||
n++
|
||||
} else if p.Type == "int64" && (*openbsd || *netbsd) {
|
||||
args = append(args, "0")
|
||||
if endianness == "big-endian" {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s>>32)", p.Name), fmt.Sprintf("uintptr(%s)", p.Name))
|
||||
} else if endianness == "little-endian" {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s)", p.Name), fmt.Sprintf("uintptr(%s>>32)", p.Name))
|
||||
} else {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s)", p.Name))
|
||||
}
|
||||
} else if p.Type == "int64" && *dragonfly {
|
||||
if regexp.MustCompile(`^(?i)extp(read|write)`).FindStringSubmatch(funct) == nil {
|
||||
args = append(args, "0")
|
||||
}
|
||||
if endianness == "big-endian" {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s>>32)", p.Name), fmt.Sprintf("uintptr(%s)", p.Name))
|
||||
} else if endianness == "little-endian" {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s)", p.Name), fmt.Sprintf("uintptr(%s>>32)", p.Name))
|
||||
} else {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s)", p.Name))
|
||||
}
|
||||
} else if p.Type == "int64" && endianness != "" {
|
||||
if len(args)%2 == 1 && *arm {
|
||||
// arm abi specifies 64-bit argument uses
|
||||
// (even, odd) pair
|
||||
args = append(args, "0")
|
||||
}
|
||||
if endianness == "big-endian" {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s>>32)", p.Name), fmt.Sprintf("uintptr(%s)", p.Name))
|
||||
} else {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s)", p.Name), fmt.Sprintf("uintptr(%s>>32)", p.Name))
|
||||
}
|
||||
} else {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s)", p.Name))
|
||||
}
|
||||
}
|
||||
|
||||
// Determine which form to use; pad args with zeros.
|
||||
asm := "Syscall"
|
||||
if nonblock != nil {
|
||||
if errvar == "" && goos == "linux" {
|
||||
asm = "RawSyscallNoError"
|
||||
} else {
|
||||
asm = "RawSyscall"
|
||||
}
|
||||
} else {
|
||||
if errvar == "" && goos == "linux" {
|
||||
asm = "SyscallNoError"
|
||||
}
|
||||
}
|
||||
if len(args) <= 3 {
|
||||
for len(args) < 3 {
|
||||
args = append(args, "0")
|
||||
}
|
||||
} else if len(args) <= 6 {
|
||||
asm += "6"
|
||||
for len(args) < 6 {
|
||||
args = append(args, "0")
|
||||
}
|
||||
} else if len(args) <= 9 {
|
||||
asm += "9"
|
||||
for len(args) < 9 {
|
||||
args = append(args, "0")
|
||||
}
|
||||
} else {
|
||||
fmt.Fprintf(os.Stderr, "%s:%s too many arguments to system call\n", path, funct)
|
||||
}
|
||||
|
||||
// System call number.
|
||||
if sysname == "" {
|
||||
sysname = "SYS_" + funct
|
||||
sysname = regexp.MustCompile(`([a-z])([A-Z])`).ReplaceAllString(sysname, `${1}_$2`)
|
||||
sysname = strings.ToUpper(sysname)
|
||||
}
|
||||
|
||||
var libcFn string
|
||||
if libc {
|
||||
asm = "syscall_" + strings.ToLower(asm[:1]) + asm[1:] // internal syscall call
|
||||
sysname = strings.TrimPrefix(sysname, "SYS_") // remove SYS_
|
||||
sysname = strings.ToLower(sysname) // lowercase
|
||||
if sysname == "getdirentries64" {
|
||||
// Special case - libSystem name and
|
||||
// raw syscall name don't match.
|
||||
sysname = "__getdirentries64"
|
||||
}
|
||||
libcFn = sysname
|
||||
sysname = "funcPC(libc_" + sysname + "_trampoline)"
|
||||
}
|
||||
|
||||
// Actual call.
|
||||
arglist := strings.Join(args, ", ")
|
||||
call := fmt.Sprintf("%s(%s, %s)", asm, sysname, arglist)
|
||||
|
||||
// Assign return values.
|
||||
body := ""
|
||||
ret := []string{"_", "_", "_"}
|
||||
doErrno := false
|
||||
for i := 0; i < len(out); i++ {
|
||||
p := parseParam(out[i])
|
||||
reg := ""
|
||||
if p.Name == "err" && !*plan9 {
|
||||
reg = "e1"
|
||||
ret[2] = reg
|
||||
doErrno = true
|
||||
} else if p.Name == "err" && *plan9 {
|
||||
ret[0] = "r0"
|
||||
ret[2] = "e1"
|
||||
break
|
||||
} else {
|
||||
reg = fmt.Sprintf("r%d", i)
|
||||
ret[i] = reg
|
||||
}
|
||||
if p.Type == "bool" {
|
||||
reg = fmt.Sprintf("%s != 0", reg)
|
||||
}
|
||||
if p.Type == "int64" && endianness != "" {
|
||||
// 64-bit number in r1:r0 or r0:r1.
|
||||
if i+2 > len(out) {
|
||||
fmt.Fprintf(os.Stderr, "%s:%s not enough registers for int64 return\n", path, funct)
|
||||
}
|
||||
if endianness == "big-endian" {
|
||||
reg = fmt.Sprintf("int64(r%d)<<32 | int64(r%d)", i, i+1)
|
||||
} else {
|
||||
reg = fmt.Sprintf("int64(r%d)<<32 | int64(r%d)", i+1, i)
|
||||
}
|
||||
ret[i] = fmt.Sprintf("r%d", i)
|
||||
ret[i+1] = fmt.Sprintf("r%d", i+1)
|
||||
}
|
||||
if reg != "e1" || *plan9 {
|
||||
body += fmt.Sprintf("\t%s = %s(%s)\n", p.Name, p.Type, reg)
|
||||
}
|
||||
}
|
||||
if ret[0] == "_" && ret[1] == "_" && ret[2] == "_" {
|
||||
text += fmt.Sprintf("\t%s\n", call)
|
||||
} else {
|
||||
if errvar == "" && goos == "linux" {
|
||||
// raw syscall without error on Linux, see golang.org/issue/22924
|
||||
text += fmt.Sprintf("\t%s, %s := %s\n", ret[0], ret[1], call)
|
||||
} else {
|
||||
text += fmt.Sprintf("\t%s, %s, %s := %s\n", ret[0], ret[1], ret[2], call)
|
||||
}
|
||||
}
|
||||
text += body
|
||||
|
||||
if *plan9 && ret[2] == "e1" {
|
||||
text += "\tif int32(r0) == -1 {\n"
|
||||
text += "\t\terr = e1\n"
|
||||
text += "\t}\n"
|
||||
} else if doErrno {
|
||||
text += "\tif e1 != 0 {\n"
|
||||
text += "\t\terr = errnoErr(e1)\n"
|
||||
text += "\t}\n"
|
||||
}
|
||||
text += "\treturn\n"
|
||||
text += "}\n\n"
|
||||
|
||||
if libc && !trampolines[libcFn] {
|
||||
// some system calls share a trampoline, like read and readlen.
|
||||
trampolines[libcFn] = true
|
||||
// Declare assembly trampoline.
|
||||
text += fmt.Sprintf("func libc_%s_trampoline()\n", libcFn)
|
||||
// Assembly trampoline calls the libc_* function, which this magic
|
||||
// redirects to use the function from libSystem.
|
||||
text += fmt.Sprintf("//go:linkname libc_%s libc_%s\n", libcFn, libcFn)
|
||||
text += fmt.Sprintf("//go:cgo_import_dynamic libc_%s %s \"/usr/lib/libSystem.B.dylib\"\n", libcFn, libcFn)
|
||||
text += "\n"
|
||||
}
|
||||
}
|
||||
if err := s.Err(); err != nil {
|
||||
fmt.Fprintf(os.Stderr, err.Error())
|
||||
os.Exit(1)
|
||||
}
|
||||
file.Close()
|
||||
}
|
||||
fmt.Printf(srcTemplate, cmdLine(), buildTags(), text)
|
||||
}
|
||||
|
||||
const srcTemplate = `// %s
|
||||
// Code generated by the command above; see README.md. DO NOT EDIT.
|
||||
|
||||
// +build %s
|
||||
|
||||
package unix
|
||||
|
||||
import (
|
||||
"syscall"
|
||||
"unsafe"
|
||||
)
|
||||
|
||||
var _ syscall.Errno
|
||||
|
||||
%s
|
||||
`
|
404
vendor/golang.org/x/sys/unix/mksyscall_aix_ppc.go
generated
vendored
404
vendor/golang.org/x/sys/unix/mksyscall_aix_ppc.go
generated
vendored
@@ -1,404 +0,0 @@
|
||||
// Copyright 2019 The Go Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
// +build ignore
|
||||
|
||||
/*
|
||||
This program reads a file containing function prototypes
|
||||
(like syscall_aix.go) and generates system call bodies.
|
||||
The prototypes are marked by lines beginning with "//sys"
|
||||
and read like func declarations if //sys is replaced by func, but:
|
||||
* The parameter lists must give a name for each argument.
|
||||
This includes return parameters.
|
||||
* The parameter lists must give a type for each argument:
|
||||
the (x, y, z int) shorthand is not allowed.
|
||||
* If the return parameter is an error number, it must be named err.
|
||||
* If go func name needs to be different than its libc name,
|
||||
* or the function is not in libc, name could be specified
|
||||
* at the end, after "=" sign, like
|
||||
//sys getsockopt(s int, level int, name int, val uintptr, vallen *_Socklen) (err error) = libsocket.getsockopt
|
||||
*/
|
||||
package main
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"flag"
|
||||
"fmt"
|
||||
"os"
|
||||
"regexp"
|
||||
"strings"
|
||||
)
|
||||
|
||||
var (
|
||||
b32 = flag.Bool("b32", false, "32bit big-endian")
|
||||
l32 = flag.Bool("l32", false, "32bit little-endian")
|
||||
aix = flag.Bool("aix", false, "aix")
|
||||
tags = flag.String("tags", "", "build tags")
|
||||
)
|
||||
|
||||
// cmdLine returns this programs's commandline arguments
|
||||
func cmdLine() string {
|
||||
return "go run mksyscall_aix_ppc.go " + strings.Join(os.Args[1:], " ")
|
||||
}
|
||||
|
||||
// buildTags returns build tags
|
||||
func buildTags() string {
|
||||
return *tags
|
||||
}
|
||||
|
||||
// Param is function parameter
|
||||
type Param struct {
|
||||
Name string
|
||||
Type string
|
||||
}
|
||||
|
||||
// usage prints the program usage
|
||||
func usage() {
|
||||
fmt.Fprintf(os.Stderr, "usage: go run mksyscall_aix_ppc.go [-b32 | -l32] [-tags x,y] [file ...]\n")
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
// parseParamList parses parameter list and returns a slice of parameters
|
||||
func parseParamList(list string) []string {
|
||||
list = strings.TrimSpace(list)
|
||||
if list == "" {
|
||||
return []string{}
|
||||
}
|
||||
return regexp.MustCompile(`\s*,\s*`).Split(list, -1)
|
||||
}
|
||||
|
||||
// parseParam splits a parameter into name and type
|
||||
func parseParam(p string) Param {
|
||||
ps := regexp.MustCompile(`^(\S*) (\S*)$`).FindStringSubmatch(p)
|
||||
if ps == nil {
|
||||
fmt.Fprintf(os.Stderr, "malformed parameter: %s\n", p)
|
||||
os.Exit(1)
|
||||
}
|
||||
return Param{ps[1], ps[2]}
|
||||
}
|
||||
|
||||
func main() {
|
||||
flag.Usage = usage
|
||||
flag.Parse()
|
||||
if len(flag.Args()) <= 0 {
|
||||
fmt.Fprintf(os.Stderr, "no files to parse provided\n")
|
||||
usage()
|
||||
}
|
||||
|
||||
endianness := ""
|
||||
if *b32 {
|
||||
endianness = "big-endian"
|
||||
} else if *l32 {
|
||||
endianness = "little-endian"
|
||||
}
|
||||
|
||||
pack := ""
|
||||
text := ""
|
||||
cExtern := "/*\n#include <stdint.h>\n#include <stddef.h>\n"
|
||||
for _, path := range flag.Args() {
|
||||
file, err := os.Open(path)
|
||||
if err != nil {
|
||||
fmt.Fprintf(os.Stderr, err.Error())
|
||||
os.Exit(1)
|
||||
}
|
||||
s := bufio.NewScanner(file)
|
||||
for s.Scan() {
|
||||
t := s.Text()
|
||||
t = strings.TrimSpace(t)
|
||||
t = regexp.MustCompile(`\s+`).ReplaceAllString(t, ` `)
|
||||
if p := regexp.MustCompile(`^package (\S+)$`).FindStringSubmatch(t); p != nil && pack == "" {
|
||||
pack = p[1]
|
||||
}
|
||||
nonblock := regexp.MustCompile(`^\/\/sysnb `).FindStringSubmatch(t)
|
||||
if regexp.MustCompile(`^\/\/sys `).FindStringSubmatch(t) == nil && nonblock == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// Line must be of the form
|
||||
// func Open(path string, mode int, perm int) (fd int, err error)
|
||||
// Split into name, in params, out params.
|
||||
f := regexp.MustCompile(`^\/\/sys(nb)? (\w+)\(([^()]*)\)\s*(?:\(([^()]+)\))?\s*(?:=\s*(?:(\w*)\.)?(\w*))?$`).FindStringSubmatch(t)
|
||||
if f == nil {
|
||||
fmt.Fprintf(os.Stderr, "%s:%s\nmalformed //sys declaration\n", path, t)
|
||||
os.Exit(1)
|
||||
}
|
||||
funct, inps, outps, modname, sysname := f[2], f[3], f[4], f[5], f[6]
|
||||
|
||||
// Split argument lists on comma.
|
||||
in := parseParamList(inps)
|
||||
out := parseParamList(outps)
|
||||
|
||||
inps = strings.Join(in, ", ")
|
||||
outps = strings.Join(out, ", ")
|
||||
|
||||
// Try in vain to keep people from editing this file.
|
||||
// The theory is that they jump into the middle of the file
|
||||
// without reading the header.
|
||||
text += "// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT\n\n"
|
||||
|
||||
// Check if value return, err return available
|
||||
errvar := ""
|
||||
retvar := ""
|
||||
rettype := ""
|
||||
for _, param := range out {
|
||||
p := parseParam(param)
|
||||
if p.Type == "error" {
|
||||
errvar = p.Name
|
||||
} else {
|
||||
retvar = p.Name
|
||||
rettype = p.Type
|
||||
}
|
||||
}
|
||||
|
||||
// System call name.
|
||||
if sysname == "" {
|
||||
sysname = funct
|
||||
}
|
||||
sysname = regexp.MustCompile(`([a-z])([A-Z])`).ReplaceAllString(sysname, `${1}_$2`)
|
||||
sysname = strings.ToLower(sysname) // All libc functions are lowercase.
|
||||
|
||||
cRettype := ""
|
||||
if rettype == "unsafe.Pointer" {
|
||||
cRettype = "uintptr_t"
|
||||
} else if rettype == "uintptr" {
|
||||
cRettype = "uintptr_t"
|
||||
} else if regexp.MustCompile(`^_`).FindStringSubmatch(rettype) != nil {
|
||||
cRettype = "uintptr_t"
|
||||
} else if rettype == "int" {
|
||||
cRettype = "int"
|
||||
} else if rettype == "int32" {
|
||||
cRettype = "int"
|
||||
} else if rettype == "int64" {
|
||||
cRettype = "long long"
|
||||
} else if rettype == "uint32" {
|
||||
cRettype = "unsigned int"
|
||||
} else if rettype == "uint64" {
|
||||
cRettype = "unsigned long long"
|
||||
} else {
|
||||
cRettype = "int"
|
||||
}
|
||||
if sysname == "exit" {
|
||||
cRettype = "void"
|
||||
}
|
||||
|
||||
// Change p.Types to c
|
||||
var cIn []string
|
||||
for _, param := range in {
|
||||
p := parseParam(param)
|
||||
if regexp.MustCompile(`^\*`).FindStringSubmatch(p.Type) != nil {
|
||||
cIn = append(cIn, "uintptr_t")
|
||||
} else if p.Type == "string" {
|
||||
cIn = append(cIn, "uintptr_t")
|
||||
} else if regexp.MustCompile(`^\[\](.*)`).FindStringSubmatch(p.Type) != nil {
|
||||
cIn = append(cIn, "uintptr_t", "size_t")
|
||||
} else if p.Type == "unsafe.Pointer" {
|
||||
cIn = append(cIn, "uintptr_t")
|
||||
} else if p.Type == "uintptr" {
|
||||
cIn = append(cIn, "uintptr_t")
|
||||
} else if regexp.MustCompile(`^_`).FindStringSubmatch(p.Type) != nil {
|
||||
cIn = append(cIn, "uintptr_t")
|
||||
} else if p.Type == "int" {
|
||||
cIn = append(cIn, "int")
|
||||
} else if p.Type == "int32" {
|
||||
cIn = append(cIn, "int")
|
||||
} else if p.Type == "int64" {
|
||||
cIn = append(cIn, "long long")
|
||||
} else if p.Type == "uint32" {
|
||||
cIn = append(cIn, "unsigned int")
|
||||
} else if p.Type == "uint64" {
|
||||
cIn = append(cIn, "unsigned long long")
|
||||
} else {
|
||||
cIn = append(cIn, "int")
|
||||
}
|
||||
}
|
||||
|
||||
if funct != "fcntl" && funct != "FcntlInt" && funct != "readlen" && funct != "writelen" {
|
||||
// Imports of system calls from libc
|
||||
cExtern += fmt.Sprintf("%s %s", cRettype, sysname)
|
||||
cIn := strings.Join(cIn, ", ")
|
||||
cExtern += fmt.Sprintf("(%s);\n", cIn)
|
||||
}
|
||||
|
||||
// So file name.
|
||||
if *aix {
|
||||
if modname == "" {
|
||||
modname = "libc.a/shr_64.o"
|
||||
} else {
|
||||
fmt.Fprintf(os.Stderr, "%s: only syscall using libc are available\n", funct)
|
||||
os.Exit(1)
|
||||
}
|
||||
}
|
||||
|
||||
strconvfunc := "C.CString"
|
||||
|
||||
// Go function header.
|
||||
if outps != "" {
|
||||
outps = fmt.Sprintf(" (%s)", outps)
|
||||
}
|
||||
if text != "" {
|
||||
text += "\n"
|
||||
}
|
||||
|
||||
text += fmt.Sprintf("func %s(%s)%s {\n", funct, strings.Join(in, ", "), outps)
|
||||
|
||||
// Prepare arguments to Syscall.
|
||||
var args []string
|
||||
n := 0
|
||||
argN := 0
|
||||
for _, param := range in {
|
||||
p := parseParam(param)
|
||||
if regexp.MustCompile(`^\*`).FindStringSubmatch(p.Type) != nil {
|
||||
args = append(args, "C.uintptr_t(uintptr(unsafe.Pointer("+p.Name+")))")
|
||||
} else if p.Type == "string" && errvar != "" {
|
||||
text += fmt.Sprintf("\t_p%d := uintptr(unsafe.Pointer(%s(%s)))\n", n, strconvfunc, p.Name)
|
||||
args = append(args, fmt.Sprintf("C.uintptr_t(_p%d)", n))
|
||||
n++
|
||||
} else if p.Type == "string" {
|
||||
fmt.Fprintf(os.Stderr, path+":"+funct+" uses string arguments, but has no error return\n")
|
||||
text += fmt.Sprintf("\t_p%d := uintptr(unsafe.Pointer(%s(%s)))\n", n, strconvfunc, p.Name)
|
||||
args = append(args, fmt.Sprintf("C.uintptr_t(_p%d)", n))
|
||||
n++
|
||||
} else if m := regexp.MustCompile(`^\[\](.*)`).FindStringSubmatch(p.Type); m != nil {
|
||||
// Convert slice into pointer, length.
|
||||
// Have to be careful not to take address of &a[0] if len == 0:
|
||||
// pass nil in that case.
|
||||
text += fmt.Sprintf("\tvar _p%d *%s\n", n, m[1])
|
||||
text += fmt.Sprintf("\tif len(%s) > 0 {\n\t\t_p%d = &%s[0]\n\t}\n", p.Name, n, p.Name)
|
||||
args = append(args, fmt.Sprintf("C.uintptr_t(uintptr(unsafe.Pointer(_p%d)))", n))
|
||||
n++
|
||||
text += fmt.Sprintf("\tvar _p%d int\n", n)
|
||||
text += fmt.Sprintf("\t_p%d = len(%s)\n", n, p.Name)
|
||||
args = append(args, fmt.Sprintf("C.size_t(_p%d)", n))
|
||||
n++
|
||||
} else if p.Type == "int64" && endianness != "" {
|
||||
if endianness == "big-endian" {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s>>32)", p.Name), fmt.Sprintf("uintptr(%s)", p.Name))
|
||||
} else {
|
||||
args = append(args, fmt.Sprintf("uintptr(%s)", p.Name), fmt.Sprintf("uintptr(%s>>32)", p.Name))
|
||||
}
|
||||
n++
|
||||
} else if p.Type == "bool" {
|
||||
text += fmt.Sprintf("\tvar _p%d uint32\n", n)
|
||||
text += fmt.Sprintf("\tif %s {\n\t\t_p%d = 1\n\t} else {\n\t\t_p%d = 0\n\t}\n", p.Name, n, n)
|
||||
args = append(args, fmt.Sprintf("_p%d", n))
|
||||
} else if regexp.MustCompile(`^_`).FindStringSubmatch(p.Type) != nil {
|
||||
args = append(args, fmt.Sprintf("C.uintptr_t(uintptr(%s))", p.Name))
|
||||
} else if p.Type == "unsafe.Pointer" {
|
||||
args = append(args, fmt.Sprintf("C.uintptr_t(uintptr(%s))", p.Name))
|
||||
} else if p.Type == "int" {
|
||||
if (argN == 2) && ((funct == "readlen") || (funct == "writelen")) {
|
||||
args = append(args, fmt.Sprintf("C.size_t(%s)", p.Name))
|
||||
} else if argN == 0 && funct == "fcntl" {
|
||||
args = append(args, fmt.Sprintf("C.uintptr_t(%s)", p.Name))
|
||||
} else if (argN == 2) && ((funct == "fcntl") || (funct == "FcntlInt")) {
|
||||
args = append(args, fmt.Sprintf("C.uintptr_t(%s)", p.Name))
|
||||
} else {
|
||||
args = append(args, fmt.Sprintf("C.int(%s)", p.Name))
|
||||
}
|
||||
} else if p.Type == "int32" {
|
||||
args = append(args, fmt.Sprintf("C.int(%s)", p.Name))
|
||||
} else if p.Type == "int64" {
|
||||
args = append(args, fmt.Sprintf("C.longlong(%s)", p.Name))
|
||||
} else if p.Type == "uint32" {
|
||||
args = append(args, fmt.Sprintf("C.uint(%s)", p.Name))
|
||||
} else if p.Type == "uint64" {
|
||||
args = append(args, fmt.Sprintf("C.ulonglong(%s)", p.Name))
|
||||
} else if p.Type == "uintptr" {
|
||||
args = append(args, fmt.Sprintf("C.uintptr_t(%s)", p.Name))
|
||||
} else {
|
||||
args = append(args, fmt.Sprintf("C.int(%s)", p.Name))
|
||||
}
|
||||
argN++
|
||||
}
|
||||
|
||||
// Actual call.
|
||||
arglist := strings.Join(args, ", ")
|
||||
call := ""
|
||||
if sysname == "exit" {
|
||||
if errvar != "" {
|
||||
call += "er :="
|
||||
} else {
|
||||
call += ""
|
||||
}
|
||||
} else if errvar != "" {
|
||||
call += "r0,er :="
|
||||
} else if retvar != "" {
|
||||
call += "r0,_ :="
|
||||
} else {
|
||||
call += ""
|
||||
}
|
||||
call += fmt.Sprintf("C.%s(%s)", sysname, arglist)
|
||||
|
||||
// Assign return values.
|
||||
body := ""
|
||||
for i := 0; i < len(out); i++ {
|
||||
p := parseParam(out[i])
|
||||
reg := ""
|
||||
if p.Name == "err" {
|
||||
reg = "e1"
|
||||
} else {
|
||||
reg = "r0"
|
||||
}
|
||||
if reg != "e1" {
|
||||
body += fmt.Sprintf("\t%s = %s(%s)\n", p.Name, p.Type, reg)
|
||||
}
|
||||
}
|
||||
|
||||
// verify return
|
||||
if sysname != "exit" && errvar != "" {
|
||||
if regexp.MustCompile(`^uintptr`).FindStringSubmatch(cRettype) != nil {
|
||||
body += "\tif (uintptr(r0) ==^uintptr(0) && er != nil) {\n"
|
||||
body += fmt.Sprintf("\t\t%s = er\n", errvar)
|
||||
body += "\t}\n"
|
||||
} else {
|
||||
body += "\tif (r0 ==-1 && er != nil) {\n"
|
||||
body += fmt.Sprintf("\t\t%s = er\n", errvar)
|
||||
body += "\t}\n"
|
||||
}
|
||||
} else if errvar != "" {
|
||||
body += "\tif (er != nil) {\n"
|
||||
body += fmt.Sprintf("\t\t%s = er\n", errvar)
|
||||
body += "\t}\n"
|
||||
}
|
||||
|
||||
text += fmt.Sprintf("\t%s\n", call)
|
||||
text += body
|
||||
|
||||
text += "\treturn\n"
|
||||
text += "}\n"
|
||||
}
|
||||
if err := s.Err(); err != nil {
|
||||
fmt.Fprintf(os.Stderr, err.Error())
|
||||
os.Exit(1)
|
||||
}
|
||||
file.Close()
|
||||
}
|
||||
imp := ""
|
||||
if pack != "unix" {
|
||||
imp = "import \"golang.org/x/sys/unix\"\n"
|
||||
|
||||
}
|
||||
fmt.Printf(srcTemplate, cmdLine(), buildTags(), pack, cExtern, imp, text)
|
||||
}
|
||||
|
||||
const srcTemplate = `// %s
|
||||
// Code generated by the command above; see README.md. DO NOT EDIT.
|
||||
|
||||
// +build %s
|
||||
|
||||
package %s
|
||||
|
||||
|
||||
%s
|
||||
*/
|
||||
import "C"
|
||||
import (
|
||||
"unsafe"
|
||||
)
|
||||
|
||||
|
||||
%s
|
||||
|
||||
%s
|
||||
`
|
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user