Skip to content

Commit

Permalink
Merge pull request #7909 from heyitsanthony/unptr-cfg
Browse files Browse the repository at this point in the history
etcdserver, embed, integration: don't use pointer for ServerConfig
  • Loading branch information
heyitsanthony committed Jun 16, 2017
2 parents 44a6c21 + dcf52bb commit e962b0c
Show file tree
Hide file tree
Showing 6 changed files with 27 additions and 40 deletions.
2 changes: 1 addition & 1 deletion embed/etcd.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) {
}
}

srvcfg := &etcdserver.ServerConfig{
srvcfg := etcdserver.ServerConfig{
Name: cfg.Name,
ClientURLs: cfg.ACUrls,
PeerURLs: cfg.APUrls,
Expand Down
8 changes: 4 additions & 4 deletions etcdserver/backend.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import (
"github.com/coreos/etcd/snap"
)

func newBackend(cfg *ServerConfig) backend.Backend {
func newBackend(cfg ServerConfig) backend.Backend {
bcfg := backend.DefaultBackendConfig()
bcfg.Path = cfg.backendPath()
if cfg.QuotaBackendBytes > 0 && cfg.QuotaBackendBytes != DefaultQuotaBytes {
Expand All @@ -37,7 +37,7 @@ func newBackend(cfg *ServerConfig) backend.Backend {
}

// openSnapshotBackend renames a snapshot db to the current etcd db and opens it.
func openSnapshotBackend(cfg *ServerConfig, ss *snap.Snapshotter, snapshot raftpb.Snapshot) (backend.Backend, error) {
func openSnapshotBackend(cfg ServerConfig, ss *snap.Snapshotter, snapshot raftpb.Snapshot) (backend.Backend, error) {
snapPath, err := ss.DBFilePath(snapshot.Metadata.Index)
if err != nil {
return nil, fmt.Errorf("database snapshot file path error: %v", err)
Expand All @@ -49,7 +49,7 @@ func openSnapshotBackend(cfg *ServerConfig, ss *snap.Snapshotter, snapshot raftp
}

// openBackend returns a backend using the current etcd db.
func openBackend(cfg *ServerConfig) backend.Backend {
func openBackend(cfg ServerConfig) backend.Backend {
fn := cfg.backendPath()
beOpened := make(chan backend.Backend)
go func() {
Expand All @@ -69,7 +69,7 @@ func openBackend(cfg *ServerConfig) backend.Backend {
// before updating the backend db after persisting raft snapshot to disk,
// violating the invariant snapshot.Metadata.Index < db.consistentIndex. In this
// case, replace the db with the snapshot db sent by the leader.
func recoverSnapshotBackend(cfg *ServerConfig, oldbe backend.Backend, snapshot raftpb.Snapshot) (backend.Backend, error) {
func recoverSnapshotBackend(cfg ServerConfig, oldbe backend.Backend, snapshot raftpb.Snapshot) (backend.Backend, error) {
var cIndex consistentIndex
kv := mvcc.New(oldbe, &lease.FakeLessor{}, &cIndex)
defer kv.Close()
Expand Down
6 changes: 3 additions & 3 deletions etcdserver/raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -378,7 +378,7 @@ func advanceTicksForElection(n raft.Node, electionTicks int) {
}
}

func startNode(cfg *ServerConfig, cl *membership.RaftCluster, ids []types.ID) (id types.ID, n raft.Node, s *raft.MemoryStorage, w *wal.WAL) {
func startNode(cfg ServerConfig, cl *membership.RaftCluster, ids []types.ID) (id types.ID, n raft.Node, s *raft.MemoryStorage, w *wal.WAL) {
var err error
member := cl.MemberByName(cfg.Name)
metadata := pbutil.MustMarshal(
Expand Down Expand Up @@ -419,7 +419,7 @@ func startNode(cfg *ServerConfig, cl *membership.RaftCluster, ids []types.ID) (i
return
}

func restartNode(cfg *ServerConfig, snapshot *raftpb.Snapshot) (types.ID, *membership.RaftCluster, raft.Node, *raft.MemoryStorage, *wal.WAL) {
func restartNode(cfg ServerConfig, snapshot *raftpb.Snapshot) (types.ID, *membership.RaftCluster, raft.Node, *raft.MemoryStorage, *wal.WAL) {
var walsnap walpb.Snapshot
if snapshot != nil {
walsnap.Index, walsnap.Term = snapshot.Metadata.Index, snapshot.Metadata.Term
Expand Down Expand Up @@ -453,7 +453,7 @@ func restartNode(cfg *ServerConfig, snapshot *raftpb.Snapshot) (types.ID, *membe
return id, cl, n, s, w
}

func restartAsStandaloneNode(cfg *ServerConfig, snapshot *raftpb.Snapshot) (types.ID, *membership.RaftCluster, raft.Node, *raft.MemoryStorage, *wal.WAL) {
func restartAsStandaloneNode(cfg ServerConfig, snapshot *raftpb.Snapshot) (types.ID, *membership.RaftCluster, raft.Node, *raft.MemoryStorage, *wal.WAL) {
var walsnap walpb.Snapshot
if snapshot != nil {
walsnap.Index, walsnap.Term = snapshot.Metadata.Index, snapshot.Metadata.Term
Expand Down
15 changes: 6 additions & 9 deletions etcdserver/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -170,12 +170,10 @@ type EtcdServer struct {
// consistIndex used to hold the offset of current executing entry
// It is initialized to 0 before executing any entry.
consistIndex consistentIndex // must use atomic operations to access; keep 64-bit aligned.
Cfg *ServerConfig
r raftNode // uses 64-bit atomics; keep 64-bit aligned.

readych chan struct{}
r raftNode

snapCount uint64
Cfg ServerConfig

w wait.Wait

Expand Down Expand Up @@ -250,7 +248,7 @@ type EtcdServer struct {

// NewServer creates a new EtcdServer from the supplied configuration. The
// configuration is considered static for the lifetime of the EtcdServer.
func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
func NewServer(cfg ServerConfig) (srv *EtcdServer, err error) {
st := store.New(StoreClusterPrefix, StoreKeysPrefix)

var (
Expand Down Expand Up @@ -408,7 +406,6 @@ func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
srv = &EtcdServer{
readych: make(chan struct{}),
Cfg: cfg,
snapCount: cfg.SnapCount,
errorc: make(chan error, 1),
store: st,
snapshotter: ss,
Expand Down Expand Up @@ -530,9 +527,9 @@ func (s *EtcdServer) Start() {
// modify a server's fields after it has been sent to Start.
// This function is just used for testing.
func (s *EtcdServer) start() {
if s.snapCount == 0 {
if s.Cfg.SnapCount == 0 {
plog.Infof("set snapshot count to default %d", DefaultSnapCount)
s.snapCount = DefaultSnapCount
s.Cfg.SnapCount = DefaultSnapCount
}
s.w = wait.New()
s.applyWait = wait.NewTimeList()
Expand Down Expand Up @@ -915,7 +912,7 @@ func (s *EtcdServer) applyEntries(ep *etcdProgress, apply *apply) {
}

func (s *EtcdServer) triggerSnapshot(ep *etcdProgress) {
if ep.appliedi-ep.snapi <= s.snapCount {
if ep.appliedi-ep.snapi <= s.Cfg.SnapCount {
return
}

Expand Down
34 changes: 12 additions & 22 deletions etcdserver/server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,6 @@ func TestApplyRepeat(t *testing.T) {
})
s := &EtcdServer{
r: *r,
Cfg: &ServerConfig{},
store: st,
cluster: cl,
reqIDGen: idutil.NewGenerator(0, time.Time{}),
Expand Down Expand Up @@ -530,7 +529,6 @@ func TestApplyConfChangeError(t *testing.T) {
srv := &EtcdServer{
r: *newRaftNode(raftNodeConfig{Node: n}),
cluster: cl,
Cfg: &ServerConfig{},
}
_, err := srv.applyConfChange(tt.cc, nil)
if err != tt.werr {
Expand Down Expand Up @@ -685,7 +683,7 @@ func TestDoProposal(t *testing.T) {
transport: rafthttp.NewNopTransporter(),
})
srv := &EtcdServer{
Cfg: &ServerConfig{TickMs: 1},
Cfg: ServerConfig{TickMs: 1},
r: *r,
store: st,
reqIDGen: idutil.NewGenerator(0, time.Time{}),
Expand Down Expand Up @@ -713,7 +711,7 @@ func TestDoProposal(t *testing.T) {
func TestDoProposalCancelled(t *testing.T) {
wt := mockwait.NewRecorder()
srv := &EtcdServer{
Cfg: &ServerConfig{TickMs: 1},
Cfg: ServerConfig{TickMs: 1},
r: *newRaftNode(raftNodeConfig{Node: newNodeNop()}),
w: wt,
reqIDGen: idutil.NewGenerator(0, time.Time{}),
Expand All @@ -735,7 +733,7 @@ func TestDoProposalCancelled(t *testing.T) {

func TestDoProposalTimeout(t *testing.T) {
srv := &EtcdServer{
Cfg: &ServerConfig{TickMs: 1},
Cfg: ServerConfig{TickMs: 1},
r: *newRaftNode(raftNodeConfig{Node: newNodeNop()}),
w: mockwait.NewNop(),
reqIDGen: idutil.NewGenerator(0, time.Time{}),
Expand All @@ -751,7 +749,7 @@ func TestDoProposalTimeout(t *testing.T) {

func TestDoProposalStopped(t *testing.T) {
srv := &EtcdServer{
Cfg: &ServerConfig{TickMs: 1},
Cfg: ServerConfig{TickMs: 1},
r: *newRaftNode(raftNodeConfig{Node: newNodeNop()}),
w: mockwait.NewNop(),
reqIDGen: idutil.NewGenerator(0, time.Time{}),
Expand Down Expand Up @@ -855,7 +853,7 @@ func TestSyncTrigger(t *testing.T) {
})

srv := &EtcdServer{
Cfg: &ServerConfig{TickMs: 1},
Cfg: ServerConfig{TickMs: 1},
r: *r,
store: mockstore.NewNop(),
SyncTicker: tk,
Expand Down Expand Up @@ -913,7 +911,6 @@ func TestSnapshot(t *testing.T) {
storage: p,
})
srv := &EtcdServer{
Cfg: &ServerConfig{},
r: *r,
store: st,
}
Expand Down Expand Up @@ -984,9 +981,7 @@ func TestSnapshotOrdering(t *testing.T) {
raftStorage: rs,
})
s := &EtcdServer{
Cfg: &ServerConfig{
DataDir: testdir,
},
Cfg: ServerConfig{DataDir: testdir},
r: *r,
store: st,
snapshotter: snap.New(snapdir),
Expand Down Expand Up @@ -1047,8 +1042,7 @@ func TestTriggerSnap(t *testing.T) {
transport: rafthttp.NewNopTransporter(),
})
srv := &EtcdServer{
Cfg: &ServerConfig{TickMs: 1},
snapCount: uint64(snapc),
Cfg: ServerConfig{TickMs: 1, SnapCount: uint64(snapc)},
r: *r,
store: st,
reqIDGen: idutil.NewGenerator(0, time.Time{}),
Expand Down Expand Up @@ -1112,9 +1106,7 @@ func TestConcurrentApplyAndSnapshotV3(t *testing.T) {
raftStorage: rs,
})
s := &EtcdServer{
Cfg: &ServerConfig{
DataDir: testdir,
},
Cfg: ServerConfig{DataDir: testdir},
r: *r,
store: st,
snapshotter: snap.New(testdir),
Expand Down Expand Up @@ -1199,7 +1191,6 @@ func TestAddMember(t *testing.T) {
})
s := &EtcdServer{
r: *r,
Cfg: &ServerConfig{},
store: st,
cluster: cl,
reqIDGen: idutil.NewGenerator(0, time.Time{}),
Expand Down Expand Up @@ -1241,7 +1232,6 @@ func TestRemoveMember(t *testing.T) {
})
s := &EtcdServer{
r: *r,
Cfg: &ServerConfig{},
store: st,
cluster: cl,
reqIDGen: idutil.NewGenerator(0, time.Time{}),
Expand Down Expand Up @@ -1316,7 +1306,7 @@ func TestPublish(t *testing.T) {
ctx, cancel := context.WithCancel(context.TODO())
srv := &EtcdServer{
readych: make(chan struct{}),
Cfg: &ServerConfig{TickMs: 1},
Cfg: ServerConfig{TickMs: 1},
id: 1,
r: *newRaftNode(raftNodeConfig{Node: n}),
attributes: membership.Attributes{Name: "node1", ClientURLs: []string{"http://a", "http://b"}},
Expand Down Expand Up @@ -1366,7 +1356,7 @@ func TestPublishStopped(t *testing.T) {
transport: rafthttp.NewNopTransporter(),
})
srv := &EtcdServer{
Cfg: &ServerConfig{TickMs: 1},
Cfg: ServerConfig{TickMs: 1},
r: *r,
cluster: &membership.RaftCluster{},
w: mockwait.NewNop(),
Expand All @@ -1388,7 +1378,7 @@ func TestPublishRetry(t *testing.T) {
ctx, cancel := context.WithCancel(context.TODO())
n := newNodeRecorderStream()
srv := &EtcdServer{
Cfg: &ServerConfig{TickMs: 1},
Cfg: ServerConfig{TickMs: 1},
r: *newRaftNode(raftNodeConfig{Node: n}),
w: mockwait.NewNop(),
stopping: make(chan struct{}),
Expand Down Expand Up @@ -1429,7 +1419,7 @@ func TestUpdateVersion(t *testing.T) {
ctx, cancel := context.WithCancel(context.TODO())
srv := &EtcdServer{
id: 1,
Cfg: &ServerConfig{TickMs: 1},
Cfg: ServerConfig{TickMs: 1},
r: *newRaftNode(raftNodeConfig{Node: n}),
attributes: membership.Attributes{Name: "node1", ClientURLs: []string{"http://node1.com"}},
cluster: &membership.RaftCluster{},
Expand Down
2 changes: 1 addition & 1 deletion integration/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -640,7 +640,7 @@ func (m *member) Clone(t *testing.T) *member {
func (m *member) Launch() error {
plog.Printf("launching %s (%s)", m.Name, m.grpcAddr)
var err error
if m.s, err = etcdserver.NewServer(&m.ServerConfig); err != nil {
if m.s, err = etcdserver.NewServer(m.ServerConfig); err != nil {
return fmt.Errorf("failed to initialize the etcd server: %v", err)
}
m.s.SyncTicker = time.NewTicker(500 * time.Millisecond)
Expand Down

0 comments on commit e962b0c

Please sign in to comment.