Browse Source

etcdserver: use ReqTimeout only

We cannot refer RTT value from heartbeat interval, so CommitTimeout
is invalid. Remove it and use ReqTimeout instead.
Yicheng Qin 10 years ago
parent
commit
2d5b95c49f
3 changed files with 5 additions and 13 deletions
  1. 2 10
      etcdserver/config.go
  2. 1 1
      etcdserver/raft.go
  3. 2 2
      etcdserver/server.go

+ 2 - 10
etcdserver/config.go

@@ -113,17 +113,9 @@ func (c *ServerConfig) ShouldDiscover() bool { return c.DiscoveryURL != "" }
 
 // ReqTimeout returns timeout for request to finish.
 func (c *ServerConfig) ReqTimeout() time.Duration {
-	// CommitTimeout
-	// + 2 * election timeout for possible leader election
-	return c.CommitTimeout() + 2*time.Duration(c.ElectionTicks)*time.Duration(c.TickMs)*time.Millisecond
-}
-
-// CommitTimeout returns commit timeout under normal case.
-func (c *ServerConfig) CommitTimeout() time.Duration {
-	// We assume that heartbeat >= TTL.
 	// 5s for queue waiting, computation and disk IO delay
-	// + 2 * heartbeat(TTL) for expected time between proposal by follower and commit at the follower
-	return 5*time.Second + 2*time.Duration(c.TickMs)*time.Millisecond
+	// + 2 * election timeout for possible leader election
+	return 5*time.Second + 2*time.Duration(c.ElectionTicks)*time.Duration(c.TickMs)*time.Millisecond
 }
 
 func (c *ServerConfig) PrintWithInitial() { c.print(true) }

+ 1 - 1
etcdserver/raft.go

@@ -185,7 +185,7 @@ func (r *raftNode) start(s *EtcdServer) {
 				}
 				r.Advance()
 			case <-syncC:
-				r.s.sync(r.s.cfg.CommitTimeout())
+				r.s.sync(r.s.cfg.ReqTimeout())
 			case <-r.stopped:
 				return
 			}

+ 2 - 2
etcdserver/server.go

@@ -343,7 +343,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
 // It also starts a goroutine to publish its server information.
 func (s *EtcdServer) Start() {
 	s.start()
-	go s.publish(s.cfg.CommitTimeout())
+	go s.publish(s.cfg.ReqTimeout())
 	go s.purgeFile()
 	go monitorFileDescriptor(s.done)
 	go s.monitorVersions()
@@ -1000,7 +1000,7 @@ func (s *EtcdServer) updateClusterVersion(ver string) {
 		Path:   path.Join(StoreClusterPrefix, "version"),
 		Val:    ver,
 	}
-	ctx, cancel := context.WithTimeout(context.Background(), s.cfg.CommitTimeout())
+	ctx, cancel := context.WithTimeout(context.Background(), s.cfg.ReqTimeout())
 	_, err := s.Do(ctx, req)
 	cancel()
 	switch err {