Skip to content

Commit

Permalink
Merge pull request #145 from goraft/remove_timeout
Browse files Browse the repository at this point in the history
remove command timeout
  • Loading branch information
benbjohnson committed Jan 2, 2014
2 parents d96553b + 42e4b8d commit c064081
Show file tree
Hide file tree
Showing 7 changed files with 59 additions and 110 deletions.
2 changes: 1 addition & 1 deletion append_entries_request_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ func createTestAppendEntriesRequest(entryCount int) (*AppendEntriesRequest, []by
entries := make([]*LogEntry, 0)
for i := 0; i < entryCount; i++ {
command := &DefaultJoinCommand{Name: "localhost:1000"}
entry, _ := newLogEntry(nil, 1, 2, command)
entry, _ := newLogEntry(nil, nil, 1, 2, command)
entries = append(entries, entry)
}
req := newAppendEntriesRequest(1, 1, 1, 1, "leader", entries)
Expand Down
67 changes: 24 additions & 43 deletions log.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ type Log struct {
file *os.File
path string
entries []*LogEntry
results []*logResult
commitIndex uint64
mutex sync.RWMutex
startIndex uint64 // the index before the first entry in the Log entries
Expand Down Expand Up @@ -162,7 +161,7 @@ func (l *Log) open(path string) error {
// Read the file and decode entries.
for {
// Instantiate log entry and decode into it.
entry, _ := newLogEntry(l, 0, 0, nil)
entry, _ := newLogEntry(l, nil, 0, 0, nil)
entry.Position, _ = l.file.Seek(0, os.SEEK_CUR)

n, err := entry.decode(l.file)
Expand Down Expand Up @@ -191,8 +190,6 @@ func (l *Log) open(path string) error {

readBytes += int64(n)
}
l.results = make([]*logResult, len(l.entries))

debugln("open.log.recovery number of log ", len(l.entries))
return nil
}
Expand All @@ -207,16 +204,15 @@ func (l *Log) close() {
l.file = nil
}
l.entries = make([]*LogEntry, 0)
l.results = make([]*logResult, 0)
}

//--------------------------------------
// Entries
//--------------------------------------

// Creates a log entry associated with this log.
func (l *Log) createEntry(term uint64, command Command) (*LogEntry, error) {
return newLogEntry(l, l.nextIndex(), term, command)
func (l *Log) createEntry(term uint64, command Command, e *ev) (*LogEntry, error) {
return newLogEntry(l, e, l.nextIndex(), term, command)
}

// Retrieves an entry from the log. If the entry has been eliminated because
Expand Down Expand Up @@ -276,35 +272,6 @@ func (l *Log) getEntriesAfter(index uint64, maxLogEntriesPerRequest uint64) ([]*
}
}

// Retrieves the return value and error for an entry. The result can only exist
// after the entry has been committed.
func (l *Log) getEntryResult(entry *LogEntry, clear bool) (interface{}, error) {
l.mutex.RLock()
defer l.mutex.RUnlock()

if entry == nil {
panic("raft: Log entry required for error retrieval")
}
debugln("getEntryResult.result index: ", entry.Index-l.startIndex-1)
// If a result exists for the entry then return it with its error.
if entry.Index > l.startIndex && entry.Index <= l.startIndex+uint64(len(l.results)) {
if result := l.results[entry.Index-l.startIndex-1]; result != nil {

// keep the records before remove it
returnValue, err := result.returnValue, result.err

// Remove reference to result if it's being cleared after retrieval.
if clear {
result.returnValue = nil
}

return returnValue, err
}
}

return nil, nil
}

//--------------------------------------
// Commit
//--------------------------------------
Expand Down Expand Up @@ -402,7 +369,10 @@ func (l *Log) setCommitIndex(index uint64) error {
// Apply the changes to the state machine and store the error code.
returnValue, err := l.ApplyFunc(command)
debugln("setCommitIndex.set.result index: ", entryIndex)
l.results[entryIndex] = &logResult{returnValue: returnValue, err: err}
if entry.event != nil {
entry.event.returnValue = returnValue
entry.event.c <- err
}
}
return nil
}
Expand Down Expand Up @@ -443,6 +413,14 @@ func (l *Log) truncate(index uint64, term uint64) error {
debugln("log.truncate.clear")
l.file.Truncate(0)
l.file.Seek(0, os.SEEK_SET)

// notify clients if this node is the previous leader
for _, entry := range l.entries {
if entry.event != nil {
entry.event.c <- errors.New("command failed to be committed due to node failure")
}
}

l.entries = []*LogEntry{}
} else {
// Do not truncate if the entry at index does not have the matching term.
Expand All @@ -458,6 +436,15 @@ func (l *Log) truncate(index uint64, term uint64) error {
position := l.entries[index-l.startIndex].Position
l.file.Truncate(position)
l.file.Seek(position, os.SEEK_SET)

// notify clients if this node is the previous leader
for i := index - l.startIndex; i < uint64(len(l.entries)); i++ {
entry := l.entries[i]
if entry.event != nil {
entry.event.c <- errors.New("command failed to be committed due to node failure")
}
}

l.entries = l.entries[0 : index-l.startIndex]
}
}
Expand Down Expand Up @@ -529,7 +516,6 @@ func (l *Log) appendEntry(entry *LogEntry) error {

// Append to entries list if stored on disk.
l.entries = append(l.entries, entry)
l.results = append(l.results, nil)

return nil
}
Expand Down Expand Up @@ -558,7 +544,6 @@ func (l *Log) writeEntry(entry *LogEntry, w io.Writer) (int64, error) {

// Append to entries list if stored on disk.
l.entries = append(l.entries, entry)
l.results = append(l.results, nil)

return int64(size), nil
}
Expand All @@ -570,7 +555,6 @@ func (l *Log) writeEntry(entry *LogEntry, w io.Writer) (int64, error) {
// compact the log before index (including index)
func (l *Log) compact(index uint64, term uint64) error {
var entries []*LogEntry
var results []*logResult

l.mutex.Lock()
defer l.mutex.Unlock()
Expand All @@ -583,11 +567,9 @@ func (l *Log) compact(index uint64, term uint64) error {
// we just recovery from on snapshot
if index >= l.internalCurrentIndex() {
entries = make([]*LogEntry, 0)
results = make([]*logResult, 0)
} else {
// get all log entries after index
entries = l.entries[index-l.startIndex:]
results = l.results[index-l.startIndex:]
}

// create a new log file and add all the entries
Expand Down Expand Up @@ -621,7 +603,6 @@ func (l *Log) compact(index uint64, term uint64) error {

// compaction the in memory log
l.entries = entries
l.results = results
l.startIndex = index
l.startTerm = term
return nil
Expand Down
6 changes: 3 additions & 3 deletions log_entry.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,11 @@ type LogEntry struct {
CommandName string
Command []byte
Position int64 // position in the log file
commit chan bool
event *ev
}

// Creates a new log entry associated with a log.
func newLogEntry(log *Log, index uint64, term uint64, command Command) (*LogEntry, error) {
func newLogEntry(log *Log, event *ev, index uint64, term uint64, command Command) (*LogEntry, error) {
var buf bytes.Buffer
var commandName string
if command != nil {
Expand All @@ -41,7 +41,7 @@ func newLogEntry(log *Log, index uint64, term uint64, command Command) (*LogEntr
Term: term,
CommandName: commandName,
Command: buf.Bytes(),
commit: make(chan bool, 5),
event: event,
}

return e, nil
Expand Down
30 changes: 15 additions & 15 deletions log_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,15 +30,15 @@ func TestLogNewLog(t *testing.T) {
defer log.close()
defer os.Remove(path)

e, _ := newLogEntry(log, 1, 1, &testCommand1{Val: "foo", I: 20})
e, _ := newLogEntry(log, nil, 1, 1, &testCommand1{Val: "foo", I: 20})
if err := log.appendEntry(e); err != nil {
t.Fatalf("Unable to append: %v", err)
}
e, _ = newLogEntry(log, 2, 1, &testCommand2{X: 100})
e, _ = newLogEntry(log, nil, 2, 1, &testCommand2{X: 100})
if err := log.appendEntry(e); err != nil {
t.Fatalf("Unable to append: %v", err)
}
e, _ = newLogEntry(log, 3, 2, &testCommand1{Val: "bar", I: 0})
e, _ = newLogEntry(log, nil, 3, 2, &testCommand1{Val: "bar", I: 0})
if err := log.appendEntry(e); err != nil {
t.Fatalf("Unable to append: %v", err)
}
Expand All @@ -63,9 +63,9 @@ func TestLogNewLog(t *testing.T) {
// Ensure that we can decode and encode to an existing log.
func TestLogExistingLog(t *testing.T) {
tmpLog := newLog()
e0, _ := newLogEntry(tmpLog, 1, 1, &testCommand1{Val: "foo", I: 20})
e1, _ := newLogEntry(tmpLog, 2, 1, &testCommand2{X: 100})
e2, _ := newLogEntry(tmpLog, 3, 2, &testCommand1{Val: "bar", I: 0})
e0, _ := newLogEntry(tmpLog, nil, 1, 1, &testCommand1{Val: "foo", I: 20})
e1, _ := newLogEntry(tmpLog, nil, 2, 1, &testCommand2{X: 100})
e2, _ := newLogEntry(tmpLog, nil, 3, 2, &testCommand1{Val: "bar", I: 0})
log, path := setupLog([]*LogEntry{e0, e1, e2})
defer log.close()
defer os.Remove(path)
Expand All @@ -88,9 +88,9 @@ func TestLogExistingLog(t *testing.T) {
// Ensure that we can check the contents of the log by index/term.
func TestLogContainsEntries(t *testing.T) {
tmpLog := newLog()
e0, _ := newLogEntry(tmpLog, 1, 1, &testCommand1{Val: "foo", I: 20})
e1, _ := newLogEntry(tmpLog, 2, 1, &testCommand2{X: 100})
e2, _ := newLogEntry(tmpLog, 3, 2, &testCommand1{Val: "bar", I: 0})
e0, _ := newLogEntry(tmpLog, nil, 1, 1, &testCommand1{Val: "foo", I: 20})
e1, _ := newLogEntry(tmpLog, nil, 2, 1, &testCommand2{X: 100})
e2, _ := newLogEntry(tmpLog, nil, 3, 2, &testCommand1{Val: "bar", I: 0})
log, path := setupLog([]*LogEntry{e0, e1, e2})
defer log.close()
defer os.Remove(path)
Expand All @@ -115,8 +115,8 @@ func TestLogContainsEntries(t *testing.T) {
// Ensure that we can recover from an incomplete/corrupt log and continue logging.
func TestLogRecovery(t *testing.T) {
tmpLog := newLog()
e0, _ := newLogEntry(tmpLog, 1, 1, &testCommand1{Val: "foo", I: 20})
e1, _ := newLogEntry(tmpLog, 2, 1, &testCommand2{X: 100})
e0, _ := newLogEntry(tmpLog, nil, 1, 1, &testCommand1{Val: "foo", I: 20})
e1, _ := newLogEntry(tmpLog, nil, 2, 1, &testCommand2{X: 100})
f, _ := ioutil.TempFile("", "raft-log-")

e0.encode(f)
Expand All @@ -134,7 +134,7 @@ func TestLogRecovery(t *testing.T) {
defer log.close()
defer os.Remove(f.Name())

e, _ := newLogEntry(log, 3, 2, &testCommand1{Val: "bat", I: -5})
e, _ := newLogEntry(log, nil, 3, 2, &testCommand1{Val: "bat", I: -5})
if err := log.appendEntry(e); err != nil {
t.Fatalf("Unable to append: %v", err)
}
Expand Down Expand Up @@ -167,15 +167,15 @@ func TestLogTruncate(t *testing.T) {

defer os.Remove(path)

entry1, _ := newLogEntry(log, 1, 1, &testCommand1{Val: "foo", I: 20})
entry1, _ := newLogEntry(log, nil, 1, 1, &testCommand1{Val: "foo", I: 20})
if err := log.appendEntry(entry1); err != nil {
t.Fatalf("Unable to append: %v", err)
}
entry2, _ := newLogEntry(log, 2, 1, &testCommand2{X: 100})
entry2, _ := newLogEntry(log, nil, 2, 1, &testCommand2{X: 100})
if err := log.appendEntry(entry2); err != nil {
t.Fatalf("Unable to append: %v", err)
}
entry3, _ := newLogEntry(log, 3, 2, &testCommand1{Val: "bar", I: 0})
entry3, _ := newLogEntry(log, nil, 3, 2, &testCommand1{Val: "bar", I: 0})
if err := log.appendEntry(entry3); err != nil {
t.Fatalf("Unable to append: %v", err)
}
Expand Down
34 changes: 1 addition & 33 deletions server.go
Original file line number Diff line number Diff line change
Expand Up @@ -837,7 +837,7 @@ func (s *server) processCommand(command Command, e *ev) {
s.debugln("server.command.process")

// Create an entry for the command in the log.
entry, err := s.log.createEntry(s.currentTerm, command)
entry, err := s.log.createEntry(s.currentTerm, command, e)

if err != nil {
s.debugln("server.command.log.entry.error:", err)
Expand All @@ -851,23 +851,6 @@ func (s *server) processCommand(command Command, e *ev) {
return
}

// Issue a callback for the entry once it's committed.
go func() {
// Wait for the entry to be committed.
select {
case <-entry.commit:
var err error
s.debugln("server.command.commit")
e.returnValue, err = s.log.getEntryResult(entry, true)
e.c <- err
case <-time.After(time.Second):
s.debugln("server.command.timeout")
e.c <- CommandTimeoutError
}

entry.commit = nil
}()

// Issue an append entries response for the server.
resp := newAppendEntriesResponse(s.currentTerm, true, s.log.currentIndex(), s.log.CommitIndex())
resp.append = true
Expand Down Expand Up @@ -972,21 +955,6 @@ func (s *server) processAppendEntriesResponse(resp *AppendEntriesResponse) {
if commitIndex > committedIndex {
s.log.setCommitIndex(commitIndex)
s.debugln("commit index ", commitIndex)
for i := committedIndex; i < commitIndex; i++ {
if entry := s.log.getEntry(i + 1); entry != nil {
// if the leader is a new one and the entry came from the
// old leader, the commit channel will be nil and no go routine
// is waiting from this channel
// if we try to send to it, the new leader will get stuck
if entry.commit != nil {
select {
case entry.commit <- true:
default:
panic("server unable to send signal to commit channel")
}
}
}
}
}
}

Expand Down
Loading

0 comments on commit c064081

Please sign in to comment.