mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #9651 from gyuho/netutil
pkg/netutil: use structured logging for TCP resolve
This commit is contained in:
commit
e83cc218fc
@ -180,7 +180,7 @@ func (c *ServerConfig) advertiseMatchesCluster() error {
|
||||
sort.Strings(apurls)
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), 30*time.Second)
|
||||
defer cancel()
|
||||
ok, err := netutil.URLStringsEqual(ctx, apurls, urls.StringSlice())
|
||||
ok, err := netutil.URLStringsEqual(ctx, c.Logger, apurls, urls.StringSlice())
|
||||
if ok {
|
||||
return nil
|
||||
}
|
||||
|
@ -649,7 +649,7 @@ func clusterVersionFromStore(lg *zap.Logger, st v2store.Store) *semver.Version {
|
||||
// with the existing cluster. If the validation succeeds, it assigns the IDs
|
||||
// from the existing cluster to the local cluster.
|
||||
// If the validation fails, an error will be returned.
|
||||
func ValidateClusterAndAssignIDs(local *RaftCluster, existing *RaftCluster) error {
|
||||
func ValidateClusterAndAssignIDs(lg *zap.Logger, local *RaftCluster, existing *RaftCluster) error {
|
||||
ems := existing.Members()
|
||||
lms := local.Members()
|
||||
if len(ems) != len(lms) {
|
||||
@ -661,7 +661,7 @@ func ValidateClusterAndAssignIDs(local *RaftCluster, existing *RaftCluster) erro
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), 30*time.Second)
|
||||
defer cancel()
|
||||
for i := range ems {
|
||||
if ok, err := netutil.URLStringsEqual(ctx, ems[i].PeerURLs, lms[i].PeerURLs); !ok {
|
||||
if ok, err := netutil.URLStringsEqual(ctx, lg, ems[i].PeerURLs, lms[i].PeerURLs); !ok {
|
||||
return fmt.Errorf("unmatched member while checking PeerURLs (%v)", err)
|
||||
}
|
||||
lms[i].ID = ems[i].ID
|
||||
|
@ -239,7 +239,7 @@ func TestClusterValidateAndAssignIDsBad(t *testing.T) {
|
||||
for i, tt := range tests {
|
||||
ecl := newTestCluster(tt.clmembs)
|
||||
lcl := newTestCluster(tt.membs)
|
||||
if err := ValidateClusterAndAssignIDs(lcl, ecl); err == nil {
|
||||
if err := ValidateClusterAndAssignIDs(zap.NewExample(), lcl, ecl); err == nil {
|
||||
t.Errorf("#%d: unexpected update success", i)
|
||||
}
|
||||
}
|
||||
@ -266,7 +266,7 @@ func TestClusterValidateAndAssignIDs(t *testing.T) {
|
||||
for i, tt := range tests {
|
||||
lcl := newTestCluster(tt.clmembs)
|
||||
ecl := newTestCluster(tt.membs)
|
||||
if err := ValidateClusterAndAssignIDs(lcl, ecl); err != nil {
|
||||
if err := ValidateClusterAndAssignIDs(zap.NewExample(), lcl, ecl); err != nil {
|
||||
t.Errorf("#%d: unexpect update error: %v", i, err)
|
||||
}
|
||||
if !reflect.DeepEqual(lcl.MemberIDs(), tt.wids) {
|
||||
|
@ -340,7 +340,7 @@ func NewServer(cfg ServerConfig) (srv *EtcdServer, err error) {
|
||||
if gerr != nil {
|
||||
return nil, fmt.Errorf("cannot fetch cluster info from peer urls: %v", gerr)
|
||||
}
|
||||
if err = membership.ValidateClusterAndAssignIDs(cl, existingCluster); err != nil {
|
||||
if err = membership.ValidateClusterAndAssignIDs(cfg.Logger, cl, existingCluster); err != nil {
|
||||
return nil, fmt.Errorf("error validating peerURLs %s: %v", existingCluster, err)
|
||||
}
|
||||
if !isCompatibleWithCluster(cfg.Logger, cl, cl.MemberByName(cfg.Name).ID, prt) {
|
||||
|
16
pkg/netutil/doc.go
Normal file
16
pkg/netutil/doc.go
Normal file
@ -0,0 +1,16 @@
|
||||
// Copyright 2018 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 netutil implements network-related utility functions.
|
||||
package netutil
|
@ -12,7 +12,6 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
// Package netutil implements network-related utility functions.
|
||||
package netutil
|
||||
|
||||
import (
|
||||
@ -25,15 +24,12 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/pkg/capnslog"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
var (
|
||||
plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "pkg/netutil")
|
||||
|
||||
// indirection for testing
|
||||
resolveTCPAddr = resolveTCPAddrDefault
|
||||
)
|
||||
// indirection for testing
|
||||
var resolveTCPAddr = resolveTCPAddrDefault
|
||||
|
||||
const retryInterval = time.Second
|
||||
|
||||
@ -67,7 +63,7 @@ func resolveTCPAddrDefault(ctx context.Context, addr string) (*net.TCPAddr, erro
|
||||
// resolveTCPAddrs is a convenience wrapper for net.ResolveTCPAddr.
|
||||
// resolveTCPAddrs return a new set of url.URLs, in which all DNS hostnames
|
||||
// are resolved.
|
||||
func resolveTCPAddrs(ctx context.Context, urls [][]url.URL) ([][]url.URL, error) {
|
||||
func resolveTCPAddrs(ctx context.Context, lg *zap.Logger, urls [][]url.URL) ([][]url.URL, error) {
|
||||
newurls := make([][]url.URL, 0)
|
||||
for _, us := range urls {
|
||||
nus := make([]url.URL, len(us))
|
||||
@ -79,7 +75,7 @@ func resolveTCPAddrs(ctx context.Context, urls [][]url.URL) ([][]url.URL, error)
|
||||
nus[i] = *nu
|
||||
}
|
||||
for i, u := range nus {
|
||||
h, err := resolveURL(ctx, u)
|
||||
h, err := resolveURL(ctx, lg, u)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to resolve %q (%v)", u.String(), err)
|
||||
}
|
||||
@ -92,14 +88,19 @@ func resolveTCPAddrs(ctx context.Context, urls [][]url.URL) ([][]url.URL, error)
|
||||
return newurls, nil
|
||||
}
|
||||
|
||||
func resolveURL(ctx context.Context, u url.URL) (string, error) {
|
||||
func resolveURL(ctx context.Context, lg *zap.Logger, u url.URL) (string, error) {
|
||||
if u.Scheme == "unix" || u.Scheme == "unixs" {
|
||||
// unix sockets don't resolve over TCP
|
||||
return "", nil
|
||||
}
|
||||
host, _, err := net.SplitHostPort(u.Host)
|
||||
if err != nil {
|
||||
plog.Errorf("could not parse url %s during tcp resolving", u.Host)
|
||||
lg.Warn(
|
||||
"failed to parse URL Host while resolving URL",
|
||||
zap.String("url", u.String()),
|
||||
zap.String("host", u.Host),
|
||||
zap.Error(err),
|
||||
)
|
||||
return "", err
|
||||
}
|
||||
if host == "localhost" || net.ParseIP(host) != nil {
|
||||
@ -108,13 +109,32 @@ func resolveURL(ctx context.Context, u url.URL) (string, error) {
|
||||
for ctx.Err() == nil {
|
||||
tcpAddr, err := resolveTCPAddr(ctx, u.Host)
|
||||
if err == nil {
|
||||
plog.Infof("resolving %s to %s", u.Host, tcpAddr.String())
|
||||
lg.Info(
|
||||
"resolved URL Host",
|
||||
zap.String("url", u.String()),
|
||||
zap.String("host", u.Host),
|
||||
zap.String("resolved-addr", tcpAddr.String()),
|
||||
)
|
||||
return tcpAddr.String(), nil
|
||||
}
|
||||
plog.Warningf("failed resolving host %s (%v); retrying in %v", u.Host, err, retryInterval)
|
||||
|
||||
lg.Warn(
|
||||
"failed to resolve URL Host",
|
||||
zap.String("url", u.String()),
|
||||
zap.String("host", u.Host),
|
||||
zap.Duration("retry-interval", retryInterval),
|
||||
zap.Error(err),
|
||||
)
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
plog.Errorf("could not resolve host %s", u.Host)
|
||||
lg.Warn(
|
||||
"failed to resolve URL Host; returning",
|
||||
zap.String("url", u.String()),
|
||||
zap.String("host", u.Host),
|
||||
zap.Duration("retry-interval", retryInterval),
|
||||
zap.Error(err),
|
||||
)
|
||||
return "", err
|
||||
case <-time.After(retryInterval):
|
||||
}
|
||||
@ -124,11 +144,11 @@ func resolveURL(ctx context.Context, u url.URL) (string, error) {
|
||||
|
||||
// urlsEqual checks equality of url.URLS between two arrays.
|
||||
// This check pass even if an URL is in hostname and opposite is in IP address.
|
||||
func urlsEqual(ctx context.Context, a []url.URL, b []url.URL) (bool, error) {
|
||||
func urlsEqual(ctx context.Context, lg *zap.Logger, a []url.URL, b []url.URL) (bool, error) {
|
||||
if len(a) != len(b) {
|
||||
return false, fmt.Errorf("len(%q) != len(%q)", urlsToStrings(a), urlsToStrings(b))
|
||||
}
|
||||
urls, err := resolveTCPAddrs(ctx, [][]url.URL{a, b})
|
||||
urls, err := resolveTCPAddrs(ctx, lg, [][]url.URL{a, b})
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
@ -150,7 +170,7 @@ func urlsEqual(ctx context.Context, a []url.URL, b []url.URL) (bool, error) {
|
||||
// URLStringsEqual returns "true" if given URLs are valid
|
||||
// and resolved to same IP addresses. Otherwise, return "false"
|
||||
// and error, if any.
|
||||
func URLStringsEqual(ctx context.Context, a []string, b []string) (bool, error) {
|
||||
func URLStringsEqual(ctx context.Context, lg *zap.Logger, a []string, b []string) (bool, error) {
|
||||
if len(a) != len(b) {
|
||||
return false, fmt.Errorf("len(%q) != len(%q)", a, b)
|
||||
}
|
||||
@ -170,7 +190,13 @@ func URLStringsEqual(ctx context.Context, a []string, b []string) (bool, error)
|
||||
}
|
||||
urlsB = append(urlsB, *u)
|
||||
}
|
||||
return urlsEqual(ctx, urlsA, urlsB)
|
||||
if lg == nil {
|
||||
lg, _ = zap.NewProduction()
|
||||
if lg == nil {
|
||||
lg = zap.NewExample()
|
||||
}
|
||||
}
|
||||
return urlsEqual(ctx, lg, urlsA, urlsB)
|
||||
}
|
||||
|
||||
func urlsToStrings(us []url.URL) []string {
|
||||
|
@ -23,6 +23,8 @@ import (
|
||||
"strconv"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
func TestResolveTCPAddrs(t *testing.T) {
|
||||
@ -118,7 +120,7 @@ func TestResolveTCPAddrs(t *testing.T) {
|
||||
return nil, err
|
||||
}
|
||||
if tt.hostMap[host] == "" {
|
||||
return nil, errors.New("cannot resolve host.")
|
||||
return nil, errors.New("cannot resolve host")
|
||||
}
|
||||
i, err := strconv.Atoi(port)
|
||||
if err != nil {
|
||||
@ -127,7 +129,7 @@ func TestResolveTCPAddrs(t *testing.T) {
|
||||
return &net.TCPAddr{IP: net.ParseIP(tt.hostMap[host]), Port: i, Zone: ""}, nil
|
||||
}
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), time.Second)
|
||||
urls, err := resolveTCPAddrs(ctx, tt.urls)
|
||||
urls, err := resolveTCPAddrs(ctx, zap.NewExample(), tt.urls)
|
||||
cancel()
|
||||
if tt.hasError {
|
||||
if err == nil {
|
||||
@ -278,7 +280,7 @@ func TestURLsEqual(t *testing.T) {
|
||||
}
|
||||
|
||||
for i, test := range tests {
|
||||
result, err := urlsEqual(context.TODO(), test.a, test.b)
|
||||
result, err := urlsEqual(context.TODO(), zap.NewExample(), test.a, test.b)
|
||||
if result != test.expect {
|
||||
t.Errorf("#%d: a:%v b:%v, expected %v but %v", i, test.a, test.b, test.expect, result)
|
||||
}
|
||||
@ -290,7 +292,7 @@ func TestURLsEqual(t *testing.T) {
|
||||
}
|
||||
}
|
||||
func TestURLStringsEqual(t *testing.T) {
|
||||
result, err := URLStringsEqual(context.TODO(), []string{"http://127.0.0.1:8080"}, []string{"http://127.0.0.1:8080"})
|
||||
result, err := URLStringsEqual(context.TODO(), zap.NewExample(), []string{"http://127.0.0.1:8080"}, []string{"http://127.0.0.1:8080"})
|
||||
if !result {
|
||||
t.Errorf("unexpected result %v", result)
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user