6260: Fix races in keepstore tests. Expose WorkQueue in-progress/queued stats.
authorTom Clegg <tom@curoverse.com>
Thu, 6 Aug 2015 03:38:05 +0000 (23:38 -0400)
committerTom Clegg <tom@curoverse.com>
Mon, 10 Aug 2015 16:44:52 +0000 (12:44 -0400)
services/keepstore/bufferpool_test.go
services/keepstore/pull_worker.go
services/keepstore/pull_worker_test.go
services/keepstore/trash_worker.go
services/keepstore/work_queue.go
services/keepstore/work_queue_test.go

index 718e2cac6092fe4400e4d370945e17d603010dac..95d118e221de6b8516654f8a133b871da00c5cd2 100644 (file)
@@ -21,6 +21,11 @@ func init() {
        bufs = newBufferPool(maxBuffers, BLOCKSIZE)
 }
 
+// Restore sane default after bufferpool's own tests
+func (s *BufferPoolSuite) TearDownTest(c *C) {
+       bufs = newBufferPool(maxBuffers, BLOCKSIZE)
+}
+
 func (s *BufferPoolSuite) TestBufferPoolBufSize(c *C) {
        bufs := newBufferPool(2, 10)
        b1 := bufs.Get(1)
index 3d67cf2c1ec0336e73e4e9e7cca26bf4e758b544..d55fd3218535735ded387d43a752ec8c4a56215b 100644 (file)
@@ -24,6 +24,7 @@ func RunPullWorker(pullq *WorkQueue, keepClient *keepclient.KeepClient) {
        for item := range nextItem {
                pullRequest := item.(PullRequest)
                err := PullItemAndProcess(item.(PullRequest), GenerateRandomApiToken(), keepClient)
+               pullq.ReportDone <- struct{}{}
                if err == nil {
                        log.Printf("Pull %s success", pullRequest)
                } else {
index 0833bc696763a867bfab287a75805b57ce933ae6..822d202bd07dea4a4ebb50bb18b91dcda5682a91 100644 (file)
@@ -9,6 +9,7 @@ import (
        "io"
        "net/http"
        "testing"
+       "time"
 )
 
 type PullWorkerTestSuite struct{}
@@ -22,7 +23,6 @@ func TestPullWorker(t *testing.T) {
 var _ = Suite(&PullWorkerTestSuite{})
 
 var testPullLists map[string]string
-var processedPullLists map[string]string
 var readContent string
 var readError error
 var putContent []byte
@@ -39,7 +39,6 @@ func (s *PullWorkerTestSuite) SetUpTest(c *C) {
        // This behavior is verified using these two maps in the
        // "TestPullWorker_pull_list_with_two_items_latest_replacing_old"
        testPullLists = make(map[string]string)
-       processedPullLists = make(map[string]string)
 }
 
 // Since keepstore does not come into picture in tests,
@@ -238,15 +237,18 @@ func (s *PullWorkerTestSuite) TestPullWorker_invalid_data_manager_token(c *C) {
 
 func performTest(testData PullWorkerTestData, c *C) {
        RunTestPullWorker(c)
+       defer pullq.Close()
 
        currentTestData = testData
        testPullLists[testData.name] = testData.response_body
 
-       // Override GetContent to mock keepclient Get functionality
-       defer func(orig func(string, *keepclient.KeepClient)(io.ReadCloser, int64, string, error)) { GetContent = orig }(GetContent)
-       GetContent = func(signedLocator string, keepClient *keepclient.KeepClient) (
-               reader io.ReadCloser, contentLength int64, url string, err error) {
+       processedPullLists := make(map[string]string)
 
+       // Override GetContent to mock keepclient Get functionality
+       defer func(orig func(string, *keepclient.KeepClient)(io.ReadCloser, int64, string, error)) {
+               GetContent = orig
+       }(GetContent)
+       GetContent = func(signedLocator string, keepClient *keepclient.KeepClient) (reader io.ReadCloser, contentLength int64, url string, err error) {
                processedPullLists[testData.name] = testData.response_body
                if testData.read_error {
                        err = errors.New("Error getting data")
@@ -278,9 +280,7 @@ func performTest(testData PullWorkerTestData, c *C) {
        c.Assert(response.Code, Equals, testData.response_code)
        c.Assert(response.Body.String(), Equals, testData.response_body)
 
-       expectWorkerChannelEmpty(c, pullq.NextItem)
-
-       pullq.Close()
+       expectEqualWithin(c, time.Second, 0, func() interface{} { return pullq.CountOutstanding() })
 
        if testData.name == "TestPullWorker_pull_list_with_two_items_latest_replacing_old" {
                c.Assert(len(testPullLists), Equals, 2)
@@ -311,6 +311,8 @@ func performTest(testData PullWorkerTestData, c *C) {
                        c.Assert(string(putContent), Equals, testData.read_content)
                }
        }
+
+       expectChannelEmpty(c, pullq.NextItem)
 }
 
 type ClosingBuffer struct {
@@ -320,19 +322,3 @@ type ClosingBuffer struct {
 func (cb *ClosingBuffer) Close() (err error) {
        return
 }
-
-func expectWorkerChannelEmpty(c *C, workerChannel <-chan interface{}) {
-       select {
-       case item := <-workerChannel:
-               c.Fatalf("Received value (%v) from channel that was expected to be empty", item)
-       default:
-       }
-}
-
-func expectWorkerChannelNotEmpty(c *C, workerChannel <-chan interface{}) {
-       select {
-       case item := <-workerChannel:
-               c.Fatalf("Received value (%v) from channel that was expected to be empty", item)
-       default:
-       }
-}
index 4fbe4bb59624e22e820cf7eab7c2bb5013212e14..2cf2dc8d327614e3f0b5b1e61b763aad4f34fc30 100644 (file)
@@ -18,6 +18,7 @@ func RunTrashWorker(trashq *WorkQueue) {
        for item := range trashq.NextItem {
                trashRequest := item.(TrashRequest)
                TrashItem(trashRequest)
+               trashq.ReportDone <- struct{}{}
        }
 }
 
index 9509cacd774f5acd64372b55b1841bab55e5b023..6ec52742a57afb826128678950d9c90000553233 100644 (file)
@@ -85,76 +85,130 @@ package main
 import "container/list"
 
 type WorkQueue struct {
-       newlist  chan *list.List
-       NextItem chan interface{}
+       countInProgress  chan int
+       countOutstanding chan int
+       countQueued      chan int
+       newlist          chan *list.List
+       // Workers get work items by reading from this channel.
+       NextItem <-chan interface{}
+       // Each worker must send struct{}{} to ReportDone exactly once
+       // for each work item received from NextItem, when it stops
+       // working on that item (regardless of whether the work was
+       // successful).
+       ReportDone chan<- struct{}
 }
 
-// NewWorkQueue returns a new worklist, and launches a listener
-// goroutine that waits for work and farms it out to workers.
+// NewWorkQueue returns a new empty WorkQueue.
 //
 func NewWorkQueue() *WorkQueue {
+       nextItem := make(chan interface{})
+       reportDone := make(chan struct{})
+       newList := make(chan *list.List)
        b := WorkQueue{
-               newlist:  make(chan *list.List),
-               NextItem: make(chan interface{}),
+               countQueued:      make(chan int),
+               countInProgress:  make(chan int),
+               countOutstanding: make(chan int),
+               newlist:          newList,
+               NextItem:         nextItem,
+               ReportDone:       reportDone,
        }
-       go b.listen()
+       go func() {
+               // Read new work lists from the newlist channel.
+               // Reply to "length" and "get next item" queries by
+               // sending to the countQueued and nextItem channels
+               // respectively. Return when the newlist channel
+               // closes.
+
+               todo := &list.List{}
+               countInProgress := 0
+
+               // When we're done, close the output channel; workers will
+               // shut down next time they ask for new work.
+               defer close(nextItem)
+               defer close(b.countInProgress)
+               defer close(b.countOutstanding)
+               defer close(b.countQueued)
+
+               var nextChan chan interface{}
+               var nextVal interface{}
+               for newList != nil || countInProgress > 0 {
+                       select {
+                       case p, ok := <-newList:
+                               if !ok {
+                                       // Closed, stop receiving
+                                       newList = nil
+                               }
+                               todo = p
+                               if todo == nil {
+                                       todo = &list.List{}
+                               }
+                               if todo.Len() == 0 {
+                                       // Stop sending work
+                                       nextChan = nil
+                                       nextVal = nil
+                               } else {
+                                       nextChan = nextItem
+                                       nextVal = todo.Front().Value
+                               }
+                       case nextChan <- nextVal:
+                               countInProgress++
+                               todo.Remove(todo.Front())
+                               if todo.Len() == 0 {
+                                       // Stop sending work
+                                       nextChan = nil
+                                       nextVal = nil
+                               } else {
+                                       nextVal = todo.Front().Value
+                               }
+                       case <-reportDone:
+                               countInProgress--
+                       case b.countInProgress <- countInProgress:
+                       case b.countOutstanding <- todo.Len() + countInProgress:
+                       case b.countQueued <- todo.Len():
+                       }
+               }
+       }()
        return &b
 }
 
-// ReplaceQueue sends a new list of pull requests to the manager goroutine.
-// The manager will discard any outstanding pull list and begin
-// working on the new list.
+// ReplaceQueue abandons any work items left in the existing queue,
+// and starts giving workers items from the given list. After giving
+// it to ReplaceQueue, the caller must not read or write the given
+// list.
 //
 func (b *WorkQueue) ReplaceQueue(list *list.List) {
        b.newlist <- list
 }
 
 // Close shuts down the manager and terminates the goroutine, which
-// completes any pull request in progress and abandons any pending
-// requests.
+// abandons any pending requests, but allows any pull request already
+// in progress to continue.
+//
+// After Close, CountX methods will return correct values, NextItem
+// will be closed, and ReplaceQueue will panic.
 //
 func (b *WorkQueue) Close() {
        close(b.newlist)
 }
 
-// listen is run in a goroutine. It reads new pull lists from its
-// input queue until the queue is closed.
-// listen takes ownership of the list that is passed to it.
+// CountOutstanding returns the number of items in the queue or in
+// progress. A return value of 0 guarantees all existing work (work
+// that was sent to ReplaceQueue before CountOutstanding was called)
+// has completed.
 //
-// Note that the routine does not ever need to access the list
-// itself once the current_item has been initialized, so we do
-// not bother to keep a pointer to the list. Because it is a
-// doubly linked list, holding on to the current item will keep
-// it from garbage collection.
+func (b *WorkQueue) CountOutstanding() int {
+       // If the channel is closed, we get zero, which is correct.
+       return <-b.countOutstanding
+}
+
+// CountQueued returns the number of items in the current queue.
 //
-func (b *WorkQueue) listen() {
-       var current_item *list.Element
-
-       // When we're done, close the output channel to shut down any
-       // workers.
-       defer close(b.NextItem)
-
-       for {
-               // If the current list is empty, wait for a new list before
-               // even checking if workers are ready.
-               if current_item == nil {
-                       if p, ok := <-b.newlist; ok {
-                               current_item = p.Front()
-                       } else {
-                               // The channel was closed; shut down.
-                               return
-                       }
-               }
-               select {
-               case p, ok := <-b.newlist:
-                       if ok {
-                               current_item = p.Front()
-                       } else {
-                               // The input channel is closed; time to shut down
-                               return
-                       }
-               case b.NextItem <- current_item.Value:
-                       current_item = current_item.Next()
-               }
-       }
+func (b *WorkQueue) CountQueued() int {
+       return <-b.countQueued
+}
+
+// Len returns the number of items in progress.
+//
+func (b *WorkQueue) CountInProgress() int {
+       return <-b.countInProgress
 }
index 144e4c252be9ba1a069d8cf862f72c4284a28479..df0fa9c61284ba37f5cc68a97bc8e664038fa1e5 100644 (file)
@@ -2,9 +2,15 @@ package main
 
 import (
        "container/list"
+       "runtime"
        "testing"
+       "time"
 )
 
+type fatalfer interface {
+       Fatalf(string, ...interface{})
+}
+
 func makeTestWorkList(ary []int) *list.List {
        l := list.New()
        for _, n := range ary {
@@ -13,50 +19,107 @@ func makeTestWorkList(ary []int) *list.List {
        return l
 }
 
-func expectChannelEmpty(t *testing.T, c <-chan interface{}) {
+func expectChannelEmpty(t fatalfer, c <-chan interface{}) {
        select {
-       case item := <-c:
-               t.Fatalf("Received value (%v) from channel that we expected to be empty", item)
+       case item, ok := <-c:
+               if ok {
+                       t.Fatalf("Received value (%+v) from channel that we expected to be empty", item)
+               }
        default:
-               // no-op
        }
 }
 
-func expectChannelNotEmpty(t *testing.T, c <-chan interface{}) {
-       if item, ok := <-c; !ok {
-               t.Fatal("expected data on a closed channel")
-       } else if item == nil {
-               t.Fatal("expected data on an empty channel")
+func expectChannelNotEmpty(t fatalfer, c <-chan interface{}) interface{} {
+       select {
+       case item, ok := <-c:
+               if !ok {
+                       t.Fatalf("expected data on a closed channel")
+               }
+               return item
+       case <-time.After(time.Second):
+               t.Fatalf("expected data on an empty channel")
+               return nil
        }
 }
 
-func expectChannelClosed(t *testing.T, c <-chan interface{}) {
-       received, ok := <-c
-       if ok {
-               t.Fatalf("Expected channel to be closed, but received %v instead", received)
+func expectChannelClosedWithin(t fatalfer, timeout time.Duration, c <-chan interface{}) {
+       select {
+       case received, ok := <-c:
+               if ok {
+                       t.Fatalf("Expected channel to be closed, but received %+v instead", received)
+               }
+       case <-time.After(timeout):
+               t.Fatalf("Expected channel to be closed, but it is still open after %v", timeout)
        }
 }
 
-func expectFromChannel(t *testing.T, c <-chan interface{}, expected []int) {
+func doWorkItems(t fatalfer, q *WorkQueue, expected []int) {
        for i := range expected {
-               actual, ok := <-c
-               t.Logf("received %v", actual)
+               actual, ok := <-q.NextItem
                if !ok {
-                       t.Fatalf("Expected %v but channel was closed after receiving the first %d elements correctly.", expected, i)
-               } else if actual.(int) != expected[i] {
-                       t.Fatalf("Expected %v but received '%v' after receiving the first %d elements correctly.", expected[i], actual, i)
+                       t.Fatalf("Expected %+v but channel was closed after receiving %+v as expected.", expected, expected[:i])
+               }
+               q.ReportDone <- struct{}{}
+               if actual.(int) != expected[i] {
+                       t.Fatalf("Expected %+v but received %+v after receiving %+v as expected.", expected[i], actual, expected[:i])
                }
        }
 }
 
+func expectEqualWithin(t fatalfer, timeout time.Duration, expect interface{}, f func() interface{}) {
+       ok := make(chan struct{})
+       giveup := false
+       go func() {
+               for f() != expect && !giveup {
+                       time.Sleep(time.Millisecond)
+               }
+               close(ok)
+       }()
+       select {
+       case <-ok:
+       case <-time.After(timeout):
+               giveup = true
+               _, file, line, _ := runtime.Caller(1)
+               t.Fatalf("Still getting %+v, timed out waiting for %+v\n%s:%d", f(), expect, file, line)
+       }
+}
+
+func expectCountQueued(t fatalfer, b *WorkQueue, expectCountQueued int) {
+       if l := b.CountQueued(); l != expectCountQueued {
+               t.Fatalf("Got CountQueued()==%d, expected %d", l, expectCountQueued)
+       }
+}
+
+func TestWorkQueueDoneness(t *testing.T) {
+       b := NewWorkQueue()
+       defer b.Close()
+       b.ReplaceQueue(makeTestWorkList([]int{1, 2, 3}))
+       expectCountQueued(t, b, 3)
+       go func() {
+               for _ = range b.NextItem {
+                       //time.Sleep(time.Duration(delay.(int)) * time.Millisecond)
+                       time.Sleep(time.Millisecond)
+                       b.ReportDone <- struct{}{}
+               }
+       }()
+       expectEqualWithin(t, time.Second, 0, func() interface{} { return b.CountOutstanding() })
+       b.ReplaceQueue(makeTestWorkList([]int{400, 5, 6}))
+       expectEqualWithin(t, time.Second, 3, func() interface{} { return b.CountOutstanding() })
+       expectEqualWithin(t, time.Second, 0, func() interface{} { return b.CountOutstanding() })
+       expectChannelEmpty(t, b.NextItem)
+}
+
 // Create a WorkQueue, generate a list for it, and instantiate a worker.
 func TestWorkQueueReadWrite(t *testing.T) {
        var input = []int{1, 1, 2, 3, 5, 8, 13, 21, 34}
 
        b := NewWorkQueue()
+       expectCountQueued(t, b, 0)
+
        b.ReplaceQueue(makeTestWorkList(input))
+       expectCountQueued(t, b, len(input))
 
-       expectFromChannel(t, b.NextItem, input)
+       doWorkItems(t, b, input)
        expectChannelEmpty(t, b.NextItem)
        b.Close()
 }
@@ -66,6 +129,7 @@ func TestWorkQueueEarlyRead(t *testing.T) {
        var input = []int{1, 1, 2, 3, 5, 8, 13, 21, 34}
 
        b := NewWorkQueue()
+       defer b.Close()
 
        // First, demonstrate that nothing is available on the NextItem
        // channel.
@@ -76,8 +140,7 @@ func TestWorkQueueEarlyRead(t *testing.T) {
        //
        done := make(chan int)
        go func() {
-               expectFromChannel(t, b.NextItem, input)
-               b.Close()
+               doWorkItems(t, b, input)
                done <- 1
        }()
 
@@ -85,8 +148,29 @@ func TestWorkQueueEarlyRead(t *testing.T) {
        // finish.
        b.ReplaceQueue(makeTestWorkList(input))
        <-done
+       expectCountQueued(t, b, 0)
+}
 
-       expectChannelClosed(t, b.NextItem)
+// After Close(), NextItem closes, work finishes, then stats return zero.
+func TestWorkQueueClose(t *testing.T) {
+       b := NewWorkQueue()
+       input := []int{1, 2, 3, 4, 5, 6, 7, 8}
+       mark := make(chan struct{})
+       go func() {
+               <-b.NextItem
+               mark <- struct{}{}
+               <-mark
+               b.ReportDone <- struct{}{}
+       }()
+       b.ReplaceQueue(makeTestWorkList(input))
+       // Wait for worker to take item 1
+       <-mark
+       b.Close()
+       expectEqualWithin(t, time.Second, 1, func() interface{} { return b.CountOutstanding() })
+       // Tell worker to report done
+       mark <- struct{}{}
+       expectEqualWithin(t, time.Second, 0, func() interface{} { return b.CountOutstanding() })
+       expectChannelClosedWithin(t, time.Second, b.NextItem)
 }
 
 // Show that a reader may block when the manager's list is exhausted,
@@ -99,10 +183,11 @@ func TestWorkQueueReaderBlocks(t *testing.T) {
        )
 
        b := NewWorkQueue()
+       defer b.Close()
        sendmore := make(chan int)
        done := make(chan int)
        go func() {
-               expectFromChannel(t, b.NextItem, inputBeforeBlock)
+               doWorkItems(t, b, inputBeforeBlock)
 
                // Confirm that the channel is empty, so a subsequent read
                // on it will block.
@@ -110,8 +195,7 @@ func TestWorkQueueReaderBlocks(t *testing.T) {
 
                // Signal that we're ready for more input.
                sendmore <- 1
-               expectFromChannel(t, b.NextItem, inputAfterBlock)
-               b.Close()
+               doWorkItems(t, b, inputAfterBlock)
                done <- 1
        }()
 
@@ -136,14 +220,14 @@ func TestWorkQueueReplaceQueue(t *testing.T) {
 
        // Read just the first five elements from the work list.
        // Confirm that the channel is not empty.
-       expectFromChannel(t, b.NextItem, firstInput[0:5])
+       doWorkItems(t, b, firstInput[0:5])
        expectChannelNotEmpty(t, b.NextItem)
 
        // Replace the work list and read five more elements.
        // The old list should have been discarded and all new
        // elements come from the new list.
        b.ReplaceQueue(makeTestWorkList(replaceInput))
-       expectFromChannel(t, b.NextItem, replaceInput[0:5])
+       doWorkItems(t, b, replaceInput[0:5])
 
        b.Close()
 }