Skip to content
This repository has been archived by the owner on Sep 6, 2018. It is now read-only.

Commit

Permalink
fix issues, further cleaning is needed
Browse files Browse the repository at this point in the history
  • Loading branch information
xiang90 committed Jun 24, 2013
1 parent d8884cb commit 2bc3845
Show file tree
Hide file tree
Showing 7 changed files with 74 additions and 75 deletions.
3 changes: 0 additions & 3 deletions command.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,6 @@ func init() {
type Command interface {
CommandName() string
Apply(server *Server) ([]byte, error)
Join() ([]byte, error)
Init()
Finish()
}

//------------------------------------------------------------------------------
Expand Down
3 changes: 2 additions & 1 deletion log.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
// A log is a collection of log entries that are persisted to durable storage.
type Log struct {
ApplyFunc func(Command) ([]byte, error)
callBackFrom uint64
file *os.File
path string
entries []*LogEntry
Expand Down Expand Up @@ -339,7 +340,7 @@ func (l *Log) SetCommitIndex(index uint64) error {

// Apply the changes to the state machine and store the error code.
entry.result, l.errors[entryIndex] = l.ApplyFunc(entry.Command)

}
return nil
}
Expand Down
4 changes: 3 additions & 1 deletion log_entry.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,8 @@ type LogEntry struct {
Index uint64 `json:"index"`
Term uint64 `json:"term"`
Command Command `json:"command"`
result []byte `json:-`
result []byte `json:"-"`
commit chan bool `json:"-"`
}

// A temporary interface used for unmarshaling log entries.
Expand All @@ -46,6 +47,7 @@ func NewLogEntry(log *Log, index uint64, term uint64, command Command) *LogEntry
Index: index,
Term: term,
Command: command,
commit: make(chan bool, 3),
}
}

Expand Down
37 changes: 0 additions & 37 deletions log_entry_test.go

This file was deleted.

3 changes: 0 additions & 3 deletions peer.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@ type Peer struct {
prevLogIndex uint64
mutex sync.Mutex
heartbeatTimer *Timer
// Collecting Info
collecting int
}

type FlushResponse struct {
Expand All @@ -43,7 +41,6 @@ func NewPeer(server *Server, name string, heartbeatTimeout time.Duration) *Peer
server: server,
name: name,
heartbeatTimer: NewTimer(heartbeatTimeout, heartbeatTimeout),
collecting: -1,
}

// Start the heartbeat timeout and wait for the goroutine to start.
Expand Down
95 changes: 67 additions & 28 deletions server.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,8 +55,6 @@ type Server struct {
context interface{}
currentTerm uint64

startIndex uint64

votedFor string
log *Log
leader string
Expand Down Expand Up @@ -312,7 +310,7 @@ func (s *Server) Initialize() error {
s.currentTerm = s.log.CurrentTerm()

// update the startIndex
s.startIndex = uint64(len(s.log.entries)) + s.log.startIndex
s.log.callBackFrom = uint64(len(s.log.entries)) + s.log.startIndex

for _, peer := range s.peers {
peer.pause()
Expand Down Expand Up @@ -340,9 +338,22 @@ func (s *Server) commitCenter() {

if s.QuorumSize() < 2 {
fmt.Println("[CommitCenter] Commit ", s.log.CurrentIndex())
s.log.SetCommitIndex(s.log.CurrentIndex())
entry := s.log.entries[int(s.log.CurrentIndex()) - 1- int(s.log.startIndex)]
entry.Command.Finish()

commited := int(s.log.CommitIndex())
commit := int(s.log.CurrentIndex())

s.log.SetCommitIndex(uint64(commit))

for i := commited; i < commit; i++ {
select {
case s.log.entries[i - int(s.log.startIndex)].commit <- true:
fmt.Println("notify")
continue
default:
continue
}
}

continue
}

Expand All @@ -356,14 +367,25 @@ func (s *Server) commitCenter() {
sort.Ints(data)
commit := data[s.QuorumSize() - 1]

if commit > int(s.log.CommitIndex()) {
commited := int(s.log.CommitIndex())

if commit > commited {
fmt.Println("[CommitCenter] Going to Commit ", commit)
s.log.SetCommitIndex(uint64(commit))
entry := s.log.entries[commit - 1- int(s.log.startIndex)]

if (commit > int(s.startIndex)) {
fmt.Println("[CommitCenter] Wait join Commit ", entry.Index)
entry.Command.Finish()
// entry := s.log.entries[commit - 1- int(s.log.startIndex)]

// if (commit > int(s.startIndex)) {
// fmt.Println("[CommitCenter] Wait join Commit ", entry.Index)
// }

for i := commited; i < commit; i++ {
select {
case s.log.entries[i - int(s.log.startIndex)].commit <- true:
fmt.Println("notify")
continue
default:
continue
}
}

fmt.Println("[CommitCenter] Commit ", commit)
Expand Down Expand Up @@ -455,34 +477,34 @@ func (s *Server) StartLeader() error {

// Attempts to execute a command and replicate it. The function will return
// when the command has been successfully committed or an error has occurred.
// func (s *Server) Do(command Command) ([]byte, error) {
// s.mutex.Lock()
// defer s.mutex.Unlock()
// result, err := s.do(command)
// return result, err
// }

// This function is the low-level interface to execute commands. This function
// does not obtain a lock so one must be obtained before executing.

func (s *Server) Do(command Command) ([]byte, error) {
if s.state != Leader {
return nil, NotLeaderError
}

command.Init()

entry := s.log.CreateEntry(s.currentTerm, command)
if err := s.log.AppendEntry(entry); err != nil {
return nil, err
}

s.response <- FlushResponse{s.currentTerm, true, nil, nil}
fmt.Println("[Do] fire!")

// to speed up the response time
for _, peer := range s.peers {
peer.heartbeatTimer.fire()
}
fmt.Println("[Do] join!")
return command.Join()

// timeout here
select {
case <-entry.commit:
fmt.Println("[Do] finish!")
return entry.result, nil
case <-time.After(time.Second):
fmt.Println("[Do] fail!")
return nil, errors.New("Command commit fails")
}
}

// Appends a log entry from the leader to this server.
Expand Down Expand Up @@ -518,16 +540,23 @@ func (s *Server) AppendEntries(req *AppendEntriesRequest) (*AppendEntriesRespons
return NewAppendEntriesResponse(s.currentTerm, false, s.log.CommitIndex()), err
}

fmt.Println("Peer ", s.Name(), "after truncate ")

// Append entries to the log.
if err := s.log.AppendEntries(req.Entries); err != nil {
return NewAppendEntriesResponse(s.currentTerm, false, s.log.CommitIndex()), err
}

fmt.Println("Peer ", s.Name(), "after append ")
// Commit up to the commit index.
if err := s.log.SetCommitIndex(req.CommitIndex); err != nil {
return NewAppendEntriesResponse(s.currentTerm, false, s.log.CommitIndex()), err
}

fmt.Println("Peer ", s.Name(), "after commit ")

fmt.Println("Peer ", s.Name(), "reply heartbeat from ", req.LeaderName,
" ",req.Term," ", s.currentTerm, " ",time.Now())
return NewAppendEntriesResponse(s.currentTerm, true, s.log.CommitIndex()), nil
}

Expand Down Expand Up @@ -582,16 +611,24 @@ func (s *Server) promote() (bool, error) {

// Collect votes until we have a quorum.
votes := map[string]bool{}

elected := false
loop:
timeout := false

for {
// if timeout happened, restart the promotion
if timeout {
break
}

// Add up all our votes.
votesGranted := 1
for _, value := range votes {
if value {
votesGranted++
}
}

// If we received enough votes then stop waiting for more votes.
if votesGranted >= s.QuorumSize() {
elected = true
Expand All @@ -615,7 +652,7 @@ func (s *Server) promote() (bool, error) {
votes[resp.peer.Name()] = resp.VoteGranted
}
case <-afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2):
break loop
timeout = true
}
}

Expand Down Expand Up @@ -652,8 +689,10 @@ func (s *Server) promoteToCandidate() (uint64, uint64, uint64, error) {
s.currentTerm++
s.votedFor = s.name
s.leader = ""

// Pause the election timer while we're a candidate.
s.electionTimer.Pause()

// Return server state so we can check for it during leader promotion.
lastLogIndex, lastLogTerm := s.log.LastInfo()

Expand Down Expand Up @@ -686,7 +725,7 @@ func (s *Server) promoteToLeader(term uint64, lastLogIndex uint64, lastLogTerm u
// Move server to become a leader and begin peer heartbeats.
s.state = Leader
s.leader = s.name
s.startIndex = lastLogIndex
s.log.callBackFrom = lastLogIndex

for _, peer := range s.peers {
// start from lastLogIndex
Expand Down
4 changes: 2 additions & 2 deletions test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,8 +9,8 @@ import (
)

const (
testHeartbeatTimeout = 30 * time.Millisecond
testElectionTimeout = 80 * time.Millisecond
testHeartbeatTimeout = 10 * time.Millisecond
testElectionTimeout = 50 * time.Millisecond
)

func init() {
Expand Down

0 comments on commit 2bc3845

Please sign in to comment.