Skip to content

Commit

Permalink
Merge #29574 #29585
Browse files Browse the repository at this point in the history
29574: storage: update import path for Raft r=petermattis a=tschottdorf

upstream moved from github.com/etcd to go.etcd.io/etcd. We're going
to have to pick up an update soon to fix #28918, so it's easier to
switch now.

We're not picking up any new commits except for the renames.

Release note: None

29585: gossip: avoid allocation of UnresolvedAddr in getNodeIDAddressLocked r=nvanbenschoten a=nvanbenschoten

`getNodeIDAddressLocked` is called from `Dialer.ConnHealth` and
`Dialer.DialInternalClient`. It was responsible for **1.71%** of all
allocations (`alloc_objects`) on a 3-node long-running cluster that
was running TPC-C 1K.

Pointing into `nd.LocalityAddress` is safe because even if the `NodeDescriptor`
itself is replaced in `Gossip`, the struct is never internally mutated. This is
the same reason why taking the address of `nd.Address` was already safe.

Release note (performance improvement): Avoid allocation when
checking RPC connection health.

Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
  • Loading branch information
3 people committed Sep 5, 2018
3 parents 3e62ce0 + 3afbd21 + 9d226a4 commit 44a5f06
Show file tree
Hide file tree
Showing 42 changed files with 65 additions and 76 deletions.
34 changes: 13 additions & 21 deletions Gopkg.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

5 changes: 1 addition & 4 deletions Gopkg.toml
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,8 @@ ignored = [
name = "golang.org/x/text"
revision = "470f45bf29f4147d6fbd7dfd0a02a848e49f5bf4"

# https://github.com/coreos/etcd/commit/f03ed33
#
# https://github.com/coreos/etcd/commit/ce0ad377
[[constraint]]
name = "github.com/coreos/etcd"
name = "go.etcd.io/etcd"
branch = "master"

# Used for the API client; we want the latest.
Expand Down
4 changes: 2 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -989,7 +989,7 @@ PROTOBUF_PATH := $(GOGO_PROTOBUF_PATH)/protobuf

GOGOPROTO_PROTO := $(GOGO_PROTOBUF_PATH)/gogoproto/gogo.proto

COREOS_PATH := ./vendor/github.com/coreos
COREOS_PATH := ./vendor/go.etcd.io

GRPC_GATEWAY_GOOGLEAPIS_PACKAGE := github.com/grpc-ecosystem/grpc-gateway/third_party/googleapis
GRPC_GATEWAY_GOOGLEAPIS_PATH := ./vendor/$(GRPC_GATEWAY_GOOGLEAPIS_PACKAGE)
Expand Down Expand Up @@ -1041,7 +1041,7 @@ bin/.go_protobuf_sources: $(PROTOC) $(GO_PROTOS) $(GOGOPROTO_PROTO) bin/.bootstr
done
$(SED_INPLACE) '/import _/d' $(GO_SOURCES)
$(SED_INPLACE) -E 's!import (fmt|math) "github.com/cockroachdb/cockroach/pkg/(fmt|math)"! !g' $(GO_SOURCES)
$(SED_INPLACE) -E 's!cockroachdb/cockroach/pkg/(etcd)!coreos/\1!g' $(GO_SOURCES)
$(SED_INPLACE) -E 's!github\.com/cockroachdb/cockroach/pkg/(etcd)!go.etcd.io/\1!g' $(GO_SOURCES)
$(SED_INPLACE) -E 's!cockroachdb/cockroach/pkg/(prometheus/client_model)!\1/go!g' $(GO_SOURCES)
$(SED_INPLACE) -E 's!github.com/cockroachdb/cockroach/pkg/(bytes|encoding/binary|errors|fmt|io|math|github\.com|(google\.)?golang\.org)!\1!g' $(GO_SOURCES)
@# TODO(benesch): Remove after https://github.com/grpc/grpc-go/issues/711.
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,11 +54,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/sysutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/coreos/etcd/raft/raftpb"
"github.com/gogo/protobuf/jsonpb"
"github.com/kr/pretty"
"github.com/pkg/errors"
"github.com/spf13/cobra"
"go.etcd.io/etcd/raft/raftpb"
)

var debugKeysCmd = &cobra.Command{
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/debug/print.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/storagebase"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/coreos/etcd/raft/raftpb"
"go.etcd.io/etcd/raft/raftpb"
)

// PrintKeyValue attempts to pretty-print the specified MVCCKeyValue to
Expand Down
6 changes: 3 additions & 3 deletions pkg/gossip/gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -1000,13 +1000,13 @@ func (g *Gossip) getNodeIDAddressLocked(nodeID roachpb.NodeID) (*util.Unresolved
if err != nil {
return nil, err
}
address := &nd.Address
for _, locality := range nd.LocalityAddress {
for i := range nd.LocalityAddress {
locality := &nd.LocalityAddress[i]
if _, ok := g.localityTierMap[locality.LocalityTier.String()]; ok {
return &locality.Address, nil
}
}
return address, nil
return &nd.Address, nil
}

// AddInfo adds or updates an info object. Returns an error if info
Expand Down
4 changes: 2 additions & 2 deletions pkg/server/serverpb/status.pb.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 1 addition & 1 deletion pkg/server/serverpb/status.proto
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ message NodeRequest {
}

// RaftState gives internal details about a Raft group's state.
// Closely mirrors the upstream definitions in github.com/coreos/etcd/raft.
// Closely mirrors the upstream definitions in github.com/etcd-io/etcd/raft.
message RaftState {
message Progress {
uint64 match = 1;
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,9 +33,9 @@ import (
"sync"

"github.com/cockroachdb/cockroach/pkg/storage/engine"
"github.com/coreos/etcd/raft"
gwruntime "github.com/grpc-ecosystem/grpc-gateway/runtime"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,8 @@ import (
"math/rand"
"time"

"github.com/coreos/etcd/raft"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"

"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/allocator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,9 @@ import (
"testing"
"time"

"github.com/coreos/etcd/raft"
"github.com/olekukonko/tablewriter"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/below_raft_protos_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import (
"reflect"
"testing"

"github.com/coreos/etcd/raft/raftpb"
"go.etcd.io/etcd/raft/raftpb"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,9 @@ import (
"testing"
"time"

"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"
"go.etcd.io/etcd/raft/raftpb"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/gossip"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@ import (
"testing"
"time"

"github.com/coreos/etcd/raft/raftpb"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft/raftpb"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,10 +34,10 @@ import (
"time"

"github.com/cenk/backoff"
"github.com/coreos/etcd/raft"
"github.com/kr/pretty"
"github.com/pkg/errors"
circuit "github.com/rubyist/circuitbreaker"
"go.etcd.io/etcd/raft"
"google.golang.org/grpc"

"github.com/cockroachdb/cockroach/pkg/base"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/entry_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ package storage

import (
"github.com/biogo/store/llrb"
"github.com/coreos/etcd/raft/raftpb"
"go.etcd.io/etcd/raft/raftpb"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/cache"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/entry_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ import (
"reflect"
"testing"

"github.com/coreos/etcd/raft/raftpb"
"go.etcd.io/etcd/raft/raftpb"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/coreos/etcd/raft/raftpb"
"go.etcd.io/etcd/raft/raftpb"
)

var (
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,8 @@ import (

"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb"
"go.etcd.io/etcd/raft"
"go.etcd.io/etcd/raft/raftpb"
)

// init installs an adapter to use clog for log messages from raft which
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/raft.pb.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 1 addition & 1 deletion pkg/storage/raft_log_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ import (
"sort"
"time"

"github.com/coreos/etcd/raft"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"

"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/gossip"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/raft_log_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@ import (
"strings"
"testing"

"github.com/coreos/etcd/raft"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"

"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/raft_snapshot_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@ import (
"context"
"time"

"github.com/coreos/etcd/raft"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"

"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/gossip"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/raft_transport.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@ import (
"time"
"unsafe"

"github.com/coreos/etcd/raft/raftpb"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft/raftpb"
"google.golang.org/grpc"

"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/raft_transport_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@ import (
"testing"
"time"

"github.com/coreos/etcd/raft/raftpb"
"github.com/gogo/protobuf/proto"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft/raftpb"

"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,12 @@ import (
"time"
"unsafe"

"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb"
"github.com/google/btree"
"github.com/kr/pretty"
opentracing "github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"
"go.etcd.io/etcd/raft/raftpb"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,8 @@ import (
"strings"
"time"

"github.com/coreos/etcd/raft/raftpb"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft/raftpb"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/internal/client"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/replica_gc_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ import (
"math"
"time"

"github.com/coreos/etcd/raft"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"

"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/gossip"
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,10 @@ import (
"time"
"unsafe"

"github.com/coreos/etcd/raft"
"github.com/kr/pretty"
opentracing "github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"
"golang.org/x/time/rate"

"github.com/cockroachdb/cockroach/pkg/keys"
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/replica_raftstorage.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,9 @@ import (
"math"
"time"

"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"
"go.etcd.io/etcd/raft/raftpb"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down
Loading

0 comments on commit 44a5f06

Please sign in to comment.