From 39ae8e64060a27ec314fc7a9c65d8d3dd4d9d60f Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 24 Feb 2015 20:32:20 -0700 Subject: [PATCH 01/20] Add topic segmentation. --- messaging/broker.go | 169 ++++++++++++++++++++++++++++++++++++-------- 1 file changed, 138 insertions(+), 31 deletions(-) diff --git a/messaging/broker.go b/messaging/broker.go index 854d1e290c2..14076acb8eb 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -17,8 +17,14 @@ import ( "github.com/influxdb/influxdb/raft" ) -// BroadcastTopicID is the topic used to communicate with all replicas. -const BroadcastTopicID = uint64(0) +const ( + // BroadcastTopicID is the topic used to communicate with all replicas. + BroadcastTopicID = uint64(0) + + // MaxSegmentSize represents the largest size a segment can be before a + // new segment is started. + MaxSegmentSize = 10 * 1024 * 1024 // 10MB +) // Broker represents distributed messaging system segmented into topics. // Each topic represents a linear series of events. @@ -50,8 +56,10 @@ func NewBroker() *Broker { // Returns empty string if the broker is not open. func (b *Broker) Path() string { return b.path } +// Log returns the underlying raft log. func (b *Broker) Log() *raft.Log { return b.log } +// metaPath returns the file path to the broker's metadata file. func (b *Broker) metaPath() string { if b.path == "" { return "" @@ -63,10 +71,11 @@ func (b *Broker) metaPath() string { // Returns 0 if the broker is closed. func (b *Broker) Index() uint64 { b.mu.Lock() - b.mu.Unlock() + defer b.mu.Unlock() return b.index } +// opened returns true if the broker is in an open and running state. func (b *Broker) opened() bool { return b.path != "" } // SetLogOutput sets writer for all Broker log output. @@ -100,6 +109,7 @@ func (b *Broker) Open(path string, u *url.URL) error { // Open underlying raft log. if err := b.log.Open(filepath.Join(path, "raft")); err != nil { + _ = b.close() return fmt.Errorf("raft: %s", err) } @@ -170,7 +180,7 @@ func (b *Broker) load() error { // Copy topic files from snapshot to local disk. for _, st := range hdr.Topics { - t := b.createTopic(st.ID) + t := b.newTopic(st.ID) t.index = st.Index // Open new empty topic file. @@ -342,9 +352,7 @@ func (b *Broker) PublishSync(m *Message) error { } // Sync pauses until the given index has been applied. -func (b *Broker) Sync(index uint64) error { - return b.log.Wait(index) -} +func (b *Broker) Sync(index uint64) error { return b.log.Wait(index) } // Replica returns a replica by id. func (b *Broker) Replica(id uint64) *Replica { @@ -366,7 +374,7 @@ func (b *Broker) Replicas() []*Replica { } // initializes a new topic object. -func (b *Broker) createTopic(id uint64) *topic { +func (b *Broker) newTopic(id uint64) *topic { t := &topic{ id: id, path: filepath.Join(b.path, strconv.FormatUint(uint64(id), 10)), @@ -381,7 +389,7 @@ func (b *Broker) createTopicIfNotExists(id uint64) *topic { return t } - t := b.createTopic(id) + t := b.newTopic(id) b.mustSave() return t } @@ -508,7 +516,7 @@ func (b *Broker) mustApplySubscribe(m *Message) { t.replicas[c.ReplicaID] = r // Catch up replica. - _, _ = t.writeTo(r, c.Index) + _ = t.writeTo(r, c.Index) b.mustSave() } @@ -670,7 +678,7 @@ func (fsm *brokerFSM) Restore(r io.Reader) error { // Copy topic files from snapshot to local disk. for _, st := range s.Topics { - t := b.createTopic(st.ID) + t := b.newTopic(st.ID) t.index = st.Index // Remove existing file if it exists. @@ -755,27 +763,47 @@ type snapshotReplicaTopic struct { // topic represents a single named queue of messages. // Each topic is identified by a unique path. +// +// Topics write their entries to segmented log files which contain a +// contiguous range of entries. These segments are periodically dropped +// as data is replicated the replicas and the replicas heartbeat back +// a confirmation of receipt. type topic struct { - id uint64 // unique identifier - index uint64 // highest index written - path string // on-disk path + id uint64 // unique identifier + index uint64 // highest index written + path string // on-disk path + segments segments // list of available segments - file *os.File // on-disk representation + file *os.File // current segment replicas map[uint64]*Replica // replicas subscribed to topic } +// segmentPath returns the path to a segment starting with a given log index. +func (t *topic) segmentPath(index uint64) string { + path := t.path + if path == "" { + return "" + } + return filepath.Join(path, strconv.FormatUint(index, 10)) +} + // open opens a topic for writing. func (t *topic) open() error { assert(t.file == nil, "topic already open: %d", t.id) // Ensure the parent directory exists. - if err := os.MkdirAll(filepath.Dir(t.path), 0700); err != nil { + if err := os.MkdirAll(t.path, 0700); err != nil { return err } - // Open the writer to the on-disk file. - f, err := os.OpenFile(t.path, os.O_RDWR|os.O_CREATE|os.O_APPEND, 0600) + // Read available segments. + if err := t.loadSegments(); err != nil { + return fmt.Errorf("read segments: %s", err) + } + + // Open the writer on the latest segment. + f, err := os.OpenFile(t.segments.last().path, os.O_RDWR|os.O_CREATE|os.O_APPEND, 0600) if err != nil { return err } @@ -784,6 +812,47 @@ func (t *topic) open() error { return nil } +// loadSegments reads all available segments for the topic. +// At least one segment will always exist. +func (t *topic) loadSegments() error { + // Open handle to directory. + f, err := os.Open(t.path) + if err != nil { + return err + } + defer func() { _ = f.Close() }() + + // Read directory items. + fis, err := f.Readdir(0) + if err != nil { + return err + } + + // Create a segment for each file with a numeric name. + var a segments + for _, fi := range fis { + index, err := strconv.ParseUint(fi.Name(), 10, 64) + if err != nil { + continue + } + a = append(a, &segment{ + index: index, + path: t.segmentPath(index), + size: fi.Size(), + }) + } + sort.Sort(a) + + // Create a first segment if one doesn't exist. + if len(a) == 0 { + a = segments{&segment{index: 0, path: t.segmentPath(0), size: 0}} + } + + t.segments = a + + return nil +} + // close closes the underlying file. func (t *topic) Close() error { // Close file. @@ -797,7 +866,7 @@ func (t *topic) Close() error { // loadIndex reads the highest available index for a topic from disk. func (t *topic) loadIndex() error { // Open topic file for reading. - f, err := os.Open(t.path) + f, err := os.Open(t.segments.last().path) if os.IsNotExist(err) { return nil } else if err != nil { @@ -823,21 +892,44 @@ func (t *topic) loadIndex() error { // writeTo writes the topic to a replica since a given index. // Returns an error if the starting index is unavailable. -func (t *topic) writeTo(r *Replica, index uint64) (int64, error) { +func (t *topic) writeTo(r *Replica, index uint64) error { // TODO: If index is too old then return an error. - // Open topic file for reading. + // Loop over each segment and write if it contains entries after index. + segments := t.segments + for i, s := range segments { + // Determine the maximum index in the range. + var next *segment + if i < len(segments)-1 { + next = segments[i+1] + } + + // If the index is after the end of the segment then ignore. + if next != nil && index >= next.index { + continue + } + + // Otherwise write segment to replica. + if err := t.writeSegmentTo(r, index, s); err != nil { + return fmt.Errorf("write segment(%d/%d): %s", t.id, s.index, err) + } + } + + return nil +} + +func (t *topic) writeSegmentTo(r *Replica, index uint64, segment *segment) error { + // Open segment for reading. // If it doesn't exist then just exit immediately. - f, err := os.Open(t.path) + f, err := os.Open(segment.path) if os.IsNotExist(err) { - return 0, nil + return nil } else if err != nil { - return 0, err + return err } defer func() { _ = f.Close() }() // Stream out all messages until EOF. - var total int64 dec := NewMessageDecoder(bufio.NewReader(f)) for { // Decode message. @@ -845,7 +937,7 @@ func (t *topic) writeTo(r *Replica, index uint64) (int64, error) { if err := dec.Decode(&m); err == io.EOF { break } else if err != nil { - return total, fmt.Errorf("decode: %s", err) + return fmt.Errorf("decode: %s", err) } // Ignore message if it's on or before high water mark. @@ -854,14 +946,13 @@ func (t *topic) writeTo(r *Replica, index uint64) (int64, error) { } // Write message out to stream. - n, err := m.WriteTo(r) + _, err := m.WriteTo(r) if err != nil { - return total, fmt.Errorf("write to: %s", err) + return fmt.Errorf("write to: %s", err) } - total += n } - return total, nil + return nil } // encode writes a message to the end of the topic. @@ -897,6 +988,22 @@ func (t *topic) encode(m *Message) error { return nil } +// segment represents a contiguous section of a topic log. +type segment struct { + index uint64 // starting index of the segment and name + path string // path to the segment file. + size int64 // total size of the segment file, in bytes. +} + +// segments represents a list of segments sorted by index. +type segments []*segment + +func (a segments) last() *segment { return a[len(a)-1] } +func (a segments) Len() int { return len(a) } +func (a segments) Less(i, j int) bool { return a[i].index < a[j].index } +func (a segments) Swap(i, j int) { a[i], a[j] = a[j], a[i] } + +// replicas represents a sortable list of replicas. type replicas []*Replica func (a replicas) Len() int { return len(a) } @@ -1004,7 +1111,7 @@ func (r *Replica) WriteTo(w io.Writer) (int64, error) { // Write topic messages from last known index. // Replica machine can ignore messages it already seen. index := r.topics[topicID] - if _, err := t.writeTo(r, index); err != nil { + if err := t.writeTo(r, index); err != nil { r.closeWriter() return 0, fmt.Errorf("add stream writer: %s", err) } From 16dbe8ba2face3b080de6f286332b9f6430ccb4d Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 1 Mar 2015 07:06:25 -0700 Subject: [PATCH 02/20] Add Broker.Truncate(). --- messaging/broker.go | 378 +++++++++++++++++++++++++++----------- messaging/broker_test.go | 3 + messaging/client.go | 57 +++++- messaging/client_test.go | 62 ++++++- messaging/errors.go | 6 + messaging/handler.go | 32 ++++ messaging/handler_test.go | 9 + 7 files changed, 435 insertions(+), 112 deletions(-) diff --git a/messaging/broker.go b/messaging/broker.go index 14076acb8eb..d3a6500d585 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -181,9 +181,9 @@ func (b *Broker) load() error { // Copy topic files from snapshot to local disk. for _, st := range hdr.Topics { t := b.newTopic(st.ID) - t.index = st.Index - // Open new empty topic file. + // Ignore segment data in the snapshot since we have the data locally. + // Simply opening the topic will automatically build the segments object. if err := t.open(); err != nil { return fmt.Errorf("open topic: %s", err) } @@ -196,8 +196,8 @@ func (b *Broker) load() error { b.replicas[r.id] = r // Append replica's topics. - for _, srt := range sr.Topics { - r.topics[srt.TopicID] = srt.Index + for _, topicID := range sr.TopicIDs { + r.topics[topicID] = struct{}{} } } @@ -258,44 +258,55 @@ func (b *Broker) mustSave() { // createSnapshotHeader creates a snapshot header. func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) { // Create parent header. - s := &snapshotHeader{} + sh := &snapshotHeader{} // Append topics. for _, t := range b.topics { - // Retrieve current topic file size. - var sz int64 - if t.file != nil { - fi, err := t.file.Stat() - if err != nil { - return nil, err + // Create snapshot topic. + st := &snapshotTopic{ID: t.id} + + // Add segments to topic. + for _, s := range t.segments { + // Retrieve current segment file size from disk. + var size int64 + fi, err := os.Stat(s.path) + if os.IsNotExist(err) { + size = 0 + } else if err == nil { + size = fi.Size() + } else { + return nil, fmt.Errorf("stat segment: %s", err) + } + + // Append segment. + st.Segments = append(st.Segments, &snapshotTopicSegment{ + Index: s.index, + Size: size, + path: s.path, + }) + + // Bump the snapshot header max index. + if s.index > sh.Index { + sh.Index = s.index } - sz = fi.Size() } // Append topic to the snapshot. - s.Topics = append(s.Topics, &snapshotTopic{ - ID: t.id, - Index: t.index, - Size: sz, - path: t.path, - }) + sh.Topics = append(sh.Topics, st) } // Append replicas and the current index for each topic. for _, r := range b.replicas { sr := &snapshotReplica{ID: r.id, URL: r.URL.String()} - for topicID, index := range r.topics { - sr.Topics = append(sr.Topics, &snapshotReplicaTopic{ - TopicID: topicID, - Index: index, - }) + for topicID := range r.topics { + sr.TopicIDs = append(sr.TopicIDs, topicID) } - s.Replicas = append(s.Replicas, sr) + sh.Replicas = append(sh.Replicas, sr) } - return s, nil + return sh, nil } // URL returns the connection url for the broker. @@ -373,7 +384,29 @@ func (b *Broker) Replicas() []*Replica { return a } -// initializes a new topic object. +// minReplicaTopicIndex returns the lowest index replicated for all replicas +// subscribed to a given topic. Requires a lock. +func (b *Broker) minReplicaTopicIndex(topicID uint64) uint64 { + var index uint64 + var updated bool + + for _, r := range b.replicas { + // Ignore replicas that are not subscribed. + if _, ok := r.topics[topicID]; !ok { + continue + } + + // Move the index down if unset or a lowest index is found. + if !updated || r.index < index { + index = r.index + updated = true + } + } + + return index +} + +// initializes a new topic object. Requires lock. func (b *Broker) newTopic(id uint64) *topic { t := &topic{ id: id, @@ -384,13 +417,31 @@ func (b *Broker) newTopic(id uint64) *topic { return t } -func (b *Broker) createTopicIfNotExists(id uint64) *topic { +// creates and opens a topic if it doesn't already exist. Requires lock. +func (b *Broker) createTopicIfNotExists(id uint64) (*topic, error) { if t := b.topics[id]; t != nil { - return t + return t, nil } + // Create topic and save metadata. t := b.newTopic(id) - b.mustSave() + if err := b.save(); err != nil { + return nil, fmt.Errorf("save: %s", err) + } + + // Open topic. + if err := t.open(); err != nil { + return nil, fmt.Errorf("open topic: %s", err) + } + + return t, nil +} + +func (b *Broker) mustCreateTopicIfNotExists(id uint64) *topic { + t, err := b.createTopicIfNotExists(id) + if err != nil { + panic(err.Error()) + } return t } @@ -420,8 +471,8 @@ func (b *Broker) mustApplyCreateReplica(m *Message) { r := newReplica(b, c.ID, c.URL) // Automatically subscribe to the config topic. - t := b.createTopicIfNotExists(BroadcastTopicID) - r.topics[BroadcastTopicID] = t.index + b.createTopicIfNotExists(BroadcastTopicID) + r.topics[BroadcastTopicID] = struct{}{} // Add replica to the broker. b.replicas[c.ID] = r @@ -462,7 +513,7 @@ func (b *Broker) mustApplyDeleteReplica(m *Message) { delete(t.replicas, r.id) } } - r.topics = make(map[uint64]uint64) + r.topics = make(map[uint64]struct{}) // Close replica's writer. r.closeWriter() @@ -503,7 +554,7 @@ func (b *Broker) mustApplySubscribe(m *Message) { } // Save current index on topic. - t := b.createTopicIfNotExists(c.TopicID) + t := b.mustCreateTopicIfNotExists(c.TopicID) // Ensure topic is not already subscribed to. if _, ok := r.topics[c.TopicID]; ok { @@ -512,11 +563,11 @@ func (b *Broker) mustApplySubscribe(m *Message) { } // Add subscription to replica. - r.topics[c.TopicID] = c.Index + r.topics[c.TopicID] = struct{}{} t.replicas[c.ReplicaID] = r // Catch up replica. - _ = t.writeTo(r, c.Index) + _ = t.writeTo(r) b.mustSave() } @@ -555,6 +606,81 @@ func (b *Broker) mustApplyUnsubscribe(m *Message) { b.mustSave() } +// ReplicaIndex returns the highest received index of a replica. +func (b *Broker) ReplicaIndex(id uint64) (uint64, error) { + b.mu.Lock() + defer b.mu.Unlock() + + // Ensure replica exists. + r := b.replicas[id] + if r == nil { + return 0, ErrReplicaNotFound + } + return r.index, nil +} + +// Heartbeat records a heartbeat from a replica. +// The heartbeat is transient and only stored on the leader. It is used for +// truncating the broker log segments but truncation can only occur if the +// broker has current heartbeats from all replicas. +func (b *Broker) Heartbeat(id, index uint64) error { + b.mu.Lock() + defer b.mu.Unlock() + + // Ignore if the broker is not the leader. + if !b.IsLeader() { + return raft.ErrNotLeader + } + + // Find replica. + r := b.replicas[id] + if r == nil { + return ErrReplicaNotFound + } + + // Update its highest index received. + r.index = index + return nil +} + +// Truncate removes log segments that have been replicated to all subscribed replicas. +func (b *Broker) Truncate() error { + b.mu.Lock() + defer b.mu.Unlock() + + // Loop over every topic. + for _, t := range b.topics { + // Determine the highest index replicated to all subscribed replicas. + minReplicaTopicIndex := b.minReplicaTopicIndex(t.id) + + // Loop over segments and close as needed. + newSegments := make(segments, 0, len(t.segments)) + for i, s := range t.segments { + // Find the next segment so we can find the upper index bound. + var next *segment + if i < len(t.segments)-1 { + next = t.segments[i+1] + } + + // Ignore the last segment or if the next index is less than + // the highest index replicated across all replicas. + if next == nil || minReplicaTopicIndex < next.index { + newSegments = append(newSegments, s) + continue + } + + // Remove the segment if the replicated index has moved pasted + // all the entries inside this segment. + s.close() + if err := os.Remove(s.path); err != nil && !os.IsNotExist(err) { + return fmt.Errorf("remove segment: topic=%d, segment=%d, err=%s", t.id, s.index, err) + } + } + } + + return nil +} + // brokerFSM implements the raft.FSM interface for the broker. // This is implemented as a separate type because it is not meant to be exported. type brokerFSM Broker @@ -595,7 +721,7 @@ func (fsm *brokerFSM) MustApply(e *raft.LogEntry) { m.Index = e.Index // Write to the topic. - t := b.createTopicIfNotExists(m.TopicID) + t := b.mustCreateTopicIfNotExists(m.TopicID) if err := t.encode(m); err != nil { panic("encode: " + err.Error()) } @@ -640,13 +766,15 @@ func (fsm *brokerFSM) Snapshot(w io.Writer) (uint64, error) { // Stream each topic sequentially. for _, t := range hdr.Topics { - if _, err := copyFileN(w, t.path, t.Size); err != nil { - return 0, err + for _, s := range t.Segments { + if _, err := copyFileN(w, s.path, s.Size); err != nil { + return 0, err + } } } // Return the snapshot and its last applied index. - return hdr.maxIndex(), nil + return hdr.Index, nil } // Restore reads the broker state. @@ -667,8 +795,8 @@ func (fsm *brokerFSM) Restore(r io.Reader) error { } // Decode header. - s := &snapshotHeader{} - if err := json.Unmarshal(buf, &s); err != nil { + sh := &snapshotHeader{} + if err := json.Unmarshal(buf, &sh); err != nil { return fmt.Errorf("decode header: %s", err) } @@ -677,35 +805,52 @@ func (fsm *brokerFSM) Restore(r io.Reader) error { b.closeReplicas() // Copy topic files from snapshot to local disk. - for _, st := range s.Topics { + for _, st := range sh.Topics { t := b.newTopic(st.ID) - t.index = st.Index // Remove existing file if it exists. - if err := os.Remove(t.path); err != nil && !os.IsNotExist(err) { + if err := os.RemoveAll(t.path); err != nil && !os.IsNotExist(err) { return err } + // Copy data from snapshot into segment files. + // We don't instantiate the segments because that will be done + // automatically when calling open() on the topic. + for _, ss := range st.Segments { + if err := func() error { + // Create a new file with the starting index. + f, err := os.Open(t.segmentPath(ss.Index)) + if err != nil { + return fmt.Errorf("open segment: %s", err) + } + defer func() { _ = f.Close() }() + + // Copy from stream into file. + if _, err := io.CopyN(f, r, ss.Size); err != nil { + return fmt.Errorf("copy segment: %s", err) + } + + return nil + }(); err != nil { + return err + } + } + // Open new empty topic file. if err := t.open(); err != nil { return fmt.Errorf("open topic: %s", err) } - - // Copy data from snapshot into file. - if _, err := io.CopyN(t.file, r, st.Size); err != nil { - return fmt.Errorf("copy topic: %s", err) - } } // Update the replicas. - for _, sr := range s.Replicas { + for _, sr := range sh.Replicas { // Create replica. r := newReplica(b, sr.ID, sr.URL) b.replicas[r.id] = r // Append replica's topics. - for _, srt := range sr.Topics { - r.topics[srt.TopicID] = srt.Index + for _, topicID := range sr.TopicIDs { + r.topics[topicID] = struct{}{} } } @@ -729,38 +874,27 @@ func copyFileN(w io.Writer, path string, n int64) (int64, error) { type snapshotHeader struct { Replicas []*snapshotReplica `json:"replicas"` Topics []*snapshotTopic `json:"topics"` -} - -// maxIndex returns the highest applied index across all topics. -func (s *snapshotHeader) maxIndex() uint64 { - var idx uint64 - for _, t := range s.Topics { - if t.Index > idx { - idx = t.Index - } - } - return idx + Index uint64 `json:"index"` } type snapshotReplica struct { - ID uint64 `json:"id"` - Topics []*snapshotReplicaTopic `json:"topics"` - URL string `json:"url"` + ID uint64 `json:"id"` + TopicIDs []uint64 `json:"topicIDs"` + URL string `json:"url"` } type snapshotTopic struct { - ID uint64 `json:"id"` + ID uint64 `json:"id"` + Segments []*snapshotTopicSegment `json:"segments"` +} + +type snapshotTopicSegment struct { Index uint64 `json:"index"` Size int64 `json:"size"` path string } -type snapshotReplicaTopic struct { - TopicID uint64 `json:"topicID"` - Index uint64 `json:"index"` -} - // topic represents a single named queue of messages. // Each topic is identified by a unique path. // @@ -774,8 +908,6 @@ type topic struct { path string // on-disk path segments segments // list of available segments - file *os.File // current segment - replicas map[uint64]*Replica // replicas subscribed to topic } @@ -790,7 +922,7 @@ func (t *topic) segmentPath(index uint64) string { // open opens a topic for writing. func (t *topic) open() error { - assert(t.file == nil, "topic already open: %d", t.id) + assert(len(t.segments) == 0, "topic already open: %d", t.id) // Ensure the parent directory exists. if err := os.MkdirAll(t.path, 0700); err != nil { @@ -802,13 +934,6 @@ func (t *topic) open() error { return fmt.Errorf("read segments: %s", err) } - // Open the writer on the latest segment. - f, err := os.OpenFile(t.segments.last().path, os.O_RDWR|os.O_CREATE|os.O_APPEND, 0600) - if err != nil { - return err - } - t.file = f - return nil } @@ -855,10 +980,8 @@ func (t *topic) loadSegments() error { // close closes the underlying file. func (t *topic) Close() error { - // Close file. - if t.file != nil { - _ = t.file.Close() - t.file = nil + for _, s := range t.segments { + _ = s.close() } return nil } @@ -890,11 +1013,14 @@ func (t *topic) loadIndex() error { } } -// writeTo writes the topic to a replica since a given index. +// writeTo writes the topic to a replica. Only writes messages after replica index. // Returns an error if the starting index is unavailable. -func (t *topic) writeTo(r *Replica, index uint64) error { +func (t *topic) writeTo(r *Replica) error { // TODO: If index is too old then return an error. + // Retrieve the replica's highest received index. + index := r.index + // Loop over each segment and write if it contains entries after index. segments := t.segments for i, s := range segments { @@ -957,24 +1083,36 @@ func (t *topic) writeSegmentTo(r *Replica, index uint64, segment *segment) error // encode writes a message to the end of the topic. func (t *topic) encode(m *Message) error { - // Ensure the topic is open and ready for writing. - if t.file == nil { - if err := t.open(); err != nil { - return fmt.Errorf("open: %s", err) - } - } - // Ensure message is in-order. assert(m.Index > t.index, "topic message out of order: %d -> %d", t.index, m.Index) + // Retrieve the last segment. + s := t.segments.last() + + // Close the segment if it's too large. + if s.size > MaxSegmentSize { + s.close() + s = nil + } + + // Create and append a new segment if we don't have one. + if s == nil { + t.segments = append(t.segments, &segment{index: m.Index, path: t.segmentPath(m.Index)}) + } + if s.file == nil { + if err := s.open(); err != nil { + return fmt.Errorf("open segment: %s", err) + } + } + // Encode message. b := make([]byte, messageHeaderSize+len(m.Data)) copy(b, m.marshalHeader()) copy(b[messageHeaderSize:], m.Data) - // Write to topic file. - if _, err := t.file.Write(b); err != nil { - return fmt.Errorf("encode header: %s", err) + // Write to segment. + if _, err := s.file.Write(b); err != nil { + return fmt.Errorf("write segment: %s", err) } // Move up high water mark on the topic. @@ -993,12 +1131,42 @@ type segment struct { index uint64 // starting index of the segment and name path string // path to the segment file. size int64 // total size of the segment file, in bytes. + + file *os.File // handle for writing, only open for last segment +} + +// open opens the file handle for append. +func (s *segment) open() error { + f, err := os.OpenFile(s.path, os.O_RDWR|os.O_CREATE|os.O_APPEND, 0600) + if err != nil { + return err + } + s.file = f + return nil +} + +// close closes the segment's writing file handle. +func (s *segment) close() error { + if s.file != nil { + err := s.file.Close() + s.file = nil + return err + } + return nil } // segments represents a list of segments sorted by index. type segments []*segment -func (a segments) last() *segment { return a[len(a)-1] } +// last returns the last segment in the slice. +// Returns nil if there are no elements. +func (a segments) last() *segment { + if len(a) == 0 { + return nil + } + return a[len(a)-1] +} + func (a segments) Len() int { return len(a) } func (a segments) Less(i, j int) bool { return a[i].index < a[j].index } func (a segments) Swap(i, j int) { a[i], a[j] = a[j], a[i] } @@ -1018,11 +1186,12 @@ type Replica struct { id uint64 broker *Broker + index uint64 // highest index replicated to the replica. writer io.Writer // currently attached writer done chan struct{} // notify when current writer is removed - topics map[uint64]uint64 // current index for each subscribed topic + topics map[uint64]struct{} // set of subscribed topics. } // newReplica returns a new Replica instance associated with a broker. @@ -1037,7 +1206,7 @@ func newReplica(b *Broker, id uint64, urlstr string) *Replica { URL: u, broker: b, id: id, - topics: make(map[uint64]uint64), + topics: make(map[uint64]struct{}), } } @@ -1108,10 +1277,8 @@ func (r *Replica) WriteTo(w io.Writer) (int64, error) { t := r.broker.topics[topicID] assert(t != nil, "topic missing: %s", topicID) - // Write topic messages from last known index. - // Replica machine can ignore messages it already seen. - index := r.topics[topicID] - if err := t.writeTo(r, index); err != nil { + // Write topic messages to replica. + if err := t.writeTo(r); err != nil { r.closeWriter() return 0, fmt.Errorf("add stream writer: %s", err) } @@ -1140,7 +1307,6 @@ type DeleteReplicaCommand struct { type SubscribeCommand struct { ReplicaID uint64 `json:"replicaID"` // replica id TopicID uint64 `json:"topicID"` // topic id - Index uint64 `json:"index"` // index } // UnsubscribeCommand removes a subscription for a topic from a replica. diff --git a/messaging/broker_test.go b/messaging/broker_test.go index 85f1a86fce4..5742a2791ad 100644 --- a/messaging/broker_test.go +++ b/messaging/broker_test.go @@ -80,6 +80,9 @@ func TestBroker_Publish(t *testing.T) { // Read out the config messages first. var m messaging.Message dec := messaging.NewMessageDecoder(&buf) + if err := dec.Decode(&m); err != nil || m.Type != messaging.InternalMessageType { + t.Fatalf("decode(internal): %x (%v)", m.Type, err) + } if err := dec.Decode(&m); err != nil || m.Type != messaging.CreateReplicaMessageType { t.Fatalf("decode(create replica): %x (%v)", m.Type, err) } diff --git a/messaging/client.go b/messaging/client.go index 3518f562f67..6fc2be02e35 100644 --- a/messaging/client.go +++ b/messaging/client.go @@ -44,8 +44,8 @@ type Client struct { opened bool done chan chan struct{} // disconnection notification - // Channel streams messages from the broker. - c chan *Message + c chan *Message // channel streams messages from the broker. + index uint64 // highest index sent over the channel // The amount of time to wait before reconnecting to a broker stream. ReconnectTimeout time.Duration @@ -71,6 +71,13 @@ func (c *Client) ReplicaID() uint64 { return c.replicaID } // of the incoming message index to make sure it has not been processed. func (c *Client) C() <-chan *Message { return c.c } +// Index returns the highest index sent over the channel. +func (c *Client) Index() uint64 { + c.mu.Lock() + defer c.mu.Unlock() + return c.index +} + // URLs returns a list of broker URLs to connect to. func (c *Client) URLs() []*url.URL { c.mu.Lock() @@ -167,9 +174,10 @@ func (c *Client) Close() error { c.done = nil } - // Close message stream. + // Close message stream & clear index. close(c.c) c.c = nil + c.index = 0 // Unset open flag. c.opened = false @@ -224,6 +232,42 @@ func (c *Client) Publish(m *Message) (uint64, error) { return index, nil } +// Heartbeat sends a heartbeat back to the broker with the client's index. +func (c *Client) Heartbeat() error { + var resp *http.Response + var err error + + // Retrieve the parameters under lock. + c.mu.Lock() + replicaID, index := c.replicaID, c.index + c.mu.Unlock() + + u := *c.LeaderURL() + // Send the message to the messages endpoint. + u.Path = "/messaging/heartbeat" + u.RawQuery = url.Values{ + "replicaID": {strconv.FormatUint(replicaID, 10)}, + "index": {strconv.FormatUint(index, 10)}, + }.Encode() + resp, err = http.Post(u.String(), "application/octet-stream", nil) + if err != nil { + return err + } + defer resp.Body.Close() + + // If the server returns a redirect then it's not the leader. + // If it returns a non-200 code then return the error. + if resp.StatusCode == http.StatusTemporaryRedirect { + return ErrNoLeader + } else if resp.StatusCode != http.StatusOK { + if errstr := resp.Header.Get("X-Broker-Error"); errstr != "" { + return errors.New(errstr) + } + return fmt.Errorf("heartbeat error: %d", resp.StatusCode) + } + return nil +} + // CreateReplica creates a replica on the broker. func (c *Client) CreateReplica(id uint64, u *url.URL) error { var resp *http.Response @@ -430,6 +474,13 @@ func (c *Client) streamFromURL(u *url.URL, done chan chan struct{}) error { // Write message to streaming channel. c.c <- m + + // Update the index on the client. + c.mu.Lock() + if m.Index > c.index { + c.index = m.Index + } + c.mu.Unlock() } }() diff --git a/messaging/client_test.go b/messaging/client_test.go index 2767f7ede31..811c8b6ba79 100644 --- a/messaging/client_test.go +++ b/messaging/client_test.go @@ -37,9 +37,11 @@ func TestClient_Open(t *testing.T) { t.Fatalf("unexpected error: %s", err) } - // Receive a message from the stream. - if m := <-c.C(); m.Type != messaging.CreateReplicaMessageType { - t.Fatalf("unexpected message type: %x", m.Type) + // Receive messages from the stream. + if m := <-c.C(); m.Type != messaging.InternalMessageType { + t.Fatalf("message type mismatch(internal): %x", m.Type) + } else if m = <-c.C(); m.Type != messaging.CreateReplicaMessageType { + t.Fatalf("message type mismatch(create replica): %x", m.Type) } // Close connection to the broker. @@ -135,6 +137,41 @@ func TestClient_Publish_ErrLogClosed(t *testing.T) { } } +// Ensure that a client can heartbeat its status to the broker. +func TestClient_Heartbeat(t *testing.T) { + c := OpenClient(1000) + defer c.Close() + + // Publish message to the broker. + index := c.MustPublish(&messaging.Message{Type: 100, TopicID: messaging.BroadcastTopicID, Data: []byte{}}) + + // Flush the channel until we receive our message. +loop: + for { + select { + case m := <-c.C(): + if m.Index == index { + break loop + } + } + } + + // Wait until the index moves forward. + c.Sync(index) + + // Send heartbeat to the broker. + if err := c.Heartbeat(); err != nil { + t.Fatal(err) + } + + // Verify the broker received the heartbeat. + if replicaIndex, err := c.Server.Broker().ReplicaIndex(1000); err != nil { + t.Fatal(err) + } else if replicaIndex != index { + t.Fatalf("unexpected replica index: exp=%d, got=%d", index, replicaIndex) + } +} + // Ensure that a client can create a replica. func TestClient_CreateReplica(t *testing.T) { c := OpenClient(0) @@ -272,6 +309,25 @@ func (c *Client) Close() { c.Server.Close() } +// MustPublish publishes a message. Panic on error. +func (c *Client) MustPublish(m *messaging.Message) uint64 { + index, err := c.Publish(m) + if err != nil { + panic(err.Error()) + } + return index +} + +// Sync waits until the client's index reaches the given index. +func (c *Client) Sync(index uint64) { + for { + if c.Index() == index { + return + } + time.Sleep(1 * time.Millisecond) + } +} + // NewTempFile returns the path of a new temporary file. // It is up to the caller to remove it when finished. func NewTempFile() string { diff --git a/messaging/errors.go b/messaging/errors.go index f07a4a621ba..4bf4a95a3aa 100644 --- a/messaging/errors.go +++ b/messaging/errors.go @@ -45,4 +45,10 @@ var ( // ErrTopicRequired is returned publishing a message without a topic ID. ErrTopicRequired = errors.New("topic required") + + // ErrNoLeader is returned when a leader cannot be reached. + ErrNoLeader = errors.New("no leader") + + // ErrIndexRequired is returned when making a call without a valid index. + ErrIndexRequired = errors.New("index required") ) diff --git a/messaging/handler.go b/messaging/handler.go index 7628c634ecf..2d31ae7f062 100644 --- a/messaging/handler.go +++ b/messaging/handler.go @@ -73,6 +73,12 @@ func (h *Handler) ServeHTTP(w http.ResponseWriter, r *http.Request) { } else { http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed) } + case "/messaging/heartbeat": + if r.Method == "POST" { + h.heartbeat(w, r) + } else { + http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed) + } default: http.NotFound(w, r) } @@ -263,6 +269,32 @@ func (h *Handler) unsubscribe(w http.ResponseWriter, r *http.Request) { w.WriteHeader(http.StatusNoContent) } +// recieves a heartbeat from a client. +func (h *Handler) heartbeat(w http.ResponseWriter, r *http.Request) { + // Read the replica id. + replicaID, err := strconv.ParseUint(r.URL.Query().Get("replicaID"), 10, 16) + if err != nil { + h.error(w, ErrReplicaIDRequired, http.StatusBadRequest) + return + } + + // Read the index. + index, err := strconv.ParseUint(r.URL.Query().Get("index"), 10, 16) + if err != nil { + h.error(w, ErrIndexRequired, http.StatusBadRequest) + return + } + + // Update the replica's index. + if err := h.broker.Heartbeat(replicaID, index); err == raft.ErrNotLeader { + h.redirectToLeader(w, r) + return + } else if err != nil { + h.error(w, err, http.StatusInternalServerError) + return + } +} + // error writes an error to the client and sets the status code. func (h *Handler) error(w http.ResponseWriter, err error, code int) { s := err.Error() diff --git a/messaging/handler_test.go b/messaging/handler_test.go index 8770f648817..c911f81c2f7 100644 --- a/messaging/handler_test.go +++ b/messaging/handler_test.go @@ -33,6 +33,15 @@ func TestHandler_stream(t *testing.T) { // Decode from body. var m messaging.Message dec := messaging.NewMessageDecoder(resp.Body) + + // First message should be an internal message. + if err := dec.Decode(&m); err != nil { + t.Fatalf("decode error: %s", err) + } else if m.Index != 1 && m.Type != messaging.InternalMessageType { + t.Fatalf("unexpected index/type: %d / %x", m.Index, m.Type) + } + + // Second message should be an the create replica message. if err := dec.Decode(&m); err != nil { t.Fatalf("decode error: %s", err) } else if m.Index != 2 && m.Type != messaging.CreateReplicaMessageType { From 1bbf154f2aad16074be2276da52ce4d82990c589 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 1 Mar 2015 22:32:20 -0700 Subject: [PATCH 03/20] Removing replicas and subscriptions from broker. --- messaging/broker.go | 889 +++++++++++++-------------------------- messaging/broker_test.go | 12 - messaging/handler.go | 223 +++------- server.go | 3 + 4 files changed, 344 insertions(+), 783 deletions(-) diff --git a/messaging/broker.go b/messaging/broker.go index d3a6500d585..0f362a431b8 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -13,18 +13,15 @@ import ( "sort" "strconv" "sync" + "time" + "github.com/boltdb/bolt" "github.com/influxdb/influxdb/raft" ) -const ( - // BroadcastTopicID is the topic used to communicate with all replicas. - BroadcastTopicID = uint64(0) - - // MaxSegmentSize represents the largest size a segment can be before a - // new segment is started. - MaxSegmentSize = 10 * 1024 * 1024 // 10MB -) +// MaxSegmentSize represents the largest size a segment can be before a +// new segment is started. +const MaxSegmentSize = 10 * 1024 * 1024 // 10MB // Broker represents distributed messaging system segmented into topics. // Each topic represents a linear series of events. @@ -34,8 +31,8 @@ type Broker struct { index uint64 // highest applied index log *raft.Log // internal raft log - replicas map[uint64]*Replica // replica by id - topics map[uint64]*topic // topics by id + meta *bolt.DB // metadata + topics map[uint64]*topic // topics by id Logger *log.Logger } @@ -43,10 +40,9 @@ type Broker struct { // NewBroker returns a new instance of a Broker with default values. func NewBroker() *Broker { b := &Broker{ - log: raft.NewLog(), - replicas: make(map[uint64]*Replica), - topics: make(map[uint64]*topic), - Logger: log.New(os.Stderr, "[broker] ", log.LstdFlags), + log: raft.NewLog(), + topics: make(map[uint64]*topic), + Logger: log.New(os.Stderr, "[broker] ", log.LstdFlags), } b.log.FSM = (*brokerFSM)(b) return b @@ -67,11 +63,19 @@ func (b *Broker) metaPath() string { return filepath.Join(b.path, "meta") } +// topicPath returns the file path to a topic's data. +func (b *Broker) topicPath(id uint64) string { + if b.path == "" { + return "" + } + return filepath.Join(b.path, strconv.FormatUint(id, 10)) +} + // Index returns the highest index seen by the broker across all topics. // Returns 0 if the broker is closed. func (b *Broker) Index() uint64 { - b.mu.Lock() - defer b.mu.Unlock() + b.mu.RLock() + defer b.mu.RUnlock() return b.index } @@ -101,12 +105,38 @@ func (b *Broker) Open(path string, u *url.URL) error { return ErrConnectionAddressRequired } - // Read meta data from snapshot. - if err := b.load(); err != nil { + // Open meta file. + meta, err := bolt.Open(b.metaPath(), 0600, &bolt.Options{Timeout: 1 * time.Second}) + if err != nil { + return fmt.Errorf("open meta: %s", err) + } + b.meta = meta + + // Initialize data from meta store. + if err := b.meta.Update(func(tx *bolt.Tx) error { + tx.CreateBucketIfNotExists([]byte("meta")) + + // Read in index from meta store, if set. + if v := tx.Bucket([]byte("meta")).Get([]byte("index")); v != nil { + b.index = btou64(v) + } + + return nil + }); err != nil { _ = b.close() return err } + // Read all topic metadata into memory. + if err := b.openTopics(); err != nil { + return fmt.Errorf("load topics: %s", err) + } + + // Read the highest index from each of the topic files. + if err := b.loadIndex(); err != nil { + return fmt.Errorf("load index: %s", err) + } + // Open underlying raft log. if err := b.log.Open(filepath.Join(path, "raft")); err != nil { _ = b.close() @@ -120,6 +150,24 @@ func (b *Broker) Open(path string, u *url.URL) error { return nil } +// loadTopics reads all topic metadata into memory. +func (b *Broker) openTopics() error { + // TODO: Determine topic metadata from directory listing. + panic("not yet implemented") +} + +// loadIndex reads through all topics to find the highest known index. +func (b *Broker) loadIndex() error { + for _, t := range b.topics { + if topicIndex, err := t.maxIndex(); err != nil { + return fmt.Errorf("topic max index: topic=%d, err=%s", t.id, err) + } else if topicIndex > b.index { + b.index = topicIndex + } + } + return nil +} + // Close closes the broker and all topics. func (b *Broker) Close() error { b.mu.Lock() @@ -134,9 +182,8 @@ func (b *Broker) close() error { } b.path = "" - // Close all topics & replicas. + // Close all topics. b.closeTopics() - b.closeReplicas() // Close raft log. _ = b.log.Close() @@ -152,109 +199,6 @@ func (b *Broker) closeTopics() { b.topics = make(map[uint64]*topic) } -// closeReplicas closes all replica writers and clears the replica map. -func (b *Broker) closeReplicas() { - for _, r := range b.replicas { - r.closeWriter() - } - b.replicas = make(map[uint64]*Replica) -} - -// load reads the broker metadata from disk. -func (b *Broker) load() error { - // Read snapshot header from disk. - // Ignore if no snapshot exists. - f, err := os.Open(b.metaPath()) - if os.IsNotExist(err) { - return nil - } else if err != nil { - return err - } - defer func() { _ = f.Close() }() - - // Read snapshot header from disk. - hdr := &snapshotHeader{} - if err := json.NewDecoder(f).Decode(&hdr); err != nil { - return err - } - - // Copy topic files from snapshot to local disk. - for _, st := range hdr.Topics { - t := b.newTopic(st.ID) - - // Ignore segment data in the snapshot since we have the data locally. - // Simply opening the topic will automatically build the segments object. - if err := t.open(); err != nil { - return fmt.Errorf("open topic: %s", err) - } - } - - // Update the replicas. - for _, sr := range hdr.Replicas { - // Create replica. - r := newReplica(b, sr.ID, sr.URL) - b.replicas[r.id] = r - - // Append replica's topics. - for _, topicID := range sr.TopicIDs { - r.topics[topicID] = struct{}{} - } - } - - // Read the highest index from each of the topic files. - if err := b.loadIndex(); err != nil { - return fmt.Errorf("load index: %s", err) - } - - return nil -} - -// loadIndex reads through all topics to find the highest known index. -func (b *Broker) loadIndex() error { - for _, t := range b.topics { - if err := t.loadIndex(); err != nil { - return fmt.Errorf("topic(%d): %s", t.id, err) - } else if t.index > b.index { - b.index = t.index - } - } - return nil -} - -// save persists the broker metadata to disk. -func (b *Broker) save() error { - if b.path == "" { - return ErrClosed - } - - // Calculate header under lock. - hdr, err := b.createSnapshotHeader() - if err != nil { - return fmt.Errorf("create snapshot: %s", err) - } - - // Write snapshot to disk. - f, err := os.Create(b.metaPath()) - if err != nil { - return err - } - defer func() { _ = f.Close() }() - - // Write snapshot to disk. - if err := json.NewEncoder(f).Encode(&hdr); err != nil { - return err - } - - return nil -} - -// mustSave persists the broker metadata to disk. Panic on error. -func (b *Broker) mustSave() { - if err := b.save(); err != nil && err != ErrClosed { - panic(err.Error()) - } -} - // createSnapshotHeader creates a snapshot header. func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) { // Create parent header. @@ -295,21 +239,10 @@ func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) { sh.Topics = append(sh.Topics, st) } - // Append replicas and the current index for each topic. - for _, r := range b.replicas { - sr := &snapshotReplica{ID: r.id, URL: r.URL.String()} - - for topicID := range r.topics { - sr.TopicIDs = append(sr.TopicIDs, topicID) - } - - sh.Replicas = append(sh.Replicas, sr) - } - return sh, nil } -// URL returns the connection url for the broker. +// U/RL returns the connection url for the broker. func (b *Broker) URL() *url.URL { return b.log.URL } @@ -342,7 +275,8 @@ func (b *Broker) Join(u *url.URL) error { // Publish writes a message. // Returns the index of the message. Otherwise returns an error. func (b *Broker) Publish(m *Message) (uint64, error) { - buf, _ := m.MarshalBinary() + buf, err := m.MarshalBinary() + assert(err == nil, "marshal binary error: %s", err) return b.log.Apply(buf) } @@ -365,54 +299,9 @@ func (b *Broker) PublishSync(m *Message) error { // Sync pauses until the given index has been applied. func (b *Broker) Sync(index uint64) error { return b.log.Wait(index) } -// Replica returns a replica by id. -func (b *Broker) Replica(id uint64) *Replica { - b.mu.RLock() - defer b.mu.RUnlock() - return b.replicas[id] -} - -// Replicas returns a list of the replicas in the system -func (b *Broker) Replicas() []*Replica { - b.mu.RLock() - defer b.mu.RUnlock() - a := make([]*Replica, 0, len(b.replicas)) - for _, r := range b.replicas { - a = append(a, r) - } - sort.Sort(replicas(a)) - return a -} - -// minReplicaTopicIndex returns the lowest index replicated for all replicas -// subscribed to a given topic. Requires a lock. -func (b *Broker) minReplicaTopicIndex(topicID uint64) uint64 { - var index uint64 - var updated bool - - for _, r := range b.replicas { - // Ignore replicas that are not subscribed. - if _, ok := r.topics[topicID]; !ok { - continue - } - - // Move the index down if unset or a lowest index is found. - if !updated || r.index < index { - index = r.index - updated = true - } - } - - return index -} - // initializes a new topic object. Requires lock. func (b *Broker) newTopic(id uint64) *topic { - t := &topic{ - id: id, - path: filepath.Join(b.path, strconv.FormatUint(uint64(id), 10)), - replicas: make(map[uint64]*Replica), - } + t := &topic{id: id, path: b.topicPath(id)} b.topics[t.id] = t return t } @@ -423,13 +312,8 @@ func (b *Broker) createTopicIfNotExists(id uint64) (*topic, error) { return t, nil } - // Create topic and save metadata. - t := b.newTopic(id) - if err := b.save(); err != nil { - return nil, fmt.Errorf("save: %s", err) - } - // Open topic. + t := b.newTopic(id) if err := t.open(); err != nil { return nil, fmt.Errorf("open topic: %s", err) } @@ -445,202 +329,65 @@ func (b *Broker) mustCreateTopicIfNotExists(id uint64) *topic { return t } -// CreateReplica creates a new named replica. -func (b *Broker) CreateReplica(id uint64, connectURL *url.URL) error { - b.mu.Lock() - defer b.mu.Unlock() - - // Ensure replica doesn't already exist. - s := b.replicas[id] - if s != nil { - return ErrReplicaExists - } - - // Add command to create replica. - return b.PublishSync(&Message{ - Type: CreateReplicaMessageType, - Data: mustMarshalJSON(&CreateReplicaCommand{ID: id, URL: connectURL.String()}), - }) -} - -func (b *Broker) mustApplyCreateReplica(m *Message) { - var c CreateReplicaCommand - mustUnmarshalJSON(m.Data, &c) - - // Create replica. - r := newReplica(b, c.ID, c.URL) - - // Automatically subscribe to the config topic. - b.createTopicIfNotExists(BroadcastTopicID) - r.topics[BroadcastTopicID] = struct{}{} - - // Add replica to the broker. - b.replicas[c.ID] = r - - b.mustSave() -} - -// DeleteReplica deletes an existing replica by id. -func (b *Broker) DeleteReplica(id uint64) error { - b.mu.Lock() - defer b.mu.Unlock() - - // Ensure replica exists. - if s := b.replicas[id]; s == nil { - return ErrReplicaNotFound - } - - // Issue command to remove replica. - return b.PublishSync(&Message{ - Type: DeleteReplicaMessageType, - Data: mustMarshalJSON(&DeleteReplicaCommand{ID: id}), - }) -} - -func (b *Broker) mustApplyDeleteReplica(m *Message) { - var c DeleteReplicaCommand - mustUnmarshalJSON(m.Data, &c) +// OpenTopicReader returns a reader on a topic that starts from a given index. +// +// If streaming is true then the reader is held open indefinitely and waits +// for new messages on the topic. If streaming is false then the reader will +// return EOF at the end of the topic. +func (b *Broker) OpenTopicReader(topicID, index uint64, streaming bool) (io.ReadCloser, error) { + b.mu.RLock() + defer b.mu.RUnlock() - // Find replica. - r := b.replicas[c.ID] - if r == nil { - return + // Exit if the broker is closed. + if !b.opened() { + return nil, ErrClosed } - // Remove replica from all subscribed topics. - for topicID := range r.topics { - if t := b.topics[topicID]; t != nil { - delete(t.replicas, r.id) - } + // Return new reader. + r := &topicReader{ + path: b.topicPath(topicID), + index: index, + streaming: streaming, } - r.topics = make(map[uint64]struct{}) - - // Close replica's writer. - r.closeWriter() - - // Remove replica from broker. - delete(b.replicas, c.ID) - - b.mustSave() + return r, nil } -// Subscribe adds a subscription to a topic from a replica. -func (b *Broker) Subscribe(replicaID, topicID uint64) error { - b.mu.Lock() - defer b.mu.Unlock() - - // TODO: Allow non-zero starting index. - - // Ensure replica & topic exist. - if b.replicas[replicaID] == nil { - return ErrReplicaNotFound - } - - // Issue command to subscribe to topic. - return b.PublishSync(&Message{ - Type: SubscribeMessageType, - Data: mustMarshalJSON(&SubscribeCommand{ReplicaID: replicaID, TopicID: topicID}), +// SetTopicMaxIndex updates the highest replicated index for a topic. +// If a higher index is already set on the topic then the call is ignored. +// This index is only held in memory and is used for topic segment reclamation. +func (b *Broker) SetTopicMaxIndex(topicID, index uint64) error { + _, err := b.Publish(&Message{ + Type: SetTopicMaxIndexMessageType, + Data: marshalTopicIndex(topicID, index), }) + return err } -func (b *Broker) mustApplySubscribe(m *Message) { - var c SubscribeCommand - mustUnmarshalJSON(m.Data, &c) - - // Retrieve replica. - r := b.replicas[c.ReplicaID] - if r == nil { - return - } - - // Save current index on topic. - t := b.mustCreateTopicIfNotExists(c.TopicID) - - // Ensure topic is not already subscribed to. - if _, ok := r.topics[c.TopicID]; ok { - b.Logger.Printf("already subscribed to topic: replica=%d, topic=%d", r.id, c.TopicID) - return - } - - // Add subscription to replica. - r.topics[c.TopicID] = struct{}{} - t.replicas[c.ReplicaID] = r - - // Catch up replica. - _ = t.writeTo(r) - - b.mustSave() -} +func (b *Broker) mustApplySetTopicMaxIndex(m *Message) { + topicID, index := unmarshalTopicIndex(m.Data) -// Unsubscribe removes a subscription for a topic from a replica. -func (b *Broker) Unsubscribe(replicaID, topicID uint64) error { b.mu.Lock() defer b.mu.Unlock() - // Ensure replica & topic exist. - if b.replicas[replicaID] == nil { - return ErrReplicaNotFound + // Ignore if the topic doesn't exist or the index is already higher. + t := b.topics[topicID] + if t == nil || t.index >= index { + return } - - // Issue command to unsubscribe from topic. - return b.PublishSync(&Message{ - Type: UnsubscribeMessageType, - Data: mustMarshalJSON(&UnsubscribeCommand{ReplicaID: replicaID, TopicID: topicID}), - }) + t.index = index } -func (b *Broker) mustApplyUnsubscribe(m *Message) { - var c UnsubscribeCommand - mustUnmarshalJSON(m.Data, &c) - - // Remove topic from replica. - if r := b.replicas[c.ReplicaID]; r != nil { - delete(r.topics, c.TopicID) - } - - // Remove replica from topic. - if t := b.topics[c.TopicID]; t != nil { - delete(t.replicas, c.ReplicaID) - } - - b.mustSave() +func marshalTopicIndex(topicID, index uint64) []byte { + b := make([]byte, 16) + binary.BigEndian.PutUint64(b[0:8], topicID) + binary.BigEndian.PutUint64(b[8:16], index) + return b } -// ReplicaIndex returns the highest received index of a replica. -func (b *Broker) ReplicaIndex(id uint64) (uint64, error) { - b.mu.Lock() - defer b.mu.Unlock() - - // Ensure replica exists. - r := b.replicas[id] - if r == nil { - return 0, ErrReplicaNotFound - } - return r.index, nil -} - -// Heartbeat records a heartbeat from a replica. -// The heartbeat is transient and only stored on the leader. It is used for -// truncating the broker log segments but truncation can only occur if the -// broker has current heartbeats from all replicas. -func (b *Broker) Heartbeat(id, index uint64) error { - b.mu.Lock() - defer b.mu.Unlock() - - // Ignore if the broker is not the leader. - if !b.IsLeader() { - return raft.ErrNotLeader - } - - // Find replica. - r := b.replicas[id] - if r == nil { - return ErrReplicaNotFound - } - - // Update its highest index received. - r.index = index - return nil +func unmarshalTopicIndex(b []byte) (topicID, index uint64) { + topicID = binary.BigEndian.Uint64(b[0:8]) + index = binary.BigEndian.Uint64(b[8:16]) + return } // Truncate removes log segments that have been replicated to all subscribed replicas. @@ -648,35 +395,42 @@ func (b *Broker) Truncate() error { b.mu.Lock() defer b.mu.Unlock() - // Loop over every topic. - for _, t := range b.topics { - // Determine the highest index replicated to all subscribed replicas. - minReplicaTopicIndex := b.minReplicaTopicIndex(t.id) - - // Loop over segments and close as needed. - newSegments := make(segments, 0, len(t.segments)) - for i, s := range t.segments { - // Find the next segment so we can find the upper index bound. - var next *segment - if i < len(t.segments)-1 { - next = t.segments[i+1] - } + // TODO: Generate a list of all segments. + // TODO: Sort by index. + // TODO: Delete segments until we reclaim enough space. + // TODO: Add tombstone for the last index. + + /* + // Loop over every topic. + for _, t := range b.topics { + // Determine the highest index replicated to all subscribed replicas. + minReplicaTopicIndex := b.minReplicaTopicIndex(t.id) + + // Loop over segments and close as needed. + newSegments := make(segments, 0, len(t.segments)) + for i, s := range t.segments { + // Find the next segment so we can find the upper index bound. + var next *segment + if i < len(t.segments)-1 { + next = t.segments[i+1] + } - // Ignore the last segment or if the next index is less than - // the highest index replicated across all replicas. - if next == nil || minReplicaTopicIndex < next.index { - newSegments = append(newSegments, s) - continue - } + // Ignore the last segment or if the next index is less than + // the highest index replicated across all replicas. + if next == nil || minReplicaTopicIndex < next.index { + newSegments = append(newSegments, s) + continue + } - // Remove the segment if the replicated index has moved pasted - // all the entries inside this segment. - s.close() - if err := os.Remove(s.path); err != nil && !os.IsNotExist(err) { - return fmt.Errorf("remove segment: topic=%d, segment=%d, err=%s", t.id, s.index, err) + // Remove the segment if the replicated index has moved pasted + // all the entries inside this segment. + s.close() + if err := os.Remove(s.path); err != nil && !os.IsNotExist(err) { + return fmt.Errorf("remove segment: topic=%d, segment=%d, err=%s", t.id, s.index, err) + } } } - } + */ return nil } @@ -690,46 +444,42 @@ type brokerFSM Broker func (fsm *brokerFSM) MustApply(e *raft.LogEntry) { b := (*Broker)(fsm) - // Create a message with the same index as Raft. - m := &Message{} - // Decode commands into messages. - // Convert internal raft entries to no-ops to move the index forward. + m := &Message{} if e.Type == raft.LogEntryCommand { - // Decode the message from the raft log. err := m.UnmarshalBinary(e.Data) assert(err == nil, "message unmarshal: %s", err) - - // Update the broker configuration. - switch m.Type { - case CreateReplicaMessageType: - b.mustApplyCreateReplica(m) - case DeleteReplicaMessageType: - b.mustApplyDeleteReplica(m) - case SubscribeMessageType: - b.mustApplySubscribe(m) - case UnsubscribeMessageType: - b.mustApplyUnsubscribe(m) - } } else { - // Internal raft commands should be broadcast out as no-ops. - m.TopicID = BroadcastTopicID m.Type = InternalMessageType } - - // Set the raft index. m.Index = e.Index - // Write to the topic. - t := b.mustCreateTopicIfNotExists(m.TopicID) - if err := t.encode(m); err != nil { - panic("encode: " + err.Error()) + // Process internal commands separately than the topic writes. + switch m.Type { + case InternalMessageType: + b.mustApplyInternal(m) + case SetTopicMaxIndexMessageType: + b.mustApplySetTopicMaxIndex(m) + default: + t := b.mustCreateTopicIfNotExists(m.TopicID) + if err := t.encode(m); err != nil { + panic("encode: " + err.Error()) + } } - // Save highest applied index. + // Save highest applied index in memory. + // Only internal messages need to have their indexes saved to disk. b.index = e.Index } +// mustApplyInternal updates the highest index applied to the broker. +func (b *Broker) mustApplyInternal(m *Message) { + err := b.meta.Update(func(tx *bolt.Tx) error { + return tx.Bucket([]byte("meta")).Put([]byte("index"), u64tob(m.Index)) + }) + assert(err == nil, "apply internal message: idx=%d, err=%s", m.Index, err) +} + // Index returns the highest index that the broker has seen. func (fsm *brokerFSM) Index() (uint64, error) { b := (*Broker)(fsm) @@ -800,9 +550,8 @@ func (fsm *brokerFSM) Restore(r io.Reader) error { return fmt.Errorf("decode header: %s", err) } - // Close any topics and replicas which might be open and clear them out. + // Close any topics which might be open and clear them out. b.closeTopics() - b.closeReplicas() // Copy topic files from snapshot to local disk. for _, st := range sh.Topics { @@ -842,18 +591,6 @@ func (fsm *brokerFSM) Restore(r io.Reader) error { } } - // Update the replicas. - for _, sr := range sh.Replicas { - // Create replica. - r := newReplica(b, sr.ID, sr.URL) - b.replicas[r.id] = r - - // Append replica's topics. - for _, topicID := range sr.TopicIDs { - r.topics[topicID] = struct{}{} - } - } - return nil } @@ -872,15 +609,8 @@ func copyFileN(w io.Writer, path string, n int64) (int64, error) { // snapshotHeader represents the header of a snapshot. type snapshotHeader struct { - Replicas []*snapshotReplica `json:"replicas"` - Topics []*snapshotTopic `json:"topics"` - Index uint64 `json:"index"` -} - -type snapshotReplica struct { - ID uint64 `json:"id"` - TopicIDs []uint64 `json:"topicIDs"` - URL string `json:"url"` + Topics []*snapshotTopic `json:"topics"` + Index uint64 `json:"index"` } type snapshotTopic struct { @@ -899,16 +629,12 @@ type snapshotTopicSegment struct { // Each topic is identified by a unique path. // // Topics write their entries to segmented log files which contain a -// contiguous range of entries. These segments are periodically dropped -// as data is replicated the replicas and the replicas heartbeat back -// a confirmation of receipt. +// contiguous range of entries. type topic struct { id uint64 // unique identifier - index uint64 // highest index written + index uint64 // highest index replicated path string // on-disk path segments segments // list of available segments - - replicas map[uint64]*Replica // replicas subscribed to topic } // segmentPath returns the path to a segment starting with a given log index. @@ -986,41 +712,44 @@ func (t *topic) Close() error { return nil } -// loadIndex reads the highest available index for a topic from disk. -func (t *topic) loadIndex() error { - // Open topic file for reading. +// maxIndex reads the highest available index for a topic from disk. +func (t *topic) maxIndex() (uint64, error) { + // Ignore if there are no available segments. + if len(t.segments) == 0 { + return 0, nil + } + + // Open last segment for reading. f, err := os.Open(t.segments.last().path) if os.IsNotExist(err) { - return nil + return 0, nil } else if err != nil { - return err + return 0, err } defer func() { _ = f.Close() }() // Read all messages. + index := uint64(0) dec := NewMessageDecoder(bufio.NewReader(f)) for { // Decode message. var m Message if err := dec.Decode(&m); err == io.EOF { - return nil + return index, nil } else if err != nil { - return fmt.Errorf("decode: %s", err) + return index, fmt.Errorf("decode: %s", err) } // Update the topic's highest index. - t.index = m.Index + index = m.Index } } // writeTo writes the topic to a replica. Only writes messages after replica index. // Returns an error if the starting index is unavailable. -func (t *topic) writeTo(r *Replica) error { +func (t *topic) writeTo(w io.Writer, index uint64) error { // TODO: If index is too old then return an error. - // Retrieve the replica's highest received index. - index := r.index - // Loop over each segment and write if it contains entries after index. segments := t.segments for i, s := range segments { @@ -1035,8 +764,8 @@ func (t *topic) writeTo(r *Replica) error { continue } - // Otherwise write segment to replica. - if err := t.writeSegmentTo(r, index, s); err != nil { + // Otherwise write segment. + if err := t.writeSegmentTo(w, index, s); err != nil { return fmt.Errorf("write segment(%d/%d): %s", t.id, s.index, err) } } @@ -1044,7 +773,7 @@ func (t *topic) writeTo(r *Replica) error { return nil } -func (t *topic) writeSegmentTo(r *Replica, index uint64, segment *segment) error { +func (t *topic) writeSegmentTo(w io.Writer, index uint64, segment *segment) error { // Open segment for reading. // If it doesn't exist then just exit immediately. f, err := os.Open(segment.path) @@ -1072,7 +801,7 @@ func (t *topic) writeSegmentTo(r *Replica, index uint64, segment *segment) error } // Write message out to stream. - _, err := m.WriteTo(r) + _, err := m.WriteTo(w) if err != nil { return fmt.Errorf("write to: %s", err) } @@ -1083,9 +812,6 @@ func (t *topic) writeSegmentTo(r *Replica, index uint64, segment *segment) error // encode writes a message to the end of the topic. func (t *topic) encode(m *Message) error { - // Ensure message is in-order. - assert(m.Index > t.index, "topic message out of order: %d -> %d", t.index, m.Index) - // Retrieve the last segment. s := t.segments.last() @@ -1115,14 +841,6 @@ func (t *topic) encode(m *Message) error { return fmt.Errorf("write segment: %s", err) } - // Move up high water mark on the topic. - t.index = m.Index - - // Write message out to all replicas. - for _, r := range t.replicas { - _, _ = r.Write(b) - } - return nil } @@ -1171,165 +889,130 @@ func (a segments) Len() int { return len(a) } func (a segments) Less(i, j int) bool { return a[i].index < a[j].index } func (a segments) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -// replicas represents a sortable list of replicas. -type replicas []*Replica - -func (a replicas) Len() int { return len(a) } -func (a replicas) Less(i, j int) bool { return a[i].id < a[j].id } -func (a replicas) Swap(i, j int) { a[i], a[j] = a[j], a[i] } - -// Replica represents a collection of subscriptions to topics on the broker. -// The replica maintains the highest index read for each topic so that the -// broker can use this high water mark for trimming the topic logs. -type Replica struct { - URL *url.URL +// topicReader reads data on a single topic from a given index. +type topicReader struct { + mu sync.Mutex + path string // topic directory path + index uint64 // starting index + streaming bool // true if reader should wait indefinitely - id uint64 - broker *Broker - index uint64 // highest index replicated to the replica. - - writer io.Writer // currently attached writer - done chan struct{} // notify when current writer is removed - - topics map[uint64]struct{} // set of subscribed topics. + segment uint64 // current segment index + file *os.File // current segment file handler + closed bool } -// newReplica returns a new Replica instance associated with a broker. -func newReplica(b *Broker, id uint64, urlstr string) *Replica { - // get the url of the replica - u, err := url.Parse(urlstr) +// Read reads the next bytes from the reader into the buffer. +func (r *topicReader) Read(p []byte) (int, error) { + // Retrieve current segment file handle. + f, err := r.File() if err != nil { - panic(err.Error()) + return 0, fmt.Errorf("file: %s", err) } - return &Replica{ - URL: u, - broker: b, - id: id, - topics: make(map[uint64]struct{}), + // Read from underlying file. + for { + // Write data to buffer. + // If no more data is available, then retry with the next segment. + if n, err := r.Read(p); err == io.EOF { + f, err = r.NextFile() + if err != nil { + return fmt.Errorf("next: %s", err) + } + continue + } else { + return n, err + } } } -// closeWriter removes the writer on the replica and closes the notify channel. -func (r *Replica) closeWriter() { - if r.writer != nil { - r.writer = nil - close(r.done) - r.done = nil - } -} +// File returns the current segment file handle. +// Returns nil when there is no more data left. +func (r *topicReader) File() (*os.File, error) { + r.mu.Lock() + defer r.mu.Unlock() -// Topics returns a list of topic names that the replica is subscribed to. -func (r *Replica) Topics() []uint64 { - a := make([]uint64, 0, len(r.topics)) - for topicID := range r.topics { - a = append(a, topicID) + // Exit if closed. + if r.closed { + return errors.New("topic reader closed") } - sort.Sort(uint64Slice(a)) - return a -} -// Write writes a byte slice to the underlying writer. -// If no writer is available then ErrReplicaUnavailable is returned. -func (r *Replica) Write(p []byte) (int, error) { - // Check if there's a replica available. - if r.writer == nil { - return 0, errReplicaUnavailable - } + // If the first file hasn't been opened then open it and seek. + if r.file == nil { + // Find the segment containing the index. + segment, err := r.segmentByIndex(r.index) + if err != nil { + return fmt.Errorf("segment by index: %s", err) + } - // If an error occurs on the write then remove the writer. - n, err := r.writer.Write(p) - if err != nil { - r.closeWriter() - return n, errReplicaUnavailable - } + // Open that segment file. + f, err := os.Open(filepath.Join(r.path, strconv.FormatUint(segment, 10))) + if err != nil { + return fmt.Errorf("open: %s", err) + } - // If the writer has a flush method then call it. - if w, ok := r.writer.(flusher); ok { - w.Flush() + // Seek to index. + if err := r.seekAfterIndex(f); err != nil { + _ = f.Close() + return fmt.Errorf("seek to index: %s", err) + } + + // Save file handle and segment name. + r.file = f + r.segment = segment } - return n, nil + return r.file } -// WriteTo begins writing messages to a named stream. -// Only one writer is allowed on a stream at a time. -func (r *Replica) WriteTo(w io.Writer) (int64, error) { - // Close previous writer, if set. - r.closeWriter() - - // Set a new writer on the replica. - r.writer = w - done := make(chan struct{}) - r.done = done - - // Create a topic list with the "config" topic first. - // Configuration changes need to be propagated to make sure topics exist. - ids := make([]uint64, 0, len(r.topics)) - for topicID := range r.topics { - ids = append(ids, topicID) +// seekAfterIndex moves a segment file to the message after a given index. +func (r *topicReader) seekAfterIndex(f *os.File, seek uint64) error { + dec := NewMessageDecoder(f) + for { + var m Message + if err := dec.Decode(&m); err == io.EOF || m.Index >= seek { + return nil + } else if err != nil { + return err + } } - sort.Sort(uint64Slice(ids)) +} - // Catch up and attach replica to all subscribed topics. - for _, topicID := range ids { - // Find topic. - t := r.broker.topics[topicID] - assert(t != nil, "topic missing: %s", topicID) +// NextFile closes the current segment's file handle and opens the next segment. +func (r *topicReader) NextFile() (*os.File, error) { + r.mu.Lock() + defer r.mu.Unlock() - // Write topic messages to replica. - if err := t.writeTo(r); err != nil { - r.closeWriter() - return 0, fmt.Errorf("add stream writer: %s", err) - } - - // Attach replica to topic to tail new messages. - t.replicas[r.id] = r - } + // - // Wait for writer to close and then return. - <-done - return 0, nil } -// CreateReplica creates a new replica. -type CreateReplicaCommand struct { - ID uint64 `json:"id"` - URL string `json:"url"` -} +// Close closes the reader. +func (r *topicReader) Close() error { + r.mu.Lock() + defer r.mu.Unlock() -// DeleteReplicaCommand removes a replica. -type DeleteReplicaCommand struct { - ID uint64 `json:"id"` -} + // Close current handle. + if r.file != nil { + _ = r.file.Close() + r.file = nil + } -// SubscribeCommand subscribes a replica to a new topic. -type SubscribeCommand struct { - ReplicaID uint64 `json:"replicaID"` // replica id - TopicID uint64 `json:"topicID"` // topic id -} + // Mark reader as closed. + r.closed = true -// UnsubscribeCommand removes a subscription for a topic from a replica. -type UnsubscribeCommand struct { - ReplicaID uint64 `json:"replicaID"` // replica id - TopicID uint64 `json:"topicID"` // topic id + return } // MessageType represents the type of message. type MessageType uint16 -const ( - BrokerMessageType = 0x8000 -) +// BrokerMessageType is a flag set on broker messages to prevent them +// from being passed through to topics. +const BrokerMessageType = 0x8000 const ( - InternalMessageType = BrokerMessageType | MessageType(0x00) - - CreateReplicaMessageType = BrokerMessageType | MessageType(0x10) - DeleteReplicaMessageType = BrokerMessageType | MessageType(0x11) - - SubscribeMessageType = BrokerMessageType | MessageType(0x20) - UnsubscribeMessageType = BrokerMessageType | MessageType(0x21) + InternalMessageType = BrokerMessageType | MessageType(0x00) + SetTopicMaxIndexMessageType = BrokerMessageType | MessageType(0x01) ) // The size of the encoded message header, in bytes. @@ -1458,5 +1141,15 @@ func assert(condition bool, msg string, v ...interface{}) { } } +// u64tob converts a uint64 into an 8-byte slice. +func u64tob(v uint64) []byte { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, v) + return b +} + +// btou64 converts an 8-byte slice into an uint64. +func btou64(b []byte) uint64 { return binary.BigEndian.Uint64(b) } + func warn(v ...interface{}) { fmt.Fprintln(os.Stderr, v...) } func warnf(msg string, v ...interface{}) { fmt.Fprintf(os.Stderr, msg+"\n", v...) } diff --git a/messaging/broker_test.go b/messaging/broker_test.go index 58702ff9e2e..01315a99894 100644 --- a/messaging/broker_test.go +++ b/messaging/broker_test.go @@ -44,21 +44,9 @@ func TestBroker_Close_ErrClosed(t *testing.T) { // Ensure the broker can write messages to the appropriate topics. func TestBroker_Publish(t *testing.T) { - //TODO fix and renable test. currently racy - t.Skip() b := NewBroker(nil) defer b.Close() - // Create a new named replica. - if err := b.CreateReplica(2000, &url.URL{Host: "localhost"}); err != nil { - t.Fatalf("create replica: %s", err) - } - - // Subscribe replica to a topic. - if err := b.Subscribe(2000, 20); err != nil { - t.Fatalf("subscribe: %s", err) - } - // Write a message to the broker. index, err := b.Publish(&messaging.Message{Type: 100, TopicID: 20, Data: []byte("0000")}) if err != nil { diff --git a/messaging/handler.go b/messaging/handler.go index 2d31ae7f062..085eb2142d2 100644 --- a/messaging/handler.go +++ b/messaging/handler.go @@ -1,9 +1,9 @@ package messaging import ( + "io" "io/ioutil" "net/http" - "net/url" "strconv" "strings" @@ -51,31 +51,15 @@ func (h *Handler) ServeHTTP(w http.ResponseWriter, r *http.Request) { switch r.URL.Path { case "/messaging/messages": if r.Method == "GET" { - h.stream(w, r) + h.getMessages(w, r) } else if r.Method == "POST" { - h.publish(w, r) - } else { - http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed) - } - case "/messaging/replicas": - if r.Method == "POST" { - h.createReplica(w, r) - } else if r.Method == "DELETE" { - h.deleteReplica(w, r) - } else { - http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed) - } - case "/messaging/subscriptions": - if r.Method == "POST" { - h.subscribe(w, r) - } else if r.Method == "DELETE" { - h.unsubscribe(w, r) + h.postMessages(w, r) } else { http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed) } case "/messaging/heartbeat": if r.Method == "POST" { - h.heartbeat(w, r) + h.postHeartbeat(w, r) } else { http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed) } @@ -84,197 +68,90 @@ func (h *Handler) ServeHTTP(w http.ResponseWriter, r *http.Request) { } } -// connects the requestor as the replica's writer. -func (h *Handler) stream(w http.ResponseWriter, r *http.Request) { - // Read the replica ID. - var replicaID uint64 - if n, err := strconv.ParseUint(r.URL.Query().Get("replicaID"), 10, 64); err != nil { - h.error(w, ErrReplicaIDRequired, http.StatusBadRequest) - return - } else { - replicaID = uint64(n) - } - - // Find the replica on the broker. - replica := h.broker.Replica(replicaID) - if replica == nil { - h.error(w, ErrReplicaNotFound, http.StatusNotFound) - return - } - - // Connect the response writer to the replica. - // This will block until the replica is closed or a new writer connects. - _, _ = replica.WriteTo(w) -} - -// publishes a message to the broker. -func (h *Handler) publish(w http.ResponseWriter, r *http.Request) { - m := &Message{} - - // Read the message type. - if n, err := strconv.ParseUint(r.URL.Query().Get("type"), 10, 16); err != nil { - h.error(w, ErrMessageTypeRequired, http.StatusBadRequest) - return - } else { - m.Type = MessageType(n) - } - +// getMessages streams messages from a topic. +func (h *Handler) getMessages(w http.ResponseWriter, req *http.Request) { // Read the topic ID. - if n, err := strconv.ParseUint(r.URL.Query().Get("topicID"), 10, 64); err != nil { + topicID, err := strconv.ParseUint(req.URL.Query().Get("topicID"), 10, 64) + if err != nil { h.error(w, ErrTopicRequired, http.StatusBadRequest) return - } else { - m.TopicID = uint64(n) } - // Read the request body. - data, err := ioutil.ReadAll(r.Body) + // Read the index to start from. + index, err := strconv.ParseUint(req.URL.Query().Get("index"), 10, 64) if err != nil { - h.error(w, err, http.StatusInternalServerError) - return - } - m.Data = data - - // Publish message to the broker. - index, err := h.broker.Publish(m) - if err == raft.ErrNotLeader { - h.redirectToLeader(w, r) - return - } else if err != nil { - h.error(w, err, http.StatusInternalServerError) + h.error(w, ErrIndexRequired, http.StatusBadRequest) return } - // Return index. - w.Header().Set("X-Broker-Index", strconv.FormatUint(index, 10)) -} + // Read the streaming flag. + streaming := (req.URL.Query().Get("streaming") == "true") -// createReplica creates a new replica with a given ID. -func (h *Handler) createReplica(w http.ResponseWriter, r *http.Request) { - // Read the replica ID. - var replicaID uint64 - if n, err := strconv.ParseUint(r.URL.Query().Get("id"), 10, 64); err != nil { - h.error(w, ErrReplicaIDRequired, http.StatusBadRequest) - return - } else { - replicaID = uint64(n) - } - u, err := url.Parse(r.URL.Query().Get("url")) + // Create a topic reader. + r, err := h.broker.OpenTopicReader(topicID, index, streaming) if err != nil { - h.error(w, err, http.StatusBadRequest) - return - } - - // Create a new replica on the broker. - if err := h.broker.CreateReplica(replicaID, u); err == raft.ErrNotLeader { - h.redirectToLeader(w, r) - return - } else if err == ErrReplicaExists { - h.error(w, err, http.StatusConflict) - return - } else if err != nil { h.error(w, err, http.StatusInternalServerError) return } - w.WriteHeader(http.StatusCreated) -} + defer r.Close() -// deleteReplica deletes an existing replica by ID. -func (h *Handler) deleteReplica(w http.ResponseWriter, r *http.Request) { - // Read the replica ID. - var replicaID uint64 - if n, err := strconv.ParseUint(r.URL.Query().Get("id"), 10, 64); err != nil { - h.error(w, ErrReplicaIDRequired, http.StatusBadRequest) - return - } else { - replicaID = uint64(n) + // Ensure we close the topic reader if the connection is disconnected. + if w, ok := w.(http.CloseNotifier); ok { + go func() { + select { + case <-w.CloseNotify(): + } + }() } - // Delete the replica on the broker. - if err := h.broker.DeleteReplica(replicaID); err == raft.ErrNotLeader { - h.redirectToLeader(w, r) - return - } else if err != nil { - h.error(w, err, http.StatusInternalServerError) - return - } - w.WriteHeader(http.StatusNoContent) + // Write out all data from the topic reader. + io.Copy(w, r) } -// subscribe creates a new subscription for a replica on a topic. -func (h *Handler) subscribe(w http.ResponseWriter, r *http.Request) { - // Read the replica ID. - var replicaID uint64 - if n, err := strconv.ParseUint(r.URL.Query().Get("replicaID"), 10, 64); err != nil { - h.error(w, ErrReplicaIDRequired, http.StatusBadRequest) +// postMessages publishes a message to the broker. +func (h *Handler) postMessages(w http.ResponseWriter, r *http.Request) { + // Read the message type. + typ, err := strconv.ParseUint(r.URL.Query().Get("type"), 10, 16) + if err != nil { + h.error(w, ErrMessageTypeRequired, http.StatusBadRequest) return - } else { - replicaID = uint64(n) } // Read the topic ID. - var topicID uint64 - if n, err := strconv.ParseUint(r.URL.Query().Get("topicID"), 10, 64); err != nil { + topicID, err := strconv.ParseUint(r.URL.Query().Get("topicID"), 10, 64) + if err != nil { h.error(w, ErrTopicRequired, http.StatusBadRequest) return - } else { - topicID = uint64(n) } - // Subscribe a replica to a topic. - if err := h.broker.Subscribe(replicaID, topicID); err == raft.ErrNotLeader { - h.redirectToLeader(w, r) - return - } else if err == ErrReplicaNotFound { - h.error(w, err, http.StatusNotFound) - return - } else if err != nil { + // Read the request body. + data, err := ioutil.ReadAll(r.Body) + if err != nil { h.error(w, err, http.StatusInternalServerError) return } - w.WriteHeader(http.StatusCreated) -} - -// unsubscribe removes a subscription from a replica for a topic. -func (h *Handler) unsubscribe(w http.ResponseWriter, r *http.Request) { - // Read the replica ID. - var replicaID uint64 - if n, err := strconv.ParseUint(r.URL.Query().Get("replicaID"), 10, 64); err != nil { - h.error(w, ErrReplicaIDRequired, http.StatusBadRequest) - return - } else { - replicaID = uint64(n) - } - // Read the topic ID. - var topicID uint64 - if n, err := strconv.ParseUint(r.URL.Query().Get("topicID"), 10, 64); err != nil { - h.error(w, ErrTopicRequired, http.StatusBadRequest) - return - } else { - topicID = uint64(n) - } - - // Unsubscribe the replica from the topic. - if err := h.broker.Unsubscribe(replicaID, topicID); err == raft.ErrNotLeader { + // Publish message to the broker. + index, err := h.broker.Publish(&Message{Type: MessageType(typ), TopicID: topicID, Data: data}) + if err == raft.ErrNotLeader { h.redirectToLeader(w, r) return - } else if err == ErrReplicaNotFound { - h.error(w, err, http.StatusNotFound) - return } else if err != nil { h.error(w, err, http.StatusInternalServerError) return } - w.WriteHeader(http.StatusNoContent) + + // Return index. + w.Header().Set("X-Broker-Index", strconv.FormatUint(index, 10)) } -// recieves a heartbeat from a client. -func (h *Handler) heartbeat(w http.ResponseWriter, r *http.Request) { - // Read the replica id. - replicaID, err := strconv.ParseUint(r.URL.Query().Get("replicaID"), 10, 16) +// postHearbeat receives a heartbeat from a client reporting the highest +// replicated index for a given topic. +func (h *Handler) postHeartbeat(w http.ResponseWriter, r *http.Request) { + // Read the topic id. + topicID, err := strconv.ParseUint(r.URL.Query().Get("topicID"), 10, 16) if err != nil { - h.error(w, ErrReplicaIDRequired, http.StatusBadRequest) + h.error(w, ErrTopicRequired, http.StatusBadRequest) return } @@ -285,8 +162,8 @@ func (h *Handler) heartbeat(w http.ResponseWriter, r *http.Request) { return } - // Update the replica's index. - if err := h.broker.Heartbeat(replicaID, index); err == raft.ErrNotLeader { + // Update the topic's highest replicated index. + if err := h.broker.SetTopicMaxIndex(topicID, index); err == raft.ErrNotLeader { h.redirectToLeader(w, r) return } else if err != nil { diff --git a/server.go b/server.go index 02044e4fc0e..621f321494e 100644 --- a/server.go +++ b/server.go @@ -43,6 +43,9 @@ const ( // DefaultShardRetention is the length of time before a shard is dropped. DefaultShardRetention = 7 * (24 * time.Hour) + + // BroadcastTopicID is the topic used for all metadata. + BroadcastTopicID = uint64(0) ) // Server represents a collection of metadata and raw metric data. From b937f06c1faa3ac216b6def1fc5249525a38f0b7 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Thu, 5 Mar 2015 19:32:20 -0700 Subject: [PATCH 04/20] Implementing stateless broker. --- messaging/broker.go | 662 ++++++++++++++++++++------------------ messaging/broker_test.go | 292 ++++++++--------- messaging/client_test.go | 175 +--------- messaging/errors.go | 9 + messaging/handler.go | 6 +- messaging/handler_test.go | 366 +++------------------ messaging/intg_test.go | 135 ++++---- 7 files changed, 611 insertions(+), 1034 deletions(-) diff --git a/messaging/broker.go b/messaging/broker.go index 0f362a431b8..84661401545 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -1,9 +1,9 @@ package messaging import ( - "bufio" "encoding/binary" "encoding/json" + "errors" "fmt" "io" "log" @@ -19,10 +19,6 @@ import ( "github.com/influxdb/influxdb/raft" ) -// MaxSegmentSize represents the largest size a segment can be before a -// new segment is started. -const MaxSegmentSize = 10 * 1024 * 1024 // 10MB - // Broker represents distributed messaging system segmented into topics. // Each topic represents a linear series of events. type Broker struct { @@ -32,7 +28,7 @@ type Broker struct { log *raft.Log // internal raft log meta *bolt.DB // metadata - topics map[uint64]*topic // topics by id + topics map[uint64]*Topic // topics by id Logger *log.Logger } @@ -41,7 +37,7 @@ type Broker struct { func NewBroker() *Broker { b := &Broker{ log: raft.NewLog(), - topics: make(map[uint64]*topic), + topics: make(map[uint64]*Topic), Logger: log.New(os.Stderr, "[broker] ", log.LstdFlags), } b.log.FSM = (*brokerFSM)(b) @@ -63,7 +59,14 @@ func (b *Broker) metaPath() string { return filepath.Join(b.path, "meta") } -// topicPath returns the file path to a topic's data. +// TopicPath returns the file path to a topic's data. +// Returns a blank string if the broker is closed. +func (b *Broker) TopicPath(id uint64) string { + b.mu.RLock() + defer b.mu.RUnlock() + return b.topicPath(id) +} + func (b *Broker) topicPath(id uint64) string { if b.path == "" { return "" @@ -105,9 +108,16 @@ func (b *Broker) Open(path string, u *url.URL) error { return ErrConnectionAddressRequired } + // Ensure root directory exists. + if err := os.MkdirAll(path, 0700); err != nil { + b.close() + return fmt.Errorf("mkdir: %s", err) + } + // Open meta file. meta, err := bolt.Open(b.metaPath(), 0600, &bolt.Options{Timeout: 1 * time.Second}) if err != nil { + b.close() return fmt.Errorf("open meta: %s", err) } b.meta = meta @@ -129,12 +139,8 @@ func (b *Broker) Open(path string, u *url.URL) error { // Read all topic metadata into memory. if err := b.openTopics(); err != nil { - return fmt.Errorf("load topics: %s", err) - } - - // Read the highest index from each of the topic files. - if err := b.loadIndex(); err != nil { - return fmt.Errorf("load index: %s", err) + b.close() + return fmt.Errorf("open topics: %s", err) } // Open underlying raft log. @@ -152,19 +158,47 @@ func (b *Broker) Open(path string, u *url.URL) error { // loadTopics reads all topic metadata into memory. func (b *Broker) openTopics() error { - // TODO: Determine topic metadata from directory listing. - panic("not yet implemented") -} + // Open handle to directory. + f, err := os.Open(b.path) + if err != nil { + return err + } + defer func() { _ = f.Close() }() + + // Read directory items. + fis, err := f.Readdir(0) + if err != nil { + return err + } + + // Create a topic for each directory with a numeric name. + for _, fi := range fis { + // Ignore non-directory entries. + if !fi.IsDir() { + continue + } + + // Filename must be numeric. + topicID, err := strconv.ParseUint(fi.Name(), 10, 64) + if err != nil { + continue + } + + // Create and open topic. + t := NewTopic(topicID) + if err := t.Open(filepath.Join(b.path, fi.Name())); err != nil { + return fmt.Errorf("open topic: id=%d, err=%s", topicID, err) + } + b.topics[t.id] = t + } -// loadIndex reads through all topics to find the highest known index. -func (b *Broker) loadIndex() error { + // Retrieve the highest index across all topics. for _, t := range b.topics { - if topicIndex, err := t.maxIndex(); err != nil { - return fmt.Errorf("topic max index: topic=%d, err=%s", t.id, err) - } else if topicIndex > b.index { - b.index = topicIndex + if t.index > b.index { + b.index = t.index } } + return nil } @@ -196,7 +230,7 @@ func (b *Broker) closeTopics() { for _, t := range b.topics { _ = t.Close() } - b.topics = make(map[uint64]*topic) + b.topics = make(map[uint64]*Topic) } // createSnapshotHeader creates a snapshot header. @@ -209,11 +243,17 @@ func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) { // Create snapshot topic. st := &snapshotTopic{ID: t.id} + // TODO: Read segments from disk, not topic. + segments, err := ReadSegments(t.path) + if err != nil { + return nil, fmt.Errorf("read segments: %s", err) + } + // Add segments to topic. - for _, s := range t.segments { + for _, s := range segments { // Retrieve current segment file size from disk. var size int64 - fi, err := os.Stat(s.path) + fi, err := os.Stat(s.Path) if os.IsNotExist(err) { size = 0 } else if err == nil { @@ -224,15 +264,15 @@ func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) { // Append segment. st.Segments = append(st.Segments, &snapshotTopicSegment{ - Index: s.index, + Index: s.Index, Size: size, - path: s.path, + path: s.Path, }) + } - // Bump the snapshot header max index. - if s.index > sh.Index { - sh.Index = s.index - } + // Bump the snapshot header max index. + if t.index > sh.Index { + sh.Index = t.index } // Append topic to the snapshot. @@ -299,59 +339,6 @@ func (b *Broker) PublishSync(m *Message) error { // Sync pauses until the given index has been applied. func (b *Broker) Sync(index uint64) error { return b.log.Wait(index) } -// initializes a new topic object. Requires lock. -func (b *Broker) newTopic(id uint64) *topic { - t := &topic{id: id, path: b.topicPath(id)} - b.topics[t.id] = t - return t -} - -// creates and opens a topic if it doesn't already exist. Requires lock. -func (b *Broker) createTopicIfNotExists(id uint64) (*topic, error) { - if t := b.topics[id]; t != nil { - return t, nil - } - - // Open topic. - t := b.newTopic(id) - if err := t.open(); err != nil { - return nil, fmt.Errorf("open topic: %s", err) - } - - return t, nil -} - -func (b *Broker) mustCreateTopicIfNotExists(id uint64) *topic { - t, err := b.createTopicIfNotExists(id) - if err != nil { - panic(err.Error()) - } - return t -} - -// OpenTopicReader returns a reader on a topic that starts from a given index. -// -// If streaming is true then the reader is held open indefinitely and waits -// for new messages on the topic. If streaming is false then the reader will -// return EOF at the end of the topic. -func (b *Broker) OpenTopicReader(topicID, index uint64, streaming bool) (io.ReadCloser, error) { - b.mu.RLock() - defer b.mu.RUnlock() - - // Exit if the broker is closed. - if !b.opened() { - return nil, ErrClosed - } - - // Return new reader. - r := &topicReader{ - path: b.topicPath(topicID), - index: index, - streaming: streaming, - } - return r, nil -} - // SetTopicMaxIndex updates the highest replicated index for a topic. // If a higher index is already set on the topic then the call is ignored. // This index is only held in memory and is used for topic segment reclamation. @@ -407,10 +394,10 @@ func (b *Broker) Truncate() error { minReplicaTopicIndex := b.minReplicaTopicIndex(t.id) // Loop over segments and close as needed. - newSegments := make(segments, 0, len(t.segments)) + newSegments := make(Segments, 0, len(t.segments)) for i, s := range t.segments { // Find the next segment so we can find the upper index bound. - var next *segment + var next *Segment if i < len(t.segments)-1 { next = t.segments[i+1] } @@ -461,9 +448,19 @@ func (fsm *brokerFSM) MustApply(e *raft.LogEntry) { case SetTopicMaxIndexMessageType: b.mustApplySetTopicMaxIndex(m) default: - t := b.mustCreateTopicIfNotExists(m.TopicID) - if err := t.encode(m); err != nil { - panic("encode: " + err.Error()) + // Create topic if not exists. + t := b.topics[m.TopicID] + if t == nil { + t = NewTopic(m.TopicID) + if err := t.Open(b.topicPath(t.id)); err != nil { + panic("open topic: " + err.Error()) + } + b.topics[t.id] = t + } + + // Write message to topic. + if err := t.WriteMessage(m); err != nil { + panic("write message: " + err.Error()) } } @@ -534,6 +531,13 @@ func (fsm *brokerFSM) Restore(r io.Reader) error { b.mu.Lock() defer b.mu.Unlock() + // Remove and recreate broker path. + if err := os.RemoveAll(b.path); err != nil && !os.IsNotExist(err) { + return fmt.Errorf("remove all: %s", err) + } else if err = os.MkdirAll(b.path, 0700); err != nil { + return fmt.Errorf("mkdir: %s", err) + } + // Read header frame. var sz uint32 if err := binary.Read(r, binary.BigEndian, &sz); err != nil { @@ -555,16 +559,11 @@ func (fsm *brokerFSM) Restore(r io.Reader) error { // Copy topic files from snapshot to local disk. for _, st := range sh.Topics { - t := b.newTopic(st.ID) - - // Remove existing file if it exists. - if err := os.RemoveAll(t.path); err != nil && !os.IsNotExist(err) { - return err - } + t := NewTopic(st.ID) // Copy data from snapshot into segment files. // We don't instantiate the segments because that will be done - // automatically when calling open() on the topic. + // automatically when calling Open() on the topic. for _, ss := range st.Segments { if err := func() error { // Create a new file with the starting index. @@ -586,9 +585,10 @@ func (fsm *brokerFSM) Restore(r io.Reader) error { } // Open new empty topic file. - if err := t.open(); err != nil { + if err := t.Open(b.topicPath(t.id)); err != nil { return fmt.Errorf("open topic: %s", err) } + b.topics[t.id] = t } return nil @@ -625,298 +625,333 @@ type snapshotTopicSegment struct { path string } +// DefaultMaxSegmentSize is the largest a segment can get before starting a new segment. +const DefaultMaxSegmentSize = 10 * 1024 * 1024 // 10MB + // topic represents a single named queue of messages. // Each topic is identified by a unique path. // // Topics write their entries to segmented log files which contain a // contiguous range of entries. -type topic struct { - id uint64 // unique identifier - index uint64 // highest index replicated - path string // on-disk path - segments segments // list of available segments +type Topic struct { + mu sync.Mutex + id uint64 // unique identifier + index uint64 // highest index replicated + path string // on-disk path + file *os.File // last segment writer + + // The largest a segment can get before splitting into a new segment. + MaxSegmentSize int64 } -// segmentPath returns the path to a segment starting with a given log index. -func (t *topic) segmentPath(index uint64) string { - path := t.path - if path == "" { +// NewTopic returns a new instance of topic. +func NewTopic(id uint64) *Topic { + return &Topic{ + id: id, + MaxSegmentSize: DefaultMaxSegmentSize, + } +} + +// SegmentPath returns the path to a segment starting with a given log index. +func (t *Topic) SegmentPath(index uint64) string { + t.mu.Lock() + defer t.mu.Lock() + return t.segmentPath(index) +} + +func (t *Topic) segmentPath(index uint64) string { + if t.path == "" { return "" } - return filepath.Join(path, strconv.FormatUint(index, 10)) + return filepath.Join(t.path, strconv.FormatUint(index, 10)) } -// open opens a topic for writing. -func (t *topic) open() error { - assert(len(t.segments) == 0, "topic already open: %d", t.id) +// Open opens a topic for writing. +func (t *Topic) Open(path string) error { + t.mu.Lock() + defer t.mu.Unlock() + + // Ensure topic is not already open. + if t.path != "" { + return ErrTopicOpen + } + t.path = path // Ensure the parent directory exists. if err := os.MkdirAll(t.path, 0700); err != nil { + t.close() return err } // Read available segments. - if err := t.loadSegments(); err != nil { + segments, err := ReadSegments(t.path) + if err != nil { + t.close() return fmt.Errorf("read segments: %s", err) } - return nil -} + // Read max index and open file handle if we have segments. + if len(segments) > 0 { + s := segments.Last() -// loadSegments reads all available segments for the topic. -// At least one segment will always exist. -func (t *topic) loadSegments() error { - // Open handle to directory. - f, err := os.Open(t.path) - if err != nil { - return err - } - defer func() { _ = f.Close() }() - - // Read directory items. - fis, err := f.Readdir(0) - if err != nil { - return err - } + // Read the last segment and extract the last message index. + index, err := ReadSegmentMaxIndex(s.Path) + if err != nil { + t.close() + return fmt.Errorf("read segment max index: %s", err) + } + t.index = index - // Create a segment for each file with a numeric name. - var a segments - for _, fi := range fis { - index, err := strconv.ParseUint(fi.Name(), 10, 64) + // Open file handle on the segment. + f, err := os.OpenFile(s.Path, os.O_RDWR|os.O_APPEND, 0600) if err != nil { - continue + t.close() + return fmt.Errorf("open segment: %s", err) } - a = append(a, &segment{ - index: index, - path: t.segmentPath(index), - size: fi.Size(), - }) + t.file = f } - sort.Sort(a) - // Create a first segment if one doesn't exist. - if len(a) == 0 { - a = segments{&segment{index: 0, path: t.segmentPath(0), size: 0}} + return nil +} + +// Close closes the topic and segment writer. +func (t *Topic) Close() error { + t.mu.Lock() + defer t.mu.Unlock() + return t.close() +} + +func (t *Topic) close() error { + if t.file != nil { + _ = t.file.Close() + t.file = nil } - t.segments = a + t.path = "" + t.index = 0 return nil } -// close closes the underlying file. -func (t *topic) Close() error { - for _, s := range t.segments { - _ = s.close() +// ReadIndex reads the highest available index for a topic from disk. +func (t *Topic) ReadIndex() (uint64, error) { + // Read a list of all segments. + segments, err := ReadSegments(t.path) + if err != nil { + return 0, fmt.Errorf("read segments: %s", err) } - return nil -} -// maxIndex reads the highest available index for a topic from disk. -func (t *topic) maxIndex() (uint64, error) { // Ignore if there are no available segments. - if len(t.segments) == 0 { + if len(segments) == 0 { return 0, nil } - // Open last segment for reading. - f, err := os.Open(t.segments.last().path) - if os.IsNotExist(err) { - return 0, nil - } else if err != nil { - return 0, err + // Read highest index on the last segment. + index, err := ReadSegmentMaxIndex(segments.Last().Path) + if err != nil { + return 0, fmt.Errorf("read segment max index: %s", err) } - defer func() { _ = f.Close() }() - // Read all messages. - index := uint64(0) - dec := NewMessageDecoder(bufio.NewReader(f)) - for { - // Decode message. - var m Message - if err := dec.Decode(&m); err == io.EOF { - return index, nil - } else if err != nil { - return index, fmt.Errorf("decode: %s", err) - } - - // Update the topic's highest index. - index = m.Index - } + return index, nil } -// writeTo writes the topic to a replica. Only writes messages after replica index. -// Returns an error if the starting index is unavailable. -func (t *topic) writeTo(w io.Writer, index uint64) error { - // TODO: If index is too old then return an error. +// WriteMessage writes a message to the end of the topic. +func (t *Topic) WriteMessage(m *Message) error { + t.mu.Lock() + defer t.mu.Unlock() - // Loop over each segment and write if it contains entries after index. - segments := t.segments - for i, s := range segments { - // Determine the maximum index in the range. - var next *segment - if i < len(segments)-1 { - next = segments[i+1] - } + // Return error if message index is lower than the topic's highest index. + if m.Index <= t.index { + return ErrStaleWrite + } - // If the index is after the end of the segment then ignore. - if next != nil && index >= next.index { - continue + // Close the current file handle if it's too large. + if t.file != nil { + if fi, err := t.file.Stat(); err != nil { + return fmt.Errorf("stat: %s", err) + } else if fi.Size() > t.MaxSegmentSize { + _ = t.file.Close() + t.file = nil } + } - // Otherwise write segment. - if err := t.writeSegmentTo(w, index, s); err != nil { - return fmt.Errorf("write segment(%d/%d): %s", t.id, s.index, err) + // Create a new segment if we have no handle. + if t.file == nil { + f, err := os.OpenFile(t.segmentPath(m.Index), os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0600) + if err != nil { + return fmt.Errorf("create segment file: %s", err) } + t.file = f } - return nil -} + // Encode message. + b := make([]byte, messageHeaderSize+len(m.Data)) + copy(b, m.marshalHeader()) + copy(b[messageHeaderSize:], m.Data) -func (t *topic) writeSegmentTo(w io.Writer, index uint64, segment *segment) error { - // Open segment for reading. - // If it doesn't exist then just exit immediately. - f, err := os.Open(segment.path) - if os.IsNotExist(err) { - return nil - } else if err != nil { - return err + // Write to last segment. + if _, err := t.file.Write(b); err != nil { + return fmt.Errorf("write segment: %s", err) } - defer func() { _ = f.Close() }() - // Stream out all messages until EOF. - dec := NewMessageDecoder(bufio.NewReader(f)) - for { - // Decode message. - var m Message - if err := dec.Decode(&m); err == io.EOF { - break - } else if err != nil { - return fmt.Errorf("decode: %s", err) - } + return nil +} - // Ignore message if it's on or before high water mark. - if m.Index <= index { - continue - } +// Segment represents a contiguous section of a topic log. +type Segment struct { + Index uint64 // starting index of the segment and name + Path string // path to the segment file. +} - // Write message out to stream. - _, err := m.WriteTo(w) - if err != nil { - return fmt.Errorf("write to: %s", err) - } +// Size returns the file size of the segment. +func (s *Segment) Size() (int64, error) { + fi, err := os.Stat(s.Path) + if err != nil { + return 0, err } - - return nil + return fi.Size(), nil } -// encode writes a message to the end of the topic. -func (t *topic) encode(m *Message) error { - // Retrieve the last segment. - s := t.segments.last() +// Segments represents a list of segments sorted by index. +type Segments []*Segment - // Close the segment if it's too large. - if s.size > MaxSegmentSize { - s.close() - s = nil +// Last returns the last segment in the slice. +// Returns nil if there are no segments. +func (a Segments) Last() *Segment { + if len(a) == 0 { + return nil } + return a[len(a)-1] +} - // Create and append a new segment if we don't have one. - if s == nil { - t.segments = append(t.segments, &segment{index: m.Index, path: t.segmentPath(m.Index)}) - } - if s.file == nil { - if err := s.open(); err != nil { - return fmt.Errorf("open segment: %s", err) - } - } +func (a Segments) Len() int { return len(a) } +func (a Segments) Less(i, j int) bool { return a[i].Index < a[j].Index } +func (a Segments) Swap(i, j int) { a[i], a[j] = a[j], a[i] } - // Encode message. - b := make([]byte, messageHeaderSize+len(m.Data)) - copy(b, m.marshalHeader()) - copy(b[messageHeaderSize:], m.Data) +// ReadSegments reads all segments from a directory path. +func ReadSegments(path string) (Segments, error) { + // Open handle to directory. + f, err := os.Open(path) + if err != nil { + return nil, err + } + defer func() { _ = f.Close() }() - // Write to segment. - if _, err := s.file.Write(b); err != nil { - return fmt.Errorf("write segment: %s", err) + // Read directory items. + fis, err := f.Readdir(0) + if err != nil { + return nil, err } - return nil -} + // Create a segment for each file with a numeric name. + var a Segments + for _, fi := range fis { + index, err := strconv.ParseUint(fi.Name(), 10, 64) + if err != nil { + continue + } -// segment represents a contiguous section of a topic log. -type segment struct { - index uint64 // starting index of the segment and name - path string // path to the segment file. - size int64 // total size of the segment file, in bytes. + a = append(a, &Segment{ + Index: index, + Path: filepath.Join(path, fi.Name()), + }) + } + sort.Sort(a) - file *os.File // handle for writing, only open for last segment + return a, nil } -// open opens the file handle for append. -func (s *segment) open() error { - f, err := os.OpenFile(s.path, os.O_RDWR|os.O_CREATE|os.O_APPEND, 0600) +// ReadSegmentByIndex returns the segment that contains a given index. +func ReadSegmentByIndex(path string, index uint64) (*Segment, error) { + // Find a list of all segments. + segments, err := ReadSegments(path) if err != nil { - return err + return nil, fmt.Errorf("read segments: %s", err) } - s.file = f - return nil -} -// close closes the segment's writing file handle. -func (s *segment) close() error { - if s.file != nil { - err := s.file.Close() - s.file = nil - return err + // If there are no segments then ignore. + // If index is zero then start from the first segment. + // If index is less than the first segment range then return error. + if len(segments) == 0 { + return nil, nil + } else if index == 0 { + return segments[0], nil + } else if index < segments[0].Index { + return nil, ErrSegmentReclaimed } - return nil + + // Find segment that contains index. + for i := range segments[:len(segments)-1] { + if index >= segments[i].Index && index < segments[i+1].Index { + return segments[i], nil + } + } + + // If no segment ranged matched then return the last segment. + return segments[len(segments)-1], nil } -// segments represents a list of segments sorted by index. -type segments []*segment +// ReadSegmentMaxIndex returns the highest index recorded in a segment. +func ReadSegmentMaxIndex(path string) (uint64, error) { + // Open segment file. + f, err := os.Open(path) + if err != nil { + return 0, fmt.Errorf("open: %s", err) + } + defer func() { _ = f.Close() }() -// last returns the last segment in the slice. -// Returns nil if there are no elements. -func (a segments) last() *segment { - if len(a) == 0 { - return nil + // Read all messages until the end. + dec := NewMessageDecoder(f) + index := uint64(0) + for { + var m Message + if err := dec.Decode(&m); err == io.EOF { + return index, nil + } else if err != nil { + return 0, fmt.Errorf("decode: %s", err) + } + index = m.Index } - return a[len(a)-1] } -func (a segments) Len() int { return len(a) } -func (a segments) Less(i, j int) bool { return a[i].index < a[j].index } -func (a segments) Swap(i, j int) { a[i], a[j] = a[j], a[i] } - -// topicReader reads data on a single topic from a given index. -type topicReader struct { +// TopicReader reads data on a single topic from a given index. +type TopicReader struct { mu sync.Mutex path string // topic directory path index uint64 // starting index - streaming bool // true if reader should wait indefinitely + streaming bool // true if reader should hang and wait for new messages - segment uint64 // current segment index - file *os.File // current segment file handler - closed bool + file *os.File // current segment file handler + closed bool } -// Read reads the next bytes from the reader into the buffer. -func (r *topicReader) Read(p []byte) (int, error) { - // Retrieve current segment file handle. - f, err := r.File() - if err != nil { - return 0, fmt.Errorf("file: %s", err) +// NewTopicReader returns a new instance of TopicReader that reads segments +// from a path starting from a given index. +func NewTopicReader(path string, index uint64, streaming bool) *TopicReader { + return &TopicReader{ + path: path, + index: index, + streaming: streaming, } +} - // Read from underlying file. +// Read reads the next bytes from the reader into the buffer. +func (r *TopicReader) Read(p []byte) (int, error) { for { + // Retrieve current segment file handle. + f, err := r.File() + if err != nil { + return 0, fmt.Errorf("file: %s", err) + } else if f == nil { + return 0, io.EOF + } + // Write data to buffer. // If no more data is available, then retry with the next segment. if n, err := r.Read(p); err == io.EOF { f, err = r.NextFile() if err != nil { - return fmt.Errorf("next: %s", err) + return 0, fmt.Errorf("next: %s", err) } continue } else { @@ -927,45 +962,47 @@ func (r *topicReader) Read(p []byte) (int, error) { // File returns the current segment file handle. // Returns nil when there is no more data left. -func (r *topicReader) File() (*os.File, error) { +func (r *TopicReader) File() (*os.File, error) { r.mu.Lock() defer r.mu.Unlock() // Exit if closed. if r.closed { - return errors.New("topic reader closed") + return nil, errors.New("topic reader closed") } // If the first file hasn't been opened then open it and seek. if r.file == nil { // Find the segment containing the index. - segment, err := r.segmentByIndex(r.index) + // Exit if no segments are available. + segment, err := ReadSegmentByIndex(r.path, r.index) if err != nil { - return fmt.Errorf("segment by index: %s", err) + return nil, fmt.Errorf("segment by index: %s", err) + } else if segment == nil { + return nil, nil } // Open that segment file. - f, err := os.Open(filepath.Join(r.path, strconv.FormatUint(segment, 10))) + f, err := os.Open(filepath.Join(r.path, segment.Path)) if err != nil { - return fmt.Errorf("open: %s", err) + return nil, fmt.Errorf("open: %s", err) } // Seek to index. - if err := r.seekAfterIndex(f); err != nil { + if err := r.seekAfterIndex(f, r.index); err != nil { _ = f.Close() - return fmt.Errorf("seek to index: %s", err) + return nil, fmt.Errorf("seek to index: %s", err) } // Save file handle and segment name. r.file = f - r.segment = segment } - return r.file + return r.file, nil } // seekAfterIndex moves a segment file to the message after a given index. -func (r *topicReader) seekAfterIndex(f *os.File, seek uint64) error { +func (r *TopicReader) seekAfterIndex(f *os.File, seek uint64) error { dec := NewMessageDecoder(f) for { var m Message @@ -978,16 +1015,15 @@ func (r *topicReader) seekAfterIndex(f *os.File, seek uint64) error { } // NextFile closes the current segment's file handle and opens the next segment. -func (r *topicReader) NextFile() (*os.File, error) { +func (r *TopicReader) NextFile() (*os.File, error) { r.mu.Lock() defer r.mu.Unlock() - // - + panic("not yet implemented") } // Close closes the reader. -func (r *topicReader) Close() error { +func (r *TopicReader) Close() error { r.mu.Lock() defer r.mu.Unlock() @@ -1000,7 +1036,7 @@ func (r *topicReader) Close() error { // Mark reader as closed. r.closed = true - return + return nil } // MessageType represents the type of message. diff --git a/messaging/broker_test.go b/messaging/broker_test.go index 01315a99894..1ce5a910f05 100644 --- a/messaging/broker_test.go +++ b/messaging/broker_test.go @@ -7,9 +7,9 @@ import ( "io/ioutil" "net/url" "os" + "path/filepath" "reflect" "testing" - "time" "github.com/influxdb/influxdb/messaging" ) @@ -42,6 +42,7 @@ func TestBroker_Close_ErrClosed(t *testing.T) { } } +/* // Ensure the broker can write messages to the appropriate topics. func TestBroker_Publish(t *testing.T) { b := NewBroker(nil) @@ -109,110 +110,10 @@ func TestBroker_Publish(t *testing.T) { } } -// Ensure that creating a duplicate replica will return an error. -func TestBroker_CreateReplica_ErrReplicaExists(t *testing.T) { - b := NewBroker(nil) - defer b.Close() - - // Create a replica twice. - b.CreateReplica(2000, &url.URL{Host: "localhost"}) - if err := b.CreateReplica(2000, &url.URL{Host: "localhost"}); err != messaging.ErrReplicaExists { - t.Fatalf("unexpected error: %s", err) - } -} - -// Ensure the broker can remove an existing replica. -func TestBroker_DeleteReplica(t *testing.T) { - b := NewBroker(nil) - defer b.Close() - - // Create a new named replica. - if err := b.CreateReplica(2000, &url.URL{Host: "localhost"}); err != nil { - t.Fatalf("create replica: %s", err) - } - - // Attach a replica writer. - var buf bytes.Buffer - var closed bool - go func() { - if _, err := b.Replica(2000).WriteTo(&buf); err != nil { - t.Fatalf("write to: %s", err) - } - closed = true - }() - time.Sleep(10 * time.Millisecond) - - // Delete the replica. - if err := b.DeleteReplica(2000); err != nil { - t.Fatalf("delete replica: %s", err) - } - time.Sleep(10 * time.Millisecond) - - // Ensure the writer was closed. - if !closed { - t.Fatal("replica writer did not close") - } - - // Ensure the replica no longer exists. - if r := b.Replica(2000); r != nil { - t.Fatal("replica still exists") - } -} - -// Ensure an error is returned when deleting a non-existent replica. -func TestBroker_DeleteReplica_ErrReplicaNotFound(t *testing.T) { - b := NewBroker(nil) - defer b.Close() - if err := b.DeleteReplica(0); err != messaging.ErrReplicaNotFound { - t.Fatalf("unexpected error: %s", err) - } -} - -// Ensure that subscribing to a missing replica returns an error. -func TestBroker_Subscribe_ErrReplicaNotFound(t *testing.T) { - b := NewBroker(nil) - defer b.Close() - b.CreateReplica(2000, &url.URL{Host: "localhost"}) - if err := b.Subscribe(3000, 20); err != messaging.ErrReplicaNotFound { - t.Fatalf("unexpected error: %s", err) - } -} - -// Ensure that unsubscribing from a missing replica returns an error. -func TestBroker_Unsubscribe_ErrReplicaNotFound(t *testing.T) { - b := NewBroker(nil) - defer b.Close() - if err := b.Unsubscribe(0, 20); err != messaging.ErrReplicaNotFound { - t.Fatalf("unexpected error: %s", err) - } -} // Ensure the broker can reopen and recover correctly. func TestBroker_Reopen(t *testing.T) { - b := NewBroker(nil) - defer b.Close() - b.MustCreateReplica(2000, &url.URL{Host: "localhost"}) - b.MustSubscribe(2000, 20) - b.MustSubscribe(2000, 21) - b.MustPublishSync(&messaging.Message{TopicID: 20, Data: []byte("0000")}) - b.MustPublishSync(&messaging.Message{TopicID: 20, Data: []byte("0000")}) - b.MustPublishSync(&messaging.Message{TopicID: 21, Data: []byte("0000")}) - index := b.MustPublishSync(&messaging.Message{TopicID: 20, Data: []byte("0000")}) - time.Sleep(100 * time.Millisecond) - - // Close broker and reopen with a new broker instance. - path, u := b.Path(), b.URL() - b.Broker.Close() - b.Broker = messaging.NewBroker() - if err := b.Broker.Open(path, u); err != nil { - t.Fatal(err) - } - - // Verify the broker is up to date. - newIndex := b.Index() - if newIndex != index { - t.Fatalf("index mismatch: exp=%d, got=%d", index, newIndex) - } + t.Skip("pending") } // Benchmarks a single broker without HTTP. @@ -236,6 +137,116 @@ func BenchmarkBroker_Publish(b *testing.B) { b.Fatalf("sync error: %s", err) } } +*/ + +// Ensure a list of segments can be read from a directory. +func TestReadSegments(t *testing.T) { + path, _ := ioutil.TempDir("", "") + defer os.RemoveAll(path) + + MustWriteFile(filepath.Join(path, "12"), []byte{}) + MustWriteFile(filepath.Join(path, "118332"), []byte{}) + MustWriteFile(filepath.Join(path, "6"), []byte{}) + MustWriteFile(filepath.Join(path, "xxx"), []byte{}) + + segments, err := messaging.ReadSegments(path) + if err != nil { + t.Fatal(err) + } else if len(segments) != 3 { + t.Fatalf("unexpected segment count: %d", len(segments)) + } else if segments[0].Index != 6 { + t.Fatalf("unexpected segment(0) index: %d", segments[0].Index) + } else if segments[0].Path != filepath.Join(path, "6") { + t.Fatalf("unexpected segment(0) path: %s", segments[0].Path) + } else if segments[1].Index != 12 { + t.Fatalf("unexpected segment(1) index: %d", segments[1].Index) + } else if segments[2].Index != 118332 { + t.Fatalf("unexpected segment(2) index: %d", segments[2].Index) + } +} + +// Ensure the appropriate segment can be found by index. +func TestReadSegmentByIndex(t *testing.T) { + path, _ := ioutil.TempDir("", "") + defer os.RemoveAll(path) + + MustWriteFile(filepath.Join(path, "6"), []byte{}) + MustWriteFile(filepath.Join(path, "12"), []byte{}) + MustWriteFile(filepath.Join(path, "20"), []byte{}) + + for i, tt := range []struct { + index uint64 + segmentIndex uint64 + err error + }{ + {index: 0, segmentIndex: 6}, + {index: 5, segmentIndex: 6, err: messaging.ErrSegmentReclaimed}, + } { + segment, err := messaging.ReadSegmentByIndex(path, tt.index) + if err != nil { + t.Errorf("%d. %d: error: %s", i, tt.index, err) + } else if tt.segmentIndex != segment.Index { + t.Errorf("%d. %d: index mismatch: exp=%d got=%d", i, tt.index, tt.segmentIndex, segment.Index) + } + } +} + +// Ensure a topic reader can read messages in order from a given index. +func TestTopicReader(t *testing.T) { + path, _ := ioutil.TempDir("", "") + defer os.RemoveAll(path) + + // Generate segments in directory. + MustWriteFile(filepath.Join(path, "6"), + MustMarshalMessages([]*messaging.Message{ + {Index: 6}, + {Index: 7}, + {Index: 10}, + }), + ) + MustWriteFile(filepath.Join(path, "12"), + MustMarshalMessages([]*messaging.Message{ + {Index: 12}, + }), + ) + MustWriteFile(filepath.Join(path, "13"), + MustMarshalMessages([]*messaging.Message{ + {Index: 13}, + {Index: 14}, + }), + ) + + // Execute table tests. + for i, tt := range []struct { + index uint64 // starting index + results []uint64 // returned indices + }{ + {index: 0, results: []uint64{6, 7, 10, 12, 13, 14}}, + } { + // Start topic reader from an index. + r := messaging.NewTopicReader(path, tt.index, false) + + // Slurp all message ids from the reader. + results := make([]uint64, 0) + dec := messaging.NewMessageDecoder(r) + for { + m := &messaging.Message{} + if err := dec.Decode(m); err == io.EOF { + break + } else if err != nil { + t.Fatalf("%d. decode error: %s", i, err) + } else { + results = append(results, m.Index) + } + } + + // Verify the retrieved indices match what's expected. + if !reflect.DeepEqual(results, tt.results) { + t.Fatalf("%d. result mismatch:\n\nexp=%#v\n\ngot=%#v", i, tt.results, results) + } + } + +} // Broker is a wrapper for broker.Broker that creates the broker in a temporary location. type Broker struct { @@ -272,45 +283,6 @@ func (b *Broker) Close() { b.Broker.Close() } -// MustReadAll reads all available messages for a replica. Panic on error. -func (b *Broker) MustReadAll(replicaID uint64) (a []*messaging.Message) { - // Read message from the replica. - var buf bytes.Buffer - go func() { - if _, err := b.Replica(replicaID).WriteTo(&buf); err != nil { - panic("write to: " + err.Error()) - } - }() - time.Sleep(10 * time.Millisecond) - - // Read out the config messages first. - dec := messaging.NewMessageDecoder(&buf) - for { - m := &messaging.Message{} - if err := dec.Decode(m); err == io.EOF { - break - } else if err != nil { - panic("decode: " + err.Error()) - } - a = append(a, m) - } - return -} - -// MustCreateReplica creates a new replica. Panic on error. -func (b *Broker) MustCreateReplica(replicaID uint64, u *url.URL) { - if err := b.CreateReplica(replicaID, u); err != nil { - panic(err.Error()) - } -} - -// MustSubscribe subscribes a replica to a topic. Panic on error. -func (b *Broker) MustSubscribe(replicaID, topicID uint64) { - if err := b.Subscribe(replicaID, topicID); err != nil { - panic(err.Error()) - } -} - // MustSync syncs to a broker index. Panic on error. func (b *Broker) MustSync(index uint64) { if err := b.Sync(index); err != nil { @@ -354,26 +326,6 @@ func (a Messages) Last() *messaging.Message { return a[len(a)-1] } -// Broadcasted returns a filtered list of all broadcasted messages. -func (a Messages) Broadcasted() (other Messages) { - for _, m := range a { - if m.TopicID == 0 { - other = append(other, m) - } - } - return -} - -// Unicasted returns a filtered list of all non-broadcasted messages. -func (a Messages) Unicasted() (other Messages) { - for _, m := range a { - if m.TopicID != 0 { - other = append(other, m) - } - } - return -} - // tempfile returns a temporary path. func tempfile() string { f, _ := ioutil.TempFile("", "influxdb-messaging-") @@ -383,5 +335,23 @@ func tempfile() string { return path } +// MustWriteFile writes data to a file. Panic on error. +func MustWriteFile(filename string, data []byte) { + if err := ioutil.WriteFile(filename, data, 0600); err != nil { + panic(err.Error()) + } +} + +// MustMarshalMessages marshals a slice of messages to bytes. Panic on error. +func MustMarshalMessages(a []*messaging.Message) []byte { + var buf bytes.Buffer + for _, m := range a { + if _, err := m.WriteTo(&buf); err != nil { + panic(err.Error()) + } + } + return buf.Bytes() +} + func warn(v ...interface{}) { fmt.Fprintln(os.Stderr, v...) } func warnf(msg string, v ...interface{}) { fmt.Fprintf(os.Stderr, msg+"\n", v...) } diff --git a/messaging/client_test.go b/messaging/client_test.go index 811c8b6ba79..62f6c0cf010 100644 --- a/messaging/client_test.go +++ b/messaging/client_test.go @@ -2,32 +2,20 @@ package messaging_test import ( "io/ioutil" - "net/url" - "os" - "reflect" - "strings" "testing" "time" "github.com/influxdb/influxdb/messaging" ) -// Ensure the client replica id can be retrieved. -func TestClient_ReplicaID(t *testing.T) { - c := NewClient(1000) - defer c.Close() - if replicaID := c.ReplicaID(); replicaID != 1000 { - t.Fatalf("unexpected replica id: %d", replicaID) - } -} - +/* // Ensure that a client can open a connect to the broker. func TestClient_Open(t *testing.T) { - c := NewClient(1000) + c := NewClient() defer c.Close() // Create replica on broker. - c.Server.Handler.Broker().CreateReplica(1000, &url.URL{Host: "localhost"}) + c.Server.Handler.Broker().PublishSync() // Open client to broker. f := NewTempFile() @@ -136,177 +124,26 @@ func TestClient_Publish_ErrLogClosed(t *testing.T) { t.Fatalf("unexpected error: %v", err) } } +*/ // Ensure that a client can heartbeat its status to the broker. -func TestClient_Heartbeat(t *testing.T) { - c := OpenClient(1000) - defer c.Close() - - // Publish message to the broker. - index := c.MustPublish(&messaging.Message{Type: 100, TopicID: messaging.BroadcastTopicID, Data: []byte{}}) - - // Flush the channel until we receive our message. -loop: - for { - select { - case m := <-c.C(): - if m.Index == index { - break loop - } - } - } - - // Wait until the index moves forward. - c.Sync(index) - - // Send heartbeat to the broker. - if err := c.Heartbeat(); err != nil { - t.Fatal(err) - } - - // Verify the broker received the heartbeat. - if replicaIndex, err := c.Server.Broker().ReplicaIndex(1000); err != nil { - t.Fatal(err) - } else if replicaIndex != index { - t.Fatalf("unexpected replica index: exp=%d, got=%d", index, replicaIndex) - } -} - -// Ensure that a client can create a replica. -func TestClient_CreateReplica(t *testing.T) { - c := OpenClient(0) - defer c.Close() - - // Create replica through client. - if err := c.CreateReplica(123, &url.URL{Host: "localhost"}); err != nil { - t.Fatalf("unexpected error: %v", err) - } - - // Verify replica was created. - if r := c.Server.Handler.Broker().Replica(123); r == nil { - t.Fatalf("replica not created") - } -} - -// Ensure that a client can passthrough an error while creating a replica. -func TestClient_CreateReplica_Err(t *testing.T) { - c := OpenClient(0) - defer c.Close() - c.Server.Handler.Broker().CreateReplica(123, &url.URL{Host: "localhost"}) - if err := c.CreateReplica(123, &url.URL{Host: "localhost"}); err == nil || err.Error() != `replica already exists` { - t.Fatalf("unexpected error: %v", err) - } -} - -// Ensure that a client can delete a replica. -func TestClient_DeleteReplica(t *testing.T) { - c := OpenClient(0) - defer c.Close() - c.Server.Handler.Broker().CreateReplica(123, &url.URL{Host: "localhost"}) - - // Delete replica through client. - if err := c.DeleteReplica(123); err != nil { - t.Fatalf("unexpected error: %v", err) - } - - // Verify replica was deleted. - if r := c.Server.Handler.Broker().Replica(123); r != nil { - t.Fatalf("replica not deleted") - } -} - -// Ensure that a client can create a subscription. -func TestClient_Subscribe(t *testing.T) { - c := OpenClient(0) - defer c.Close() - c.Server.Broker().CreateReplica(100, &url.URL{Host: "localhost"}) - - // Create subscription through client. - if err := c.Subscribe(100, 200); err != nil { - t.Fatalf("unexpected error: %v", err) - } - - // Verify subscription was created. - if a := c.Server.Handler.Broker().Replica(100).Topics(); !reflect.DeepEqual([]uint64{0, 200}, a) { - t.Fatalf("topics mismatch: %v", a) - } -} - -// Ensure that a client can passthrough an error while creating a subscription. -func TestClient_Subscribe_Err(t *testing.T) { - c := OpenClient(0) - defer c.Close() - if err := c.Subscribe(123, 100); err == nil || err.Error() != `replica not found` { - t.Fatalf("unexpected error: %v", err) - } -} - -// Ensure that a client can remove a subscription. -func TestClient_Unsubscribe(t *testing.T) { - c := OpenClient(0) - defer c.Close() - c.Server.Broker().CreateReplica(100, &url.URL{Host: "localhost"}) - c.Server.Broker().Subscribe(100, 200) - - // Remove subscription through client. - if err := c.Unsubscribe(100, 200); err != nil { - t.Fatalf("unexpected error: %v", err) - } - - // Verify subscription was removed. - if a := c.Server.Handler.Broker().Replica(100).Topics(); !reflect.DeepEqual([]uint64{0}, a) { - t.Fatalf("topics mismatch: %v", a) - } -} - -// Ensure that a client can passthrough an error while removing a subscription. -func TestClient_Unsubscribe_Err(t *testing.T) { - c := OpenClient(0) - defer c.Close() - if err := c.Unsubscribe(123, 100); err == nil || err.Error() != `replica not found` { - t.Fatalf("unexpected error: %v", err) - } -} +func TestClient_Heartbeat(t *testing.T) { t.Skip("pending") } // Client represents a test wrapper for the broker client. type Client struct { - clientConfig string // Temporary file for client config. *messaging.Client - Server *Server // test server } // NewClient returns a new instance of Client. func NewClient(replicaID uint64) *Client { return &Client{ - clientConfig: "", // Not all tests with NewClient require automatic temp file creation. - Client: messaging.NewClient(replicaID), - Server: NewServer(), + Client: messaging.NewClient(replicaID), } } -// OpenClient returns a new, open instance of Client. -func OpenClient(replicaID uint64) *Client { - c := NewClient(replicaID) - c.Server.Handler.Broker().CreateReplica(replicaID, &url.URL{Host: "localhost"}) - - // Open client to broker. - c.clientConfig = NewTempFile() - u, _ := url.Parse(c.Server.URL) - if err := c.Open(c.clientConfig, []*url.URL{u}); err != nil { - panic(err) - } - time.Sleep(10 * time.Millisecond) - - return c -} - // Close shuts down the client and server. func (c *Client) Close() { c.Client.Close() - if c.clientConfig != "" { - os.Remove(c.clientConfig) - } - c.Server.Close() } // MustPublish publishes a message. Panic on error. diff --git a/messaging/errors.go b/messaging/errors.go index 4bf4a95a3aa..13394a9f480 100644 --- a/messaging/errors.go +++ b/messaging/errors.go @@ -51,4 +51,13 @@ var ( // ErrIndexRequired is returned when making a call without a valid index. ErrIndexRequired = errors.New("index required") + + // ErrTopicOpen is returned when opening an already open topic. + ErrTopicOpen = errors.New("topic already open") + + // ErrSegmentReclaimed is returned when requesting a segment that has been deleted. + ErrSegmentReclaimed = errors.New("segment reclaimed") + + // ErrStaleWrite is returned when writing a message with an old index to a topic. + ErrStaleWrite = errors.New("stale write") ) diff --git a/messaging/handler.go b/messaging/handler.go index 085eb2142d2..d91118c162c 100644 --- a/messaging/handler.go +++ b/messaging/handler.go @@ -88,11 +88,7 @@ func (h *Handler) getMessages(w http.ResponseWriter, req *http.Request) { streaming := (req.URL.Query().Get("streaming") == "true") // Create a topic reader. - r, err := h.broker.OpenTopicReader(topicID, index, streaming) - if err != nil { - h.error(w, err, http.StatusInternalServerError) - return - } + r := NewTopicReader(h.broker.TopicPath(topicID), index, streaming) defer r.Close() // Ensure we close the topic reader if the connection is disconnected. diff --git a/messaging/handler_test.go b/messaging/handler_test.go index c911f81c2f7..cdb3790a211 100644 --- a/messaging/handler_test.go +++ b/messaging/handler_test.go @@ -4,24 +4,19 @@ import ( "net/http" "net/http/httptest" "net/url" - "reflect" - "strings" "testing" "time" "github.com/influxdb/influxdb/messaging" ) -// Ensure a replica can connect and stream messages. -func TestHandler_stream(t *testing.T) { +// Ensure a topic can be streamed from an index. +func TestHandler_getMessages(t *testing.T) { s := NewServer() defer s.Close() - // Create replica. - s.Handler.Broker().CreateReplica(2000, &url.URL{Host: "localhost"}) - // Send request to stream the replica. - resp, err := http.Get(s.URL + `/messaging/messages?replicaID=2000`) + resp, err := http.Get(s.URL + `/messaging/messages?topicID=2000`) defer resp.Body.Close() if err != nil { t.Fatalf("unexpected error: %s", err) @@ -30,47 +25,7 @@ func TestHandler_stream(t *testing.T) { } time.Sleep(10 * time.Millisecond) - // Decode from body. - var m messaging.Message - dec := messaging.NewMessageDecoder(resp.Body) - - // First message should be an internal message. - if err := dec.Decode(&m); err != nil { - t.Fatalf("decode error: %s", err) - } else if m.Index != 1 && m.Type != messaging.InternalMessageType { - t.Fatalf("unexpected index/type: %d / %x", m.Index, m.Type) - } - - // Second message should be an the create replica message. - if err := dec.Decode(&m); err != nil { - t.Fatalf("decode error: %s", err) - } else if m.Index != 2 && m.Type != messaging.CreateReplicaMessageType { - t.Fatalf("unexpected index/type: %d / %x", m.Index, m.Type) - } -} - -// Ensure an error is returned when requesting a stream without a replica id. -func TestHandler_stream_ErrReplicaIDRequired(t *testing.T) { - s := NewServer() - defer s.Close() - - resp, _ := http.Get(s.URL + `/messaging/messages`) - defer resp.Body.Close() - if msg := resp.Header.Get("X-Broker-Error"); resp.StatusCode != http.StatusBadRequest || msg != "replica id required" { - t.Fatalf("unexpected status/error: %d/%s", resp.StatusCode, msg) - } -} - -// Ensure an error is returned when requesting a stream for a non-existent replica. -func TestHandler_stream_ErrReplicaNotFound(t *testing.T) { - s := NewServer() - defer s.Close() - - resp, _ := http.Get(s.URL + `/messaging/messages?replicaID=0`) - defer resp.Body.Close() - if msg := resp.Header.Get("X-Broker-Error"); resp.StatusCode != http.StatusNotFound || msg != "replica not found" { - t.Fatalf("unexpected status/error: %d/%s", resp.StatusCode, msg) - } + // TODO: Decode from body. } // Ensure an error is returned when requesting a stream with the wrong HTTP method. @@ -85,89 +40,64 @@ func TestHandler_stream_ErrMethodNotAllowed(t *testing.T) { } } -// Ensure a handler can publish a message. -func TestHandler_publish(t *testing.T) { - s := NewServer() - defer s.Close() +// Ensure a handler can receive a message. +func TestHandler_postMessages(t *testing.T) { + t.Skip("pending") + /* + s := NewServer() + defer s.Close() - // Stream subscription for a replica. - var m messaging.Message - s.Handler.Broker().CreateReplica(2000, &url.URL{Host: "localhost"}) - s.Handler.Broker().Subscribe(2000, 200) - go func() { - resp, _ := http.Get(s.URL + `/messaging/messages?replicaID=2000`) + // Send request to the broker. + resp, _ := http.Post(s.URL+`/messaging/messages?type=100&topicID=200`, "application/octet-stream", strings.NewReader(`abc`)) defer resp.Body.Close() if resp.StatusCode != http.StatusOK { - t.Fatalf("unexpected response code: %d", resp.StatusCode) + t.Fatalf("unexpected status: %d: %s", resp.StatusCode, resp.Header.Get("X-Broker-Error")) } + s.Handler.Broker().Sync(4) - dec := messaging.NewMessageDecoder(resp.Body) - for { - if err := dec.Decode(&m); err != nil { - return - } + // Check if the last message received is our new message. + time.Sleep(10 * time.Millisecond) + if !reflect.DeepEqual(&m, &messaging.Message{Type: 100, Index: 4, TopicID: 200, Data: []byte("abc")}) { + t.Fatalf("unexpected message: %#v", &m) } - }() - - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/messages?type=100&topicID=200`, "application/octet-stream", strings.NewReader(`abc`)) - defer resp.Body.Close() - if resp.StatusCode != http.StatusOK { - t.Fatalf("unexpected status: %d: %s", resp.StatusCode, resp.Header.Get("X-Broker-Error")) - } - s.Handler.Broker().Sync(4) - - // Check if the last message received is our new message. - time.Sleep(10 * time.Millisecond) - if !reflect.DeepEqual(&m, &messaging.Message{Type: 100, Index: 4, TopicID: 200, Data: []byte("abc")}) { - t.Fatalf("unexpected message: %#v", &m) - } + */ } // Ensure a handler returns an error when publishing a message without a type. func TestHandler_publish_ErrMessageTypeRequired(t *testing.T) { - s := NewServer() - defer s.Close() + t.Skip("pending") + /* + s := NewServer() + defer s.Close() - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/messages?topicID=200`, "application/octet-stream", strings.NewReader(`foo`)) - defer resp.Body.Close() - if resp.StatusCode != http.StatusBadRequest { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "message type required" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } + // Send request to the broker. + resp, _ := http.Post(s.URL+`/messaging/messages?topicID=200`, "application/octet-stream", strings.NewReader(`foo`)) + defer resp.Body.Close() + if resp.StatusCode != http.StatusBadRequest { + t.Fatalf("unexpected status: %d", resp.StatusCode) + } else if resp.Header.Get("X-Broker-Error") != "message type required" { + t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) + } + */ } // Ensure a handler returns an error when publishing a message without a topic. func TestHandler_publish_ErrTopicRequired(t *testing.T) { - s := NewServer() - defer s.Close() - - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/messages?type=100`, "application/octet-stream", strings.NewReader(`foo`)) - defer resp.Body.Close() - if resp.StatusCode != http.StatusBadRequest { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "topic required" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } -} + t.Skip("pending") -// Ensure a handler returns an error when publishing to a closed broker. -func TestHandler_publish_ErrClosed(t *testing.T) { - s := NewServer() - s.Handler.Broker().Close() - defer s.Close() + /* + s := NewServer() + defer s.Close() - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/messages?type=100&topicID=200`, "application/octet-stream", strings.NewReader(`foo`)) - defer resp.Body.Close() - if resp.StatusCode != http.StatusInternalServerError { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "log closed" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } + // Send request to the broker. + resp, _ := http.Post(s.URL+`/messaging/messages?type=100`, "application/octet-stream", strings.NewReader(`foo`)) + defer resp.Body.Close() + if resp.StatusCode != http.StatusBadRequest { + t.Fatalf("unexpected status: %d", resp.StatusCode) + } else if resp.Header.Get("X-Broker-Error") != "topic required" { + t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) + } + */ } // Ensure the handler routes raft requests to the raft handler. @@ -192,212 +122,6 @@ func TestHandler_ErrNotFound(t *testing.T) { } } -// Ensure a handler can correctly create a replica. -func TestHandler_createReplica(t *testing.T) { - s := NewServer() - defer s.Close() - - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/replicas?id=200`, "application/octet-stream", nil) - defer resp.Body.Close() - if resp.StatusCode != http.StatusCreated { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } - - // Verify replica was created. - if r := s.Handler.Broker().Replica(200); r == nil { - t.Fatalf("replica not created") - } -} - -// Ensure a handler returns an error when creating a replica without an id. -func TestHandler_createReplica_ErrReplicaIDRequired(t *testing.T) { - s := NewServer() - defer s.Close() - - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/replicas`, "application/octet-stream", nil) - defer resp.Body.Close() - if resp.StatusCode != http.StatusBadRequest { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "replica id required" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } -} - -// Ensure a handler returns an error when creating a replica that already exists. -func TestHandler_createReplica_ErrReplicaExists(t *testing.T) { - s := NewServer() - defer s.Close() - s.Handler.Broker().CreateReplica(200, &url.URL{Host: "localhost"}) - - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/replicas?id=200`, "application/octet-stream", nil) - defer resp.Body.Close() - if resp.StatusCode != http.StatusConflict { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "replica already exists" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } -} - -// Ensure a handler can correctly delete a replica. -func TestHandler_deleteReplica(t *testing.T) { - s := NewServer() - defer s.Close() - s.Handler.Broker().CreateReplica(200, &url.URL{Host: "localhost"}) - - // Send request to the broker. - req, _ := http.NewRequest("DELETE", s.URL+`/messaging/replicas?id=200`, nil) - resp, _ := http.DefaultClient.Do(req) - defer resp.Body.Close() - if resp.StatusCode != http.StatusNoContent { - t.Fatalf("unexpected status: %d (%s)", resp.StatusCode, resp.Header.Get("X-Broker-Error")) - } - - // Verify replica was deleted. - if r := s.Handler.Broker().Replica(200); r != nil { - t.Fatalf("replica not deleted") - } -} - -// Ensure a handler returns an error when deleting a replica without an id. -func TestHandler_deleteReplica_ErrReplicaIDRequired(t *testing.T) { - s := NewServer() - defer s.Close() - - // Send request to the broker. - req, _ := http.NewRequest("DELETE", s.URL+`/messaging/replicas`, nil) - resp, _ := http.DefaultClient.Do(req) - defer resp.Body.Close() - if resp.StatusCode != http.StatusBadRequest { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "replica id required" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } -} - -// Ensure a handler can add a subscription for a replica/topic. -func TestHandler_subscribe(t *testing.T) { - s := NewServer() - defer s.Close() - s.Broker().CreateReplica(100, &url.URL{Host: "localhost"}) - - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/subscriptions?replicaID=100&topicID=200`, "application/octet-stream", nil) - defer resp.Body.Close() - if resp.StatusCode != http.StatusCreated { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } - - // Verify subscription was created. - if a := s.Handler.Broker().Replica(100).Topics(); !reflect.DeepEqual([]uint64{0, 200}, a) { - t.Fatalf("topics mismatch: %v", a) - } -} - -// Ensure a handler returns an error when subscribing without a replica id. -func TestHandler_subscribe_ErrReplicaIDRequired(t *testing.T) { - s := NewServer() - defer s.Close() - resp, _ := http.Post(s.URL+`/messaging/subscriptions?topicID=200`, "application/octet-stream", nil) - defer resp.Body.Close() - if resp.StatusCode != http.StatusBadRequest { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "replica id required" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } -} - -// Ensure a handler returns an error when subscribing without a topic id. -func TestHandler_subscribe_ErrTopicRequired(t *testing.T) { - s := NewServer() - defer s.Close() - resp, _ := http.Post(s.URL+`/messaging/subscriptions?replicaID=200`, "application/octet-stream", nil) - defer resp.Body.Close() - if resp.StatusCode != http.StatusBadRequest { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "topic required" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } -} - -// Ensure a handler returns an error when subscribing to a replica that doesn't exist. -func TestHandler_subscribe_ErrReplicaNotFound(t *testing.T) { - s := NewServer() - defer s.Close() - resp, _ := http.Post(s.URL+`/messaging/subscriptions?replicaID=200&topicID=100`, "application/octet-stream", nil) - defer resp.Body.Close() - if resp.StatusCode != http.StatusNotFound { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "replica not found" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } -} - -// Ensure a handler can unsubscribe a replica from a topic. -func TestHandler_unsubscribe(t *testing.T) { - s := NewServer() - defer s.Close() - s.Handler.Broker().CreateReplica(200, &url.URL{Host: "localhost"}) - s.Handler.Broker().Subscribe(200, 100) - - // Send request to the broker. - req, _ := http.NewRequest("DELETE", s.URL+`/messaging/subscriptions?replicaID=200&topicID=100`, nil) - resp, _ := http.DefaultClient.Do(req) - defer resp.Body.Close() - if resp.StatusCode != http.StatusNoContent { - t.Fatalf("unexpected status: %d (%s)", resp.StatusCode, resp.Header.Get("X-Broker-Error")) - } - - // Verify subscription was removed. - if a := s.Handler.Broker().Replica(200).Topics(); !reflect.DeepEqual([]uint64{0}, a) { - t.Fatalf("topics mismatch: %v", a) - } -} - -// Ensure a handler returns an error when unsubscribing without a replica id. -func TestHandler_unsubscribe_ErrReplicaIDRequired(t *testing.T) { - s := NewServer() - defer s.Close() - req, _ := http.NewRequest("DELETE", s.URL+`/messaging/subscriptions?topicID=100`, nil) - resp, _ := http.DefaultClient.Do(req) - defer resp.Body.Close() - if resp.StatusCode != http.StatusBadRequest { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "replica id required" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } -} - -// Ensure a handler returns an error when unsubscribing without a topic id. -func TestHandler_unsubscribe_ErrTopicRequired(t *testing.T) { - s := NewServer() - defer s.Close() - req, _ := http.NewRequest("DELETE", s.URL+`/messaging/subscriptions?replicaID=100`, nil) - resp, _ := http.DefaultClient.Do(req) - defer resp.Body.Close() - if resp.StatusCode != http.StatusBadRequest { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "topic required" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } -} - -// Ensure a handler returns an error when unsubscribing to a replica that doesn't exist. -func TestHandler_unsubscribe_ErrReplicaNotFound(t *testing.T) { - s := NewServer() - defer s.Close() - req, _ := http.NewRequest("DELETE", s.URL+`/messaging/subscriptions?replicaID=100&topicID=200`, nil) - resp, _ := http.DefaultClient.Do(req) - defer resp.Body.Close() - if resp.StatusCode != http.StatusNotFound { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "replica not found" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } -} - // Server is an test HTTP server that wraps a handler and broker. type Server struct { *httptest.Server diff --git a/messaging/intg_test.go b/messaging/intg_test.go index 7fae76e7bc3..1f20d953cb6 100644 --- a/messaging/intg_test.go +++ b/messaging/intg_test.go @@ -5,87 +5,92 @@ import ( "net/url" "testing" - "github.com/influxdb/influxdb/messaging" + //"github.com/influxdb/influxdb/messaging" ) // Ensure a broker can join to another existing broker and copy a snapshot. func TestBroker_Join(t *testing.T) { t.Skip() - s0, s1 := NewServer(), NewUninitializedServer() - defer s0.Close() - defer s1.Close() - - // Retrieve broker references. - b0, b1 := s0.Broker(), s1.Broker() - - // Create data on the first server. - b0.CreateReplica(20, &url.URL{Host: "localhost"}) - b0.Subscribe(20, 1000) - index, _ := b0.Publish(&messaging.Message{Type: 100, TopicID: 1000, Data: []byte("XXXX")}) - b0.Sync(index) - - // Join the second server. - if err := b1.Join(b0.URL()); err != nil { - t.Fatalf("join error: %s", err) - } - // Publish a message after the join & sync second broker. - index, _ = b0.Publish(&messaging.Message{Type: 100, TopicID: 1000, Data: []byte("YYYY")}) - if err := b1.Sync(index); err != nil { - t.Fatalf("unable to sync: idx=%d; err=%s", index, err) - } + /* + s0, s1 := NewServer(), NewUninitializedServer() + defer s0.Close() + defer s1.Close() - // Verify the second server copied a snapshot of the first server. - if r := b1.Replica(20); r == nil { - t.Fatalf("replica not found") - } + // Retrieve broker references. + b0, b1 := s0.Broker(), s1.Broker() - // Check that one publish message was sent. - if a := Messages(b1.MustReadAll(20)).Unicasted(); len(a) != 2 { - t.Fatalf("message count mismatch: %d", len(a)) - } else if m := a[0]; string(m.Data) != `XXXX` { - t.Fatalf("unexpected message: %s", m.Data) - } else if m := a[1]; string(m.Data) != `YYYY` { - t.Fatalf("unexpected message: %s", m.Data) - } + // Create data on the first server. + b0.CreateReplica(20, &url.URL{Host: "localhost"}) + b0.Subscribe(20, 1000) + index, _ := b0.Publish(&messaging.Message{Type: 100, TopicID: 1000, Data: []byte("XXXX")}) + b0.Sync(index) - // Publish another message to ensure logs are appended after writer is advanced. - index, _ = b0.Publish(&messaging.Message{Type: 100, TopicID: 1000, Data: []byte("ZZZZ")}) - if err := b1.Sync(index); err != nil { - t.Fatalf("unable to sync: idx=%d; err=%s", index, err) - } + // Join the second server. + if err := b1.Join(b0.URL()); err != nil { + t.Fatalf("join error: %s", err) + } - // Check messages one more time to ensure we have the last one. - if a := Messages(b1.MustReadAll(20)).Unicasted(); len(a) != 3 { - t.Fatalf("message count mismatch: %d", len(a)) - } else if m := a.Last(); string(m.Data) != `ZZZZ` { - t.Fatalf("unexpected message: %s", m.Data) - } + // Publish a message after the join & sync second broker. + index, _ = b0.Publish(&messaging.Message{Type: 100, TopicID: 1000, Data: []byte("YYYY")}) + if err := b1.Sync(index); err != nil { + t.Fatalf("unable to sync: idx=%d; err=%s", index, err) + } + + // Verify the second server copied a snapshot of the first server. + if r := b1.Replica(20); r == nil { + t.Fatalf("replica not found") + } + + // Check that one publish message was sent. + if a := Messages(b1.MustReadAll(20)).Unicasted(); len(a) != 2 { + t.Fatalf("message count mismatch: %d", len(a)) + } else if m := a[0]; string(m.Data) != `XXXX` { + t.Fatalf("unexpected message: %s", m.Data) + } else if m := a[1]; string(m.Data) != `YYYY` { + t.Fatalf("unexpected message: %s", m.Data) + } + + // Publish another message to ensure logs are appended after writer is advanced. + index, _ = b0.Publish(&messaging.Message{Type: 100, TopicID: 1000, Data: []byte("ZZZZ")}) + if err := b1.Sync(index); err != nil { + t.Fatalf("unable to sync: idx=%d; err=%s", index, err) + } + + // Check messages one more time to ensure we have the last one. + if a := Messages(b1.MustReadAll(20)).Unicasted(); len(a) != 3 { + t.Fatalf("message count mismatch: %d", len(a)) + } else if m := a.Last(); string(m.Data) != `ZZZZ` { + t.Fatalf("unexpected message: %s", m.Data) + } + */ } // Benchmarks a cluster of 3 brokers over HTTP. func BenchmarkCluster_Publish(b *testing.B) { - c := NewCluster(3) - defer c.Close() - - // Create replica and connect client. - c.Leader().Broker().CreateReplica(100, &url.URL{Host: "localhost"}) - client := messaging.NewClient(100) - client.Open("", []*url.URL{c.URL()}) - - b.ResetTimer() - - var index uint64 - for i := 0; i < b.N; i++ { - var err error - index, err = client.Publish(&messaging.Message{Type: 0, TopicID: 1, Data: make([]byte, 50)}) - if err != nil { - b.Fatalf("unexpected error: %s", err) + b.Skip("pending") + /* + c := NewCluster(3) + defer c.Close() + + // Create replica and connect client. + client := messaging.NewClient() + client.Open("", []*url.URL{c.URL()}) + + b.ResetTimer() + + var index uint64 + for i := 0; i < b.N; i++ { + var err error + index, err = client.Publish(&messaging.Message{Type: 0, TopicID: 1, Data: make([]byte, 50)}) + if err != nil { + b.Fatalf("unexpected error: %s", err) + } } - } - // Wait for the broker to commit. - c.MustSync(index) + // Wait for the broker to commit. + c.MustSync(index) + */ } // Cluster represents a set of joined Servers. From ef8658e0fc9422b58815fbcdccde953bc123224d Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 8 Mar 2015 15:28:43 -0600 Subject: [PATCH 05/20] Continuing stateless broker refactor. --- messaging/broker.go | 702 +++++++++++++++++++------------------- messaging/broker_test.go | 480 ++++++++++++++++++++------ messaging/client.go | 250 ++++---------- messaging/handler.go | 42 +-- messaging/handler_test.go | 268 ++++++++++----- messaging/intg_test.go | 6 +- 6 files changed, 1006 insertions(+), 742 deletions(-) diff --git a/messaging/broker.go b/messaging/broker.go index 84661401545..7260723a4d6 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -3,11 +3,9 @@ package messaging import ( "encoding/binary" "encoding/json" - "errors" "fmt" "io" "log" - "net/url" "os" "path/filepath" "sort" @@ -23,24 +21,26 @@ import ( // Each topic represents a linear series of events. type Broker struct { mu sync.RWMutex - path string // data directory - index uint64 // highest applied index - log *raft.Log // internal raft log + path string // data directory + index uint64 // highest applied index meta *bolt.DB // metadata topics map[uint64]*Topic // topics by id + // Log is the distributed raft log that commands are applied to. + Log interface { + Apply(data []byte) (index uint64, err error) + } + Logger *log.Logger } // NewBroker returns a new instance of a Broker with default values. func NewBroker() *Broker { b := &Broker{ - log: raft.NewLog(), topics: make(map[uint64]*Topic), - Logger: log.New(os.Stderr, "[broker] ", log.LstdFlags), } - b.log.FSM = (*brokerFSM)(b) + b.SetLogOutput(os.Stderr) return b } @@ -48,9 +48,6 @@ func NewBroker() *Broker { // Returns empty string if the broker is not open. func (b *Broker) Path() string { return b.path } -// Log returns the underlying raft log. -func (b *Broker) Log() *raft.Log { return b.log } - // metaPath returns the file path to the broker's metadata file. func (b *Broker) metaPath() string { if b.path == "" { @@ -74,6 +71,14 @@ func (b *Broker) topicPath(id uint64) string { return filepath.Join(b.path, strconv.FormatUint(id, 10)) } +// Topic returns a topic on a broker by id. +// Returns nil if the topic doesn't exist or the broker is closed. +func (b *Broker) Topic(id uint64) *Topic { + b.mu.RLock() + defer b.mu.RUnlock() + return b.topics[id] +} + // Index returns the highest index seen by the broker across all topics. // Returns 0 if the broker is closed. func (b *Broker) Index() uint64 { @@ -86,14 +91,11 @@ func (b *Broker) Index() uint64 { func (b *Broker) opened() bool { return b.path != "" } // SetLogOutput sets writer for all Broker log output. -func (b *Broker) SetLogOutput(w io.Writer) { - b.Logger = log.New(w, "[broker] ", log.LstdFlags) - b.log.SetLogOutput(w) -} +func (b *Broker) SetLogOutput(w io.Writer) { b.Logger = log.New(w, "[broker] ", log.LstdFlags) } // Open initializes the log. // The broker then must be initialized or join a cluster before it can be used. -func (b *Broker) Open(path string, u *url.URL) error { +func (b *Broker) Open(path string) error { b.mu.Lock() defer b.mu.Unlock() @@ -103,11 +105,6 @@ func (b *Broker) Open(path string, u *url.URL) error { } b.path = path - // Require a non-blank connection address. - if u == nil { - return ErrConnectionAddressRequired - } - // Ensure root directory exists. if err := os.MkdirAll(path, 0700); err != nil { b.close() @@ -143,51 +140,22 @@ func (b *Broker) Open(path string, u *url.URL) error { return fmt.Errorf("open topics: %s", err) } - // Open underlying raft log. - if err := b.log.Open(filepath.Join(path, "raft")); err != nil { - _ = b.close() - return fmt.Errorf("raft: %s", err) - } - - // Copy connection URL. - b.log.URL = &url.URL{} - *b.log.URL = *u - return nil } // loadTopics reads all topic metadata into memory. func (b *Broker) openTopics() error { - // Open handle to directory. - f, err := os.Open(b.path) - if err != nil { - return err - } - defer func() { _ = f.Close() }() - - // Read directory items. - fis, err := f.Readdir(0) + // Read all topics from the broker directory. + topics, err := ReadTopics(b.path) if err != nil { - return err + return fmt.Errorf("read topics: %s", err) } - // Create a topic for each directory with a numeric name. - for _, fi := range fis { - // Ignore non-directory entries. - if !fi.IsDir() { - continue - } - - // Filename must be numeric. - topicID, err := strconv.ParseUint(fi.Name(), 10, 64) - if err != nil { - continue - } - - // Create and open topic. - t := NewTopic(topicID) - if err := t.Open(filepath.Join(b.path, fi.Name())); err != nil { - return fmt.Errorf("open topic: id=%d, err=%s", topicID, err) + // Open each topic and append to the map. + b.topics = make(map[uint64]*Topic) + for _, t := range topics { + if err := t.Open(); err != nil { + return fmt.Errorf("open topic: id=%d, err=%s", t.id, err) } b.topics[t.id] = t } @@ -216,12 +184,15 @@ func (b *Broker) close() error { } b.path = "" + // Close meta data. + if b.meta != nil { + _ = b.meta.Close() + b.meta = nil + } + // Close all topics. b.closeTopics() - // Close raft log. - _ = b.log.Close() - return nil } @@ -233,17 +204,64 @@ func (b *Broker) closeTopics() { b.topics = make(map[uint64]*Topic) } +// SetMaxIndex sets the highest index seen by the broker. +// This is only used for internal log messages and topics may have a higher index. +func (b *Broker) SetMaxIndex(index uint64) error { + return b.meta.Update(func(tx *bolt.Tx) error { + return tx.Bucket([]byte("meta")).Put([]byte("index"), u64tob(index)) + }) +} + +// Snapshot streams the current state of the broker and returns the index. +func (b *Broker) Snapshot(w io.Writer) (uint64, error) { + // TODO: Prevent truncation during snapshot. + + // Calculate header under lock. + b.mu.RLock() + hdr, err := b.createSnapshotHeader() + b.mu.RUnlock() + if err != nil { + return 0, fmt.Errorf("create snapshot: %s", err) + } + + // Encode snapshot header. + buf, err := json.Marshal(&hdr) + if err != nil { + return 0, fmt.Errorf("encode snapshot header: %s", err) + } + + // Write header frame. + if err := binary.Write(w, binary.BigEndian, uint32(len(buf))); err != nil { + return 0, fmt.Errorf("write header size: %s", err) + } + if _, err := w.Write(buf); err != nil { + return 0, fmt.Errorf("write header: %s", err) + } + + // Stream each topic sequentially. + for _, t := range hdr.Topics { + for _, s := range t.Segments { + if _, err := copyFileN(w, s.path, s.Size); err != nil { + return 0, err + } + } + } + + // Return the snapshot and its last applied index. + return hdr.Index, nil +} + // createSnapshotHeader creates a snapshot header. func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) { // Create parent header. - sh := &snapshotHeader{} + sh := &snapshotHeader{Index: b.index} // Append topics. for _, t := range b.topics { // Create snapshot topic. st := &snapshotTopic{ID: t.id} - // TODO: Read segments from disk, not topic. + // Read segments from disk, not topic. segments, err := ReadSegments(t.path) if err != nil { return nil, fmt.Errorf("read segments: %s", err) @@ -270,11 +288,6 @@ func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) { }) } - // Bump the snapshot header max index. - if t.index > sh.Index { - sh.Index = t.index - } - // Append topic to the snapshot. sh.Topics = append(sh.Topics, st) } @@ -282,33 +295,95 @@ func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) { return sh, nil } -// U/RL returns the connection url for the broker. -func (b *Broker) URL() *url.URL { - return b.log.URL -} +// copyFileN copies n bytes from a path to a writer. +func copyFileN(w io.Writer, path string, n int64) (int64, error) { + // Open file for reading. + f, err := os.Open(path) + if err != nil { + return 0, err + } + defer func() { _ = f.Close() }() -// LeaderURL returns the connection url for the leader broker. -func (b *Broker) LeaderURL() *url.URL { - _, u := b.log.Leader() - return u + // Copy file up to n bytes. + return io.CopyN(w, f, n) } -// IsLeader returns true if the broker is the current leader. -func (b *Broker) IsLeader() bool { return b.log.State() == raft.Leader } +// Restore reads the broker state. +func (b *Broker) Restore(r io.Reader) error { + b.mu.Lock() + defer b.mu.Unlock() + + // Remove and recreate broker path. + if err := os.RemoveAll(b.path); err != nil && !os.IsNotExist(err) { + return fmt.Errorf("remove all: %s", err) + } else if err = os.MkdirAll(b.path, 0700); err != nil { + return fmt.Errorf("mkdir: %s", err) + } + + // Read header frame. + var sz uint32 + if err := binary.Read(r, binary.BigEndian, &sz); err != nil { + return fmt.Errorf("read header size: %s", err) + } + buf := make([]byte, sz) + if _, err := io.ReadFull(r, buf); err != nil { + return fmt.Errorf("read header: %s", err) + } -// Initialize creates a new cluster. -func (b *Broker) Initialize() error { - if err := b.log.Initialize(); err != nil { - return fmt.Errorf("raft: %s", err) + // Decode header. + sh := &snapshotHeader{} + if err := json.Unmarshal(buf, &sh); err != nil { + return fmt.Errorf("decode header: %s", err) } - return nil -} -// Join joins an existing cluster. -func (b *Broker) Join(u *url.URL) error { - if err := b.log.Join(u); err != nil { - return fmt.Errorf("raft: %s", err) + // Close any topics which might be open and clear them out. + b.closeTopics() + + // Copy topic files from snapshot to local disk. + for _, st := range sh.Topics { + t := NewTopic(st.ID, b.topicPath(st.ID)) + + // Create topic directory. + if err := os.MkdirAll(t.Path(), 0700); err != nil { + return fmt.Errorf("make topic dir: %s", err) + } + + // Copy data from snapshot into segment files. + // We don't instantiate the segments because that will be done + // automatically when calling Open() on the topic. + for _, ss := range st.Segments { + if err := func() error { + // Create a new file with the starting index. + f, err := os.Create(t.segmentPath(ss.Index)) + if err != nil { + return fmt.Errorf("open segment: %s", err) + } + defer func() { _ = f.Close() }() + + // Copy from stream into file. + if _, err := io.CopyN(f, r, ss.Size); err != nil { + return fmt.Errorf("copy segment: %s", err) + } + + return nil + }(); err != nil { + return err + } + } + + // Open new empty topic file. + if err := t.Open(); err != nil { + return fmt.Errorf("open topic: %s", err) + } + b.topics[t.id] = t } + + // Set the highest seen index. + if err := b.SetMaxIndex(sh.Index); err != nil { + return fmt.Errorf("set max index: %s", err) + } + b.index = sh.Index + return nil } @@ -317,28 +392,14 @@ func (b *Broker) Join(u *url.URL) error { func (b *Broker) Publish(m *Message) (uint64, error) { buf, err := m.MarshalBinary() assert(err == nil, "marshal binary error: %s", err) - return b.log.Apply(buf) + return b.Log.Apply(buf) } -// PublishSync writes a message and waits until the change is applied. -func (b *Broker) PublishSync(m *Message) error { - // Publish message. - index, err := b.Publish(m) - if err != nil { - return err - } - - // Wait for message to apply. - if err := b.Sync(index); err != nil { - return err - } - - return nil +// TopicReader returns a new topic reader for a topic starting from a given index. +func (b *Broker) TopicReader(topicID, index uint64, streaming bool) *TopicReader { + return NewTopicReader(b.TopicPath(topicID), index, streaming) } -// Sync pauses until the given index has been applied. -func (b *Broker) Sync(index uint64) error { return b.log.Wait(index) } - // SetTopicMaxIndex updates the highest replicated index for a topic. // If a higher index is already set on the topic then the call is ignored. // This index is only held in memory and is used for topic segment reclamation. @@ -350,18 +411,14 @@ func (b *Broker) SetTopicMaxIndex(topicID, index uint64) error { return err } -func (b *Broker) mustApplySetTopicMaxIndex(m *Message) { +func (b *Broker) applySetTopicMaxIndex(m *Message) { topicID, index := unmarshalTopicIndex(m.Data) - b.mu.Lock() - defer b.mu.Unlock() - - // Ignore if the topic doesn't exist or the index is already higher. + // Set index if it's not already set higher. t := b.topics[topicID] - if t == nil || t.index >= index { - return + if t != nil && t.index < index { + t.index = index } - t.index = index } func marshalTopicIndex(topicID, index uint64) []byte { @@ -377,236 +434,47 @@ func unmarshalTopicIndex(b []byte) (topicID, index uint64) { return } -// Truncate removes log segments that have been replicated to all subscribed replicas. -func (b *Broker) Truncate() error { +// Apply executes a message against the broker. +func (b *Broker) Apply(m *Message) error { b.mu.Lock() defer b.mu.Unlock() - // TODO: Generate a list of all segments. - // TODO: Sort by index. - // TODO: Delete segments until we reclaim enough space. - // TODO: Add tombstone for the last index. - - /* - // Loop over every topic. - for _, t := range b.topics { - // Determine the highest index replicated to all subscribed replicas. - minReplicaTopicIndex := b.minReplicaTopicIndex(t.id) - - // Loop over segments and close as needed. - newSegments := make(Segments, 0, len(t.segments)) - for i, s := range t.segments { - // Find the next segment so we can find the upper index bound. - var next *Segment - if i < len(t.segments)-1 { - next = t.segments[i+1] - } - - // Ignore the last segment or if the next index is less than - // the highest index replicated across all replicas. - if next == nil || minReplicaTopicIndex < next.index { - newSegments = append(newSegments, s) - continue - } - - // Remove the segment if the replicated index has moved pasted - // all the entries inside this segment. - s.close() - if err := os.Remove(s.path); err != nil && !os.IsNotExist(err) { - return fmt.Errorf("remove segment: topic=%d, segment=%d, err=%s", t.id, s.index, err) - } - } - } - */ - - return nil -} - -// brokerFSM implements the raft.FSM interface for the broker. -// This is implemented as a separate type because it is not meant to be exported. -type brokerFSM Broker - -// MustApply executes a raft log entry against the broker. -// Non-repeatable errors such as system or disk errors must panic. -func (fsm *brokerFSM) MustApply(e *raft.LogEntry) { - b := (*Broker)(fsm) - - // Decode commands into messages. - m := &Message{} - if e.Type == raft.LogEntryCommand { - err := m.UnmarshalBinary(e.Data) - assert(err == nil, "message unmarshal: %s", err) - } else { - m.Type = InternalMessageType + // Exit if broker isn't open. + if !b.opened() { + return ErrClosed } - m.Index = e.Index + + // Ensure messages with old indexes aren't re-applied. + assert(m.Index > b.index, "stale apply: msg=%d, broker=%d", m.Index, b.index) // Process internal commands separately than the topic writes. switch m.Type { - case InternalMessageType: - b.mustApplyInternal(m) case SetTopicMaxIndexMessageType: - b.mustApplySetTopicMaxIndex(m) + b.applySetTopicMaxIndex(m) default: // Create topic if not exists. t := b.topics[m.TopicID] if t == nil { - t = NewTopic(m.TopicID) - if err := t.Open(b.topicPath(t.id)); err != nil { - panic("open topic: " + err.Error()) + t = NewTopic(m.TopicID, b.topicPath(m.TopicID)) + if err := t.Open(); err != nil { + return fmt.Errorf("open topic: %s", err) } b.topics[t.id] = t } // Write message to topic. if err := t.WriteMessage(m); err != nil { - panic("write message: " + err.Error()) + return fmt.Errorf("write message: %s", err) } } // Save highest applied index in memory. // Only internal messages need to have their indexes saved to disk. - b.index = e.Index -} - -// mustApplyInternal updates the highest index applied to the broker. -func (b *Broker) mustApplyInternal(m *Message) { - err := b.meta.Update(func(tx *bolt.Tx) error { - return tx.Bucket([]byte("meta")).Put([]byte("index"), u64tob(m.Index)) - }) - assert(err == nil, "apply internal message: idx=%d, err=%s", m.Index, err) -} - -// Index returns the highest index that the broker has seen. -func (fsm *brokerFSM) Index() (uint64, error) { - b := (*Broker)(fsm) - return b.index, nil -} - -// Snapshot streams the current state of the broker and returns the index. -func (fsm *brokerFSM) Snapshot(w io.Writer) (uint64, error) { - b := (*Broker)(fsm) - - // TODO: Prevent truncation during snapshot. - - // Calculate header under lock. - b.mu.RLock() - hdr, err := b.createSnapshotHeader() - b.mu.RUnlock() - if err != nil { - return 0, fmt.Errorf("create snapshot: %s", err) - } - - // Encode snapshot header. - buf, err := json.Marshal(&hdr) - if err != nil { - return 0, fmt.Errorf("encode snapshot header: %s", err) - } - - // Write header frame. - if err := binary.Write(w, binary.BigEndian, uint32(len(buf))); err != nil { - return 0, fmt.Errorf("write header size: %s", err) - } - if _, err := w.Write(buf); err != nil { - return 0, fmt.Errorf("write header: %s", err) - } - - // Stream each topic sequentially. - for _, t := range hdr.Topics { - for _, s := range t.Segments { - if _, err := copyFileN(w, s.path, s.Size); err != nil { - return 0, err - } - } - } - - // Return the snapshot and its last applied index. - return hdr.Index, nil -} - -// Restore reads the broker state. -func (fsm *brokerFSM) Restore(r io.Reader) error { - b := (*Broker)(fsm) - - b.mu.Lock() - defer b.mu.Unlock() - - // Remove and recreate broker path. - if err := os.RemoveAll(b.path); err != nil && !os.IsNotExist(err) { - return fmt.Errorf("remove all: %s", err) - } else if err = os.MkdirAll(b.path, 0700); err != nil { - return fmt.Errorf("mkdir: %s", err) - } - - // Read header frame. - var sz uint32 - if err := binary.Read(r, binary.BigEndian, &sz); err != nil { - return fmt.Errorf("read header size: %s", err) - } - buf := make([]byte, sz) - if _, err := io.ReadFull(r, buf); err != nil { - return fmt.Errorf("read header: %s", err) - } - - // Decode header. - sh := &snapshotHeader{} - if err := json.Unmarshal(buf, &sh); err != nil { - return fmt.Errorf("decode header: %s", err) - } - - // Close any topics which might be open and clear them out. - b.closeTopics() - - // Copy topic files from snapshot to local disk. - for _, st := range sh.Topics { - t := NewTopic(st.ID) - - // Copy data from snapshot into segment files. - // We don't instantiate the segments because that will be done - // automatically when calling Open() on the topic. - for _, ss := range st.Segments { - if err := func() error { - // Create a new file with the starting index. - f, err := os.Open(t.segmentPath(ss.Index)) - if err != nil { - return fmt.Errorf("open segment: %s", err) - } - defer func() { _ = f.Close() }() - - // Copy from stream into file. - if _, err := io.CopyN(f, r, ss.Size); err != nil { - return fmt.Errorf("copy segment: %s", err) - } - - return nil - }(); err != nil { - return err - } - } - - // Open new empty topic file. - if err := t.Open(b.topicPath(t.id)); err != nil { - return fmt.Errorf("open topic: %s", err) - } - b.topics[t.id] = t - } + b.index = m.Index return nil } -// copyFileN copies n bytes from a path to a writer. -func copyFileN(w io.Writer, path string, n int64) (int64, error) { - // Open file for reading. - f, err := os.Open(path) - if err != nil { - return 0, err - } - defer func() { _ = f.Close() }() - - // Copy file up to n bytes. - return io.CopyN(w, f, n) -} - // snapshotHeader represents the header of a snapshot. type snapshotHeader struct { Topics []*snapshotTopic `json:"topics"` @@ -625,6 +493,46 @@ type snapshotTopicSegment struct { path string } +// RaftFSM is a wrapper struct around the broker that implements the raft.FSM interface. +// It will panic for any errors that occur during Apply. +type RaftFSM struct { + Broker interface { + Apply(m *Message) error + Index() (uint64, error) + SetMaxIndex(uint64) error + Snapshot(w io.Writer) (uint64, error) + Restore(r io.Reader) error + } +} + +func (fsm *RaftFSM) Index() (uint64, error) { return fsm.Broker.Index() } +func (fsm *RaftFSM) Snapshot(w io.Writer) (uint64, error) { return fsm.Broker.Snapshot(w) } +func (fsm *RaftFSM) Restore(r io.Reader) error { return fsm.Broker.Restore(r) } + +// MustApply applies a raft command to the broker. Panic on error. +func (fsm *RaftFSM) MustApply(e *raft.LogEntry) { + switch e.Type { + case raft.LogEntryCommand: + // Decode message. + m := &Message{} + if err := m.UnmarshalBinary(e.Data); err != nil { + panic("message unmarshal: " + err.Error()) + } + m.Index = e.Index + + // Apply message. + if err := fsm.Broker.Apply(m); err != nil { + panic(err.Error()) + } + + default: + // Move internal index forward if it's an internal raft comand. + if err := fsm.Broker.SetMaxIndex(e.Index); err != nil { + panic(fmt.Sprintf("set max index: idx=%d, err=%s", e.Index, err)) + } + } +} + // DefaultMaxSegmentSize is the largest a segment can get before starting a new segment. const DefaultMaxSegmentSize = 10 * 1024 * 1024 // 10MB @@ -635,27 +543,44 @@ const DefaultMaxSegmentSize = 10 * 1024 * 1024 // 10MB // contiguous range of entries. type Topic struct { mu sync.Mutex - id uint64 // unique identifier - index uint64 // highest index replicated - path string // on-disk path - file *os.File // last segment writer + id uint64 // unique identifier + index uint64 // highest index replicated + path string // on-disk path + + file *os.File // last segment writer + opened bool // The largest a segment can get before splitting into a new segment. MaxSegmentSize int64 } -// NewTopic returns a new instance of topic. -func NewTopic(id uint64) *Topic { +// NewTopic returns a new instance of Topic. +func NewTopic(id uint64, path string) *Topic { return &Topic{ - id: id, + id: id, + path: path, + MaxSegmentSize: DefaultMaxSegmentSize, } } +// ID returns the topic identifier. +func (t *Topic) ID() uint64 { return t.id } + +// Path returns the topic path. +func (t *Topic) Path() string { return t.path } + +// Index returns the highest replicated index for the topic. +func (t *Topic) Index() uint64 { + t.mu.Lock() + defer t.mu.Unlock() + return t.index +} + // SegmentPath returns the path to a segment starting with a given log index. func (t *Topic) SegmentPath(index uint64) string { t.mu.Lock() - defer t.mu.Lock() + defer t.mu.Unlock() return t.segmentPath(index) } @@ -667,15 +592,17 @@ func (t *Topic) segmentPath(index uint64) string { } // Open opens a topic for writing. -func (t *Topic) Open(path string) error { +func (t *Topic) Open() error { t.mu.Lock() defer t.mu.Unlock() - // Ensure topic is not already open. - if t.path != "" { + // Ensure topic is not already open and it has a path. + if t.opened { return ErrTopicOpen + } else if t.path == "" { + return ErrPathRequired } - t.path = path + t.opened = true // Ensure the parent directory exists. if err := os.MkdirAll(t.path, 0700); err != nil { @@ -727,7 +654,7 @@ func (t *Topic) close() error { t.file = nil } - t.path = "" + t.opened = false t.index = 0 return nil @@ -797,6 +724,48 @@ func (t *Topic) WriteMessage(m *Message) error { return nil } +// Topics represents a list of topics sorted by id. +type Topics []*Topic + +func (a Topics) Len() int { return len(a) } +func (a Topics) Less(i, j int) bool { return a[i].id < a[j].id } +func (a Topics) Swap(i, j int) { a[i], a[j] = a[j], a[i] } + +// ReadTopics reads all topics from a directory path. +func ReadTopics(path string) (Topics, error) { + // Open handle to directory. + f, err := os.Open(path) + if err != nil { + return nil, err + } + defer func() { _ = f.Close() }() + + // Read directory items. + fis, err := f.Readdir(0) + if err != nil { + return nil, err + } + + // Create a topic for each directory with a numeric name. + var a Topics + for _, fi := range fis { + // Skip non-directory paths. + if !fi.IsDir() { + continue + } + + topicID, err := strconv.ParseUint(fi.Name(), 10, 64) + if err != nil { + continue + } + + a = append(a, NewTopic(topicID, filepath.Join(path, fi.Name()))) + } + sort.Sort(a) + + return a, nil +} + // Segment represents a contiguous section of a topic log. type Segment struct { Index uint64 // starting index of the segment and name @@ -948,10 +917,9 @@ func (r *TopicReader) Read(p []byte) (int, error) { // Write data to buffer. // If no more data is available, then retry with the next segment. - if n, err := r.Read(p); err == io.EOF { - f, err = r.NextFile() - if err != nil { - return 0, fmt.Errorf("next: %s", err) + if n, err := r.file.Read(p); err == io.EOF { + if err := r.nextSegment(); err != nil { + return 0, fmt.Errorf("next segment: %s", err) } continue } else { @@ -968,7 +936,7 @@ func (r *TopicReader) File() (*os.File, error) { // Exit if closed. if r.closed { - return nil, errors.New("topic reader closed") + return nil, nil } // If the first file hasn't been opened then open it and seek. @@ -983,7 +951,7 @@ func (r *TopicReader) File() (*os.File, error) { } // Open that segment file. - f, err := os.Open(filepath.Join(r.path, segment.Path)) + f, err := os.Open(segment.Path) if err != nil { return nil, fmt.Errorf("open: %s", err) } @@ -1006,20 +974,62 @@ func (r *TopicReader) seekAfterIndex(f *os.File, seek uint64) error { dec := NewMessageDecoder(f) for { var m Message - if err := dec.Decode(&m); err == io.EOF || m.Index >= seek { + if err := dec.Decode(&m); err == io.EOF { return nil } else if err != nil { return err + } else if m.Index >= seek { + // Seek to message start. + if _, err := f.Seek(-int64(messageHeaderSize+len(m.Data)), os.SEEK_CUR); err != nil { + return fmt.Errorf("seek: %s", err) + } + return nil } } } -// NextFile closes the current segment's file handle and opens the next segment. -func (r *TopicReader) NextFile() (*os.File, error) { +// nextSegment closes the current segment's file handle and opens the next segment. +func (r *TopicReader) nextSegment() error { r.mu.Lock() defer r.mu.Unlock() - panic("not yet implemented") + // Find current segment index. + index, err := strconv.ParseUint(filepath.Base(r.file.Name()), 10, 64) + if err != nil { + return fmt.Errorf("parse current segment index: %s", err) + } + + // Clear file. + if r.file != nil { + r.file.Close() + r.file = nil + } + + // Read current segment list. + // If no segments exist then exit. + segments, err := ReadSegments(r.path) + if err != nil { + return fmt.Errorf("read segments: %s", err) + } else if len(segments) == 0 { + return nil + } + + // Loop over segments and find the next one. + for i := range segments[:len(segments)-1] { + if segments[i].Index == index { + f, err := os.Open(segments[i+1].Path) + if err != nil { + return fmt.Errorf("open next segment: %s", err) + } + r.file = f + return nil + } + } + + // If we didn't find the current segment or the current segment is the + // last segment then mark the reader as closed. + r.closed = true + return nil } // Close closes the reader. @@ -1047,8 +1057,7 @@ type MessageType uint16 const BrokerMessageType = 0x8000 const ( - InternalMessageType = BrokerMessageType | MessageType(0x00) - SetTopicMaxIndexMessageType = BrokerMessageType | MessageType(0x01) + SetTopicMaxIndexMessageType = BrokerMessageType | MessageType(0x00) ) // The size of the encoded message header, in bytes. @@ -1139,6 +1148,15 @@ func (dec *MessageDecoder) Decode(m *Message) error { return nil } +// UnmarshalMessage decodes a byte slice into a message. +func UnmarshalMessage(data []byte) (*Message, error) { + m := &Message{} + if err := m.UnmarshalBinary(data); err != nil { + return nil, err + } + return m, nil +} + type flusher interface { Flush() } diff --git a/messaging/broker_test.go b/messaging/broker_test.go index 1ce5a910f05..8838c57dc78 100644 --- a/messaging/broker_test.go +++ b/messaging/broker_test.go @@ -5,139 +5,338 @@ import ( "fmt" "io" "io/ioutil" - "net/url" "os" "path/filepath" "reflect" + "strings" "testing" "github.com/influxdb/influxdb/messaging" + "github.com/influxdb/influxdb/raft" ) // Ensure that opening a broker without a path returns an error. func TestBroker_Open_ErrPathRequired(t *testing.T) { b := messaging.NewBroker() - if err := b.Open("", &url.URL{Host: "127.0.0.1:8080"}); err != messaging.ErrPathRequired { - t.Fatalf("unexpected error: %s", err) - } -} - -// Ensure that opening a broker without a connection address returns an error. -func TestBroker_Open_ErrAddressRequired(t *testing.T) { - b := messaging.NewBroker() - f := tempfile() - defer os.Remove(f) - - if err := b.Open(f, nil); err != messaging.ErrConnectionAddressRequired { + if err := b.Open(""); err != messaging.ErrPathRequired { t.Fatalf("unexpected error: %s", err) } } // Ensure that closing an already closed broker returns an error. func TestBroker_Close_ErrClosed(t *testing.T) { - b := NewBroker(nil) + b := NewBroker() b.Close() if err := b.Broker.Close(); err != messaging.ErrClosed { t.Fatalf("unexpected error: %s", err) } } -/* // Ensure the broker can write messages to the appropriate topics. func TestBroker_Publish(t *testing.T) { - b := NewBroker(nil) + b := NewBroker() defer b.Close() + b.Log().ApplyFunc = func(data []byte) (uint64, error) { + var m messaging.Message + dec := messaging.NewMessageDecoder(bytes.NewReader(data)) + if err := dec.Decode(&m); err != nil { + t.Fatal(err) + } else if !reflect.DeepEqual(&m, &messaging.Message{Type: 100, TopicID: 20, Data: []byte("0000")}) { + t.Fatalf("unexpected message: %#v", &m) + } + return 2, nil + } + // Write a message to the broker. index, err := b.Publish(&messaging.Message{Type: 100, TopicID: 20, Data: []byte("0000")}) if err != nil { t.Fatalf("unexpected error: %s", err) - } else if index != 4 { + } else if index != 2 { t.Fatalf("unexpected index: %d", index) } - if err := b.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) +} + +// Ensure the broker can apply messages from the log. +func TestBroker_Apply(t *testing.T) { + b := OpenBroker() + defer b.Close() + + // Write two message to topic #20, one message to topic #21. + if err := b.Apply(&messaging.Message{Index: 2, TopicID: 20, Data: []byte{0}}); err != nil { + t.Fatal(err) + } else if err := b.Apply(&messaging.Message{Index: 3, TopicID: 21, Data: []byte{100}}); err != nil { + t.Fatal(err) + } else if err := b.Apply(&messaging.Message{Index: 4, TopicID: 20, Data: []byte{200}}); err != nil { + t.Fatal(err) } - // Read message from the replica. - var buf bytes.Buffer - go func() { - if _, err := b.Replica(2000).WriteTo(&buf); err != nil { - t.Fatalf("write to: %s", err) - } - }() - time.Sleep(10 * time.Millisecond) + // Ensure topic exists. + if topic := b.Topic(20); topic == nil { + t.Fatal("topic not created") + } + + // Read message from topic. + r := b.TopicReader(20, 0, false) + defer r.Close() + dec := messaging.NewMessageDecoder(r) - // Read out the config messages first. var m messaging.Message - dec := messaging.NewMessageDecoder(&buf) - if err := dec.Decode(&m); err != nil || m.Type != messaging.InternalMessageType { - t.Fatalf("decode(internal): %x (%v)", m.Type, err) + if err := dec.Decode(&m); err != nil { + t.Fatalf("message decode error: %s", err) + } else if !reflect.DeepEqual(&m, &messaging.Message{Index: 2, TopicID: 20, Data: []byte{0}}) { + t.Fatalf("unexpected message: %#v", &m) } - if err := dec.Decode(&m); err != nil || m.Type != messaging.CreateReplicaMessageType { - t.Fatalf("decode(create replica): %x (%v)", m.Type, err) + if err := dec.Decode(&m); err != nil { + t.Fatalf("message decode error: %s", err) + } else if !reflect.DeepEqual(&m, &messaging.Message{Index: 4, TopicID: 20, Data: []byte{200}}) { + t.Fatalf("unexpected message: %#v", &m) } - if err := dec.Decode(&m); err != nil || m.Type != messaging.SubscribeMessageType { - t.Fatalf("decode(subscribe): %x (%v)", m.Type, err) + + // Verify broker high water mark. + if index := b.Index(); index != 4 { + t.Fatalf("unexpected broker index: %d", index) } +} - // Read out the published message. - if err := dec.Decode(&m); err != nil { - t.Fatalf("decode: %s", err) - } else if !reflect.DeepEqual(&m, &messaging.Message{Type: 100, TopicID: 20, Index: 4, Data: []byte("0000")}) { - t.Fatalf("unexpected message: %#v", &m) +// Ensure the broker can apply topic high water mark messages. +func TestBroker_Apply_SetMaxTopicIndex(t *testing.T) { + b := OpenBroker() + defer b.Close() + + // Write data to topic so it exists. + if err := b.Apply(&messaging.Message{Index: 1, TopicID: 20}); err != nil { + t.Fatal(err) + } else if b.Topic(20) == nil { + t.Fatal("topic not created") } - // Unsubscribe replica from the topic. - if err := b.Unsubscribe(2000, 20); err != nil { - t.Fatalf("unsubscribe: %s", err) + // Set topic #1's index to "2". + if err := b.Apply(&messaging.Message{ + Index: 2, + Type: messaging.SetTopicMaxIndexMessageType, + Data: []byte{0, 0, 0, 0, 0, 0, 0, 20, 0, 0, 0, 0, 0, 0, 0, 5}, // topicID=20, index=5 + }); err != nil { + t.Fatalf("apply error: %s", err) } + if topic := b.Topic(20); topic.Index() != 5 { + t.Fatalf("unexpected topic index: %d", topic.Index()) + } +} - // Write another message (that shouldn't be read). - if _, err := b.Publish(&messaging.Message{Type: 101, TopicID: 20}); err != nil { - t.Fatalf("unexpected error: %s", err) +// Ensure the broker can read from topics after reopening. +func TestBroker_Reopen(t *testing.T) { + b := OpenBroker() + defer b.Close() + + // Write two message to topic #20, one message to topic #21. + if err := b.Apply(&messaging.Message{Index: 2, TopicID: 20, Data: []byte{0}}); err != nil { + t.Fatal(err) + } else if err := b.Apply(&messaging.Message{Index: 3, TopicID: 21, Data: []byte{100}}); err != nil { + t.Fatal(err) + } else if err := b.Apply(&messaging.Message{Index: 4, TopicID: 20, Data: []byte{200}}); err != nil { + t.Fatal(err) } - time.Sleep(10 * time.Millisecond) - // Read unsubscribe. - if err := dec.Decode(&m); err != nil || m.Type != messaging.UnsubscribeMessageType { - t.Fatalf("decode(unsubscribe): %x (%v)", m.Type, err) + // Reopen the broker. + b.MustReopen() + + // Ensure topics exist. + if b.Topic(20) == nil { + t.Fatal("topic(20) not created") + } else if b.Topic(21) == nil { + t.Fatal("topic(21) not created") + } + + // Verify broker high water mark. + if index := b.Index(); index != 4 { + t.Fatalf("unexpected broker index: %d", index) } - // EOF - if err := dec.Decode(&m); err != io.EOF { - t.Fatalf("decode(eof): %x (%v)", m.Type, err) + // Read messages from topic. + if a := b.MustReadAllTopic(20); len(a) != 2 { + t.Fatalf("unexpected message count: %d", len(a)) + } else if !reflect.DeepEqual(a[0], &messaging.Message{Index: 2, TopicID: 20, Data: []byte{0}}) { + t.Fatalf("unexpected message(0): %d", a[0]) + } else if !reflect.DeepEqual(a[1], &messaging.Message{Index: 4, TopicID: 20, Data: []byte{200}}) { + t.Fatalf("unexpected message(1): %d", a[1]) } } +// Ensure the broker can snapshot and restore its data. +func TestBroker_Snapshot(t *testing.T) { + b0 := OpenBroker() + defer b0.Close() -// Ensure the broker can reopen and recover correctly. -func TestBroker_Reopen(t *testing.T) { - t.Skip("pending") + // Write messages to the first broker. + if err := b0.Apply(&messaging.Message{Index: 2, TopicID: 20, Data: []byte{0}}); err != nil { + t.Fatal(err) + } else if err := b0.Apply(&messaging.Message{Index: 3, TopicID: 21, Data: []byte{100}}); err != nil { + t.Fatal(err) + } else if err := b0.Apply(&messaging.Message{Index: 4, TopicID: 20, Data: []byte{200}}); err != nil { + t.Fatal(err) + } + + // Snapshot the first broker. + var buf bytes.Buffer + if index, err := b0.Snapshot(&buf); err != nil { + t.Fatalf("snapshot error: %s", err) + } else if index != 4 { + t.Fatalf("unexpected snapshot index: %d", index) + } + + // Restore to the second broker. + b1 := OpenBroker() + defer b1.Close() + if err := b1.Restore(&buf); err != nil { + t.Fatalf("restore error: %s", err) + } + + // Ensure topic exists. + if topic := b1.Topic(20); topic == nil { + t.Fatal("topic not created") + } + + // Read message from topic. + r := b1.TopicReader(20, 0, false) + defer r.Close() + dec := messaging.NewMessageDecoder(r) + + var m messaging.Message + if err := dec.Decode(&m); err != nil { + t.Fatalf("message decode error: %s", err) + } else if !reflect.DeepEqual(&m, &messaging.Message{Index: 2, TopicID: 20, Data: []byte{0}}) { + t.Fatalf("unexpected message: %#v", &m) + } + if err := dec.Decode(&m); err != nil { + t.Fatalf("message decode error: %s", err) + } else if !reflect.DeepEqual(&m, &messaging.Message{Index: 4, TopicID: 20, Data: []byte{200}}) { + t.Fatalf("unexpected message: %#v", &m) + } + + // Verify broker high water mark. + if index := b1.Index(); index != 4 { + t.Fatalf("unexpected broker index: %d", index) + } +} + +// Ensure the broker can set the topic high water mark. +func TestBroker_SetTopicMaxIndex(t *testing.T) { + b := OpenBroker() + defer b.Close() + + // Ensure the appropriate message is sent to the log. + b.Log().ApplyFunc = func(data []byte) (uint64, error) { + m, _ := messaging.UnmarshalMessage(data) + if !bytes.Equal(m.Data[0:8], []byte{0, 0, 0, 0, 0, 0, 0, 1}) { + t.Fatalf("unexpected topic id data: %x", data[0:8]) + } else if !bytes.Equal(m.Data[8:16], []byte{0, 0, 0, 0, 0, 0, 0, 2}) { + t.Fatalf("unexpected index data: %x", data[8:16]) + } + return 1, nil + } + + // Set the highest replicated topic index. + if err := b.SetTopicMaxIndex(1, 2); err != nil { + t.Fatal(err) + } } -// Benchmarks a single broker without HTTP. -func BenchmarkBroker_Publish(b *testing.B) { - br := NewBroker(nil) - defer br.Close() +// Ensure the FSM can apply messages. +func TestRaftFSM_MustApply_Message(t *testing.T) { + fsm := NewRaftFSM() + + // Ensure index is added to message. + var called bool + fsm.Broker().ApplyFunc = func(m *messaging.Message) error { + if !reflect.DeepEqual(m, &messaging.Message{Index: 2, TopicID: 20, Data: []byte{}}) { + t.Fatalf("unexpected message: %#v", m) + } + called = true + return nil + } + + // Encode message and apply it as a log entry. + m := messaging.Message{TopicID: 20} + data, _ := m.MarshalBinary() + fsm.MustApply(&raft.LogEntry{Index: 2, Data: data}) + if !called { + t.Fatal("Apply() not called") + } +} - b.ResetTimer() +// Ensure the FSM can move the index forward with raft internal messages. +func TestRaftFSM_MustApply_Internal(t *testing.T) { + fsm := NewRaftFSM() - var index uint64 - for i := 0; i < b.N; i++ { - var err error - index, err = br.Publish(&messaging.Message{Type: 0, TopicID: 1, Data: make([]byte, 50)}) - if err != nil { - b.Fatalf("unexpected error: %s", err) + // Ensure index is added to message. + var called bool + fsm.Broker().SetMaxIndexFunc = func(index uint64) error { + if index != 2 { + t.Fatalf("unexpected index: %#v", index) } + called = true + return nil + } + + // Encode message and apply it as a log entry. + fsm.MustApply(&raft.LogEntry{Type: raft.LogEntryAddPeer, Index: 2}) + if !called { + t.Fatal("Apply() not called") + } +} + +// RaftFSM is a mockable wrapper around messaging.RaftFSM. +type RaftFSM struct { + *messaging.RaftFSM +} + +// NewRaftFSM returns a new instance of RaftFSM. +func NewRaftFSM() *RaftFSM { + return &RaftFSM{ + &messaging.RaftFSM{Broker: &RaftFSMBroker{}}, } +} + +func (fsm *RaftFSM) Broker() *RaftFSMBroker { return fsm.RaftFSM.Broker.(*RaftFSMBroker) } + +// RaftFSMBroker is a mockable object implementing RaftFSM.Broker. +type RaftFSMBroker struct { + ApplyFunc func(*messaging.Message) error + SetMaxIndexFunc func(uint64) error +} + +func (b *RaftFSMBroker) Apply(m *messaging.Message) error { return b.ApplyFunc(m) } +func (b *RaftFSMBroker) SetMaxIndex(index uint64) error { return b.SetMaxIndexFunc(index) } + +func (b *RaftFSMBroker) Index() (uint64, error) { return 0, nil } +func (b *RaftFSMBroker) Snapshot(w io.Writer) (uint64, error) { return 0, nil } +func (b *RaftFSMBroker) Restore(r io.Reader) error { return nil } + +// Ensure a list of topics can be read from a directory. +func TestReadTopics(t *testing.T) { + path, _ := ioutil.TempDir("", "") + defer os.RemoveAll(path) + + MustWriteFile(filepath.Join(path, "1/12"), []byte{}) // ok + MustWriteFile(filepath.Join(path, "1/20"), []byte{}) // ok + MustWriteFile(filepath.Join(path, "312/35"), []byte{}) // ok + MustWriteFile(filepath.Join(path, "meta/data"), []byte{}) // non-numeric name + MustWriteFile(filepath.Join(path, "123"), []byte{}) // non-directory - // Wait for the broker to commit. - if err := br.Sync(index); err != nil { - b.Fatalf("sync error: %s", err) + a, err := messaging.ReadTopics(path) + if err != nil { + t.Fatal(err) + } else if len(a) != 2 { + t.Fatalf("unexpected count: %d", len(a)) + } else if a[0].ID() != 1 { + t.Fatalf("unexpected topic(0) id: %d", a[0].ID()) + } else if a[0].Path() != filepath.Join(path, "1") { + t.Fatalf("unexpected topic(0) path: %s", a[0].Path()) + } else if a[1].ID() != 312 { + t.Fatalf("unexpected topic(1) id: %d", a[1].ID()) } } -*/ // Ensure a list of segments can be read from a directory. func TestReadSegments(t *testing.T) { @@ -165,6 +364,17 @@ func TestReadSegments(t *testing.T) { } } +// Ensure a list of segments returns an error if the path doesn't exist. +func TestReadSegments_ENOENT(t *testing.T) { + path, _ := ioutil.TempDir("", "") + os.RemoveAll(path) + + _, err := messaging.ReadSegments(path) + if err == nil || !strings.Contains(err.Error(), "no such file or directory") { + t.Fatal(err) + } +} + // Ensure the appropriate segment can be found by index. func TestReadSegmentByIndex(t *testing.T) { path, _ := ioutil.TempDir("", "") @@ -180,17 +390,53 @@ func TestReadSegmentByIndex(t *testing.T) { err error }{ {index: 0, segmentIndex: 6}, + {index: 6, segmentIndex: 6}, + {index: 7, segmentIndex: 6}, + {index: 11, segmentIndex: 6}, + {index: 12, segmentIndex: 12}, + {index: 13, segmentIndex: 12}, + {index: 19, segmentIndex: 12}, + {index: 20, segmentIndex: 20}, + {index: 21, segmentIndex: 20}, {index: 5, segmentIndex: 6, err: messaging.ErrSegmentReclaimed}, } { segment, err := messaging.ReadSegmentByIndex(path, tt.index) - if err != nil { - t.Errorf("%d. %d: error: %s", i, tt.index, err) + if tt.err != nil { + if tt.err != err { + t.Errorf("%d. %d: error mismatch: exp=%s, got=%s", i, tt.index, tt.err, err) + } + } else if err != nil { + t.Errorf("%d. %d: unexpected error: %s", i, tt.index, err) } else if tt.segmentIndex != segment.Index { - t.Errorf("%d. %d: index mismatch: exp=%d got=%d", i, tt.index, tt.segmentIndex, segment.Index) + t.Errorf("%d. %d: index mismatch: exp=%d, got=%d", i, tt.index, tt.segmentIndex, segment.Index) } } } +// Ensure reading a segment by index with no segments returns a nil segment. +func TestReadSegmentByIndex_NoSegment(t *testing.T) { + path, _ := ioutil.TempDir("", "") + defer os.RemoveAll(path) + + segment, err := messaging.ReadSegmentByIndex(path, 0) + if err != nil { + t.Fatalf("unexpected error: %s", err) + } else if segment != nil { + t.Fatalf("expected nil segment: %#v", segment) + } +} + +// Ensure reading a segment by index fails if the path doesn't exist. +func TestReadSegmentByIndex_ENOENT(t *testing.T) { + path, _ := ioutil.TempDir("", "") + os.RemoveAll(path) + + _, err := messaging.ReadSegmentByIndex(path, 0) + if err == nil || !strings.Contains(err.Error(), "no such file or directory") { + t.Fatalf("unexpected error: %s", err) + } +} + // Ensure a topic reader can read messages in order from a given index. func TestTopicReader(t *testing.T) { path, _ := ioutil.TempDir("", "") @@ -222,6 +468,15 @@ func TestTopicReader(t *testing.T) { results []uint64 // returned indices }{ {index: 0, results: []uint64{6, 7, 10, 12, 13, 14}}, + {index: 6, results: []uint64{6, 7, 10, 12, 13, 14}}, + {index: 7, results: []uint64{7, 10, 12, 13, 14}}, + {index: 9, results: []uint64{10, 12, 13, 14}}, + {index: 10, results: []uint64{10, 12, 13, 14}}, + {index: 11, results: []uint64{12, 13, 14}}, + {index: 12, results: []uint64{12, 13, 14}}, + {index: 13, results: []uint64{13, 14}}, + {index: 14, results: []uint64{14}}, + {index: 15, results: []uint64{}}, } { // Start topic reader from an index. r := messaging.NewTopicReader(path, tt.index, false) @@ -242,7 +497,7 @@ func TestTopicReader(t *testing.T) { // Verify the retrieved indices match what's expected. if !reflect.DeepEqual(results, tt.results) { - t.Fatalf("%d. result mismatch:\n\nexp=%#v\n\ngot=%#v", i, tt.results, results) + t.Fatalf("%d. %v: result mismatch:\n\nexp=%#v\n\ngot=%#v", i, tt.index, tt.results, results) } } @@ -253,28 +508,20 @@ type Broker struct { *messaging.Broker } -// NewBroker returns a new open tempoarary broker. -func NewBroker(u *url.URL) *Broker { - b := NewUninitializedBroker(u) - if err := b.Initialize(); err != nil { - panic("initialize: " + err.Error()) - } +// NewBroker returns a new Broker instance with a mockable log. +func NewBroker() *Broker { + b := &Broker{messaging.NewBroker()} + b.Broker.Log = &BrokerLog{} return b } -// NewUninitializedBroker returns a new broker that has not been initialized. -func NewUninitializedBroker(u *url.URL) *Broker { - // Default the broker URL if not passed in. - if u == nil { - u = &url.URL{Scheme: "http", Host: "127.0.0.1:8080"} - } - - // Open a new broker. - b := messaging.NewBroker() - if err := b.Open(tempfile(), u); err != nil { +// OpenBroker returns a new, open Broker instance. +func OpenBroker() *Broker { + b := NewBroker() + if err := b.Open(tempfile()); err != nil { panic("open: " + err.Error()) } - return &Broker{b} + return b } // Close closes and deletes the temporary broker. @@ -283,13 +530,24 @@ func (b *Broker) Close() { b.Broker.Close() } -// MustSync syncs to a broker index. Panic on error. -func (b *Broker) MustSync(index uint64) { - if err := b.Sync(index); err != nil { - panic(err.Error()) +// MustReopen closes and reopens the broker in a new instance. +func (b *Broker) MustReopen() { + path, log := b.Broker.Path(), b.Broker.Log + b.Broker.Close() + + b.Broker = messaging.NewBroker() + b.Broker.Log = log + + if err := b.Open(path); err != nil { + panic("reopen: " + err.Error()) } } +// Log returns the mock broker log on the underlying broker. +func (b *Broker) Log() *BrokerLog { + return b.Broker.Log.(*BrokerLog) +} + // MustPublish publishes a message to the broker. Panic on error. func (b *Broker) MustPublish(m *messaging.Message) uint64 { index, err := b.Publish(&messaging.Message{Type: 100, TopicID: 20, Data: []byte("0000")}) @@ -299,13 +557,30 @@ func (b *Broker) MustPublish(m *messaging.Message) uint64 { return index } -// MustPublishSync publishes a message to the broker and syncs to that index. Panic on error. -func (b *Broker) MustPublishSync(m *messaging.Message) uint64 { - index := b.MustPublish(m) - b.MustSync(index) - return index +// MustReadAllTopic reads all messages on a topic. Panic on error. +func (b *Broker) MustReadAllTopic(topicID uint64) (a []*messaging.Message) { + r := b.TopicReader(topicID, 0, false) + defer r.Close() + + dec := messaging.NewMessageDecoder(r) + for { + m := &messaging.Message{} + if err := dec.Decode(m); err == io.EOF { + return + } else if err != nil { + panic("read all topic: " + err.Error()) + } + a = append(a, m) + } } +// BrokerLog is a mockable object that implements Broker.Log. +type BrokerLog struct { + ApplyFunc func(data []byte) (uint64, error) +} + +func (l *BrokerLog) Apply(data []byte) (uint64, error) { return l.ApplyFunc(data) } + // Messages represents a collection of messages. // This type provides helper functions. type Messages []*messaging.Message @@ -337,6 +612,9 @@ func tempfile() string { // MustWriteFile writes data to a file. Panic on error. func MustWriteFile(filename string, data []byte) { + if err := os.MkdirAll(filepath.Dir(filename), 0700); err != nil { + panic(err.Error()) + } if err := ioutil.WriteFile(filename, data, 0600); err != nil { panic(err.Error()) } diff --git a/messaging/client.go b/messaging/client.go index 6fc2be02e35..679e9c7f5a1 100644 --- a/messaging/client.go +++ b/messaging/client.go @@ -35,18 +35,14 @@ func NewClientConfig(u []*url.URL) *ClientConfig { } // Client represents a client for the broker's HTTP API. -// Once opened, the client will stream down all messages that type Client struct { - mu sync.Mutex - replicaID uint64 // the replica that the client is connecting as. - config ClientConfig // The Client state that must be persisted to disk. + mu sync.Mutex + conns []*Conn + config ClientConfig // The Client state that must be persisted to disk. opened bool done chan chan struct{} // disconnection notification - c chan *Message // channel streams messages from the broker. - index uint64 // highest index sent over the channel - // The amount of time to wait before reconnecting to a broker stream. ReconnectTimeout time.Duration @@ -54,28 +50,13 @@ type Client struct { Logger *log.Logger } -// NewClient returns a new instance of Client. -func NewClient(replicaID uint64) *Client { - return &Client{ - replicaID: replicaID, +// NewClient returns a new instance of Client with defaults set. +func NewClient() *Client { + c := &Client{ ReconnectTimeout: DefaultReconnectTimeout, - Logger: log.New(os.Stderr, "[messaging] ", log.LstdFlags), } -} - -// ReplicaID returns the replica id that the client was opened with. -func (c *Client) ReplicaID() uint64 { return c.replicaID } - -// C returns streaming channel. -// Messages can be duplicated so it is important to check the index -// of the incoming message index to make sure it has not been processed. -func (c *Client) C() <-chan *Message { return c.c } - -// Index returns the highest index sent over the channel. -func (c *Client) Index() uint64 { - c.mu.Lock() - defer c.mu.Unlock() - return c.index + c.SetLogOutput(os.Stderr) + return c } // URLs returns a list of broker URLs to connect to. @@ -232,22 +213,77 @@ func (c *Client) Publish(m *Message) (uint64, error) { return index, nil } +// Conn represents a stream over the client for a single topic. +type Conn struct { + topicID uint64 // topic identifier + index uint64 // highest index sent over the channel + url url.URL // current broker url + + c chan *Message // channel streams messages from the broker. + reconnect chan struct{} // notification channel for broker change. +} + +// NewConn returns a new connection to the broker for a topic. +func NewConn(topicID uint64, index uint64) *Conn { + return &Conn{ + topicID: topicID, + index: index, + } +} + +// TopicID returns the connection's topic id. +func (c *Conn) TopicID() uint64 { return c.topicID } + +// C returns streaming channel for the connection. +func (c *Conn) C() <-chan *Message { return c.c } + +// Index returns the highest index sent over the channel. +func (c *Conn) Index() uint64 { + c.mu.Lock() + defer c.mu.Unlock() + return c.index +} + +// URL returns the current URL of the connection. +func (c *Conn) URL() url.URL { + c.mu.Lock() + defer c.mu.Unlock() + return c.url +} + +// SetURL sets the current URL of the connection. +func (c *Conn) SetURL(u url.URL) { + c.mu.Lock() + defer c.mu.Unlock() + + c.url = u + + // Notify streamer of change. + select { + case c.reconnect <- struct{}{}: + default: + } +} + +// Open opens a streaming connection to the broker. +func (c *Conn) Open() error { +} + // Heartbeat sends a heartbeat back to the broker with the client's index. -func (c *Client) Heartbeat() error { +func (c *Conn) Heartbeat() error { var resp *http.Response var err error // Retrieve the parameters under lock. c.mu.Lock() - replicaID, index := c.replicaID, c.index + topicID, index, u := c.topicID, c.index, c.url c.mu.Unlock() - u := *c.LeaderURL() // Send the message to the messages endpoint. u.Path = "/messaging/heartbeat" u.RawQuery = url.Values{ - "replicaID": {strconv.FormatUint(replicaID, 10)}, - "index": {strconv.FormatUint(index, 10)}, + "topicID": {strconv.FormatUint(topicID, 10)}, + "index": {strconv.FormatUint(index, 10)}, }.Encode() resp, err = http.Post(u.String(), "application/octet-stream", nil) if err != nil { @@ -268,151 +304,8 @@ func (c *Client) Heartbeat() error { return nil } -// CreateReplica creates a replica on the broker. -func (c *Client) CreateReplica(id uint64, u *url.URL) error { - var resp *http.Response - var err error - - leaderURL := *c.LeaderURL() - for { - leaderURL.Path = "/messaging/replicas" - leaderURL.RawQuery = url.Values{ - "id": {strconv.FormatUint(id, 10)}, - "url": {u.String()}, - }.Encode() - resp, err = http.Post(leaderURL.String(), "application/octet-stream", nil) - if err != nil { - return err - } - defer func() { _ = resp.Body.Close() }() - - // If a temporary redirect occurs then update the leader and retry. - // If a non-201 status is returned then an error occurred. - if resp.StatusCode == http.StatusTemporaryRedirect { - redirectURL, err := url.Parse(resp.Header.Get("Location")) - if err != nil { - return fmt.Errorf("bad redirect: %s", resp.Header.Get("Location")) - } - leaderURL = *redirectURL - continue - } else if resp.StatusCode != http.StatusCreated { - return errors.New(resp.Header.Get("X-Broker-Error")) - } - break - } - - return nil -} - -// DeleteReplica removes a replica on the broker. -func (c *Client) DeleteReplica(id uint64) error { - var resp *http.Response - var err error - - u := *c.LeaderURL() - for { - u.Path = "/messaging/replicas" - u.RawQuery = url.Values{"id": {strconv.FormatUint(id, 10)}}.Encode() - req, _ := http.NewRequest("DELETE", u.String(), nil) - resp, err = http.DefaultClient.Do(req) - if err != nil { - return err - } - defer func() { _ = resp.Body.Close() }() - - // If a temporary redirect occurs then update the leader and retry. - // If a non-204 status is returned then an error occurred. - if resp.StatusCode == http.StatusTemporaryRedirect { - redirectURL, err := url.Parse(resp.Header.Get("Location")) - if err != nil { - return fmt.Errorf("bad redirect: %s", resp.Header.Get("Location")) - } - u = *redirectURL - continue - } else if resp.StatusCode != http.StatusNoContent { - return errors.New(resp.Header.Get("X-Broker-Error")) - } - break - } - - return nil -} - -// Subscribe subscribes a replica to a topic on the broker. -func (c *Client) Subscribe(replicaID, topicID uint64) error { - var resp *http.Response - var err error - - u := *c.LeaderURL() - for { - u.Path = "/messaging/subscriptions" - u.RawQuery = url.Values{ - "replicaID": {strconv.FormatUint(replicaID, 10)}, - "topicID": {strconv.FormatUint(topicID, 10)}, - }.Encode() - resp, err = http.Post(u.String(), "application/octet-stream", nil) - if err != nil { - return err - } - defer func() { _ = resp.Body.Close() }() - - // If a temporary redirect occurs then update the leader and retry. - // If a non-201 status is returned then an error occurred. - if resp.StatusCode == http.StatusTemporaryRedirect { - redirectURL, err := url.Parse(resp.Header.Get("Location")) - if err != nil { - return fmt.Errorf("bad redirect: %s", resp.Header.Get("Location")) - } - u = *redirectURL - continue - } else if resp.StatusCode != http.StatusCreated { - return errors.New(resp.Header.Get("X-Broker-Error")) - } - break - } - - return nil -} - -// Unsubscribe unsubscribes a replica from a topic on the broker. -func (c *Client) Unsubscribe(replicaID, topicID uint64) error { - var resp *http.Response - var err error - - u := *c.LeaderURL() - for { - u.Path = "/messaging/subscriptions" - u.RawQuery = url.Values{ - "replicaID": {strconv.FormatUint(replicaID, 10)}, - "topicID": {strconv.FormatUint(topicID, 10)}, - }.Encode() - req, _ := http.NewRequest("DELETE", u.String(), nil) - resp, err = http.DefaultClient.Do(req) - if err != nil { - return err - } - defer func() { _ = resp.Body.Close() }() - - // If a temporary redirect occurs then update the leader and retry. - // If a non-204 status is returned then an error occurred. - if resp.StatusCode == http.StatusTemporaryRedirect { - redirectURL, err := url.Parse(resp.Header.Get("Location")) - if err != nil { - return fmt.Errorf("bad redirect: %s", resp.Header.Get("Location")) - } - u = *redirectURL - continue - } else if resp.StatusCode != http.StatusNoContent { - return errors.New(resp.Header.Get("X-Broker-Error")) - } - break - } - - return nil -} - // streamer connects to a broker server and streams the replica's messages. -func (c *Client) streamer(done chan chan struct{}) { +func (c *Conn) streamer(done chan chan struct{}) { for { // Check for the client disconnection. select { @@ -424,11 +317,8 @@ func (c *Client) streamer(done chan chan struct{}) { // TODO: Validate that there is at least one broker URL. - // Choose a random broker URL. - urls := c.URLs() - u := *urls[rand.Intn(len(urls))] - // Connect to broker and stream. + u := c.URL() u.Path = "/messaging/messages" if err := c.streamFromURL(&u, done); err == errDone { return diff --git a/messaging/handler.go b/messaging/handler.go index d91118c162c..c302da83357 100644 --- a/messaging/handler.go +++ b/messaging/handler.go @@ -4,6 +4,7 @@ import ( "io" "io/ioutil" "net/http" + "net/url" "strconv" "strings" @@ -12,38 +13,21 @@ import ( // Handler represents an HTTP handler by the broker. type Handler struct { - raftHandler *raft.Handler - broker *Broker -} - -// NewHandler returns a new instance of Handler. -func NewHandler(b *Broker) *Handler { - h := &Handler{} - h.SetBroker(b) - return h -} - -// Broker returns the broker on the handler. -func (h *Handler) Broker() *Broker { return h.broker } - -// SetBroker sets the broker on the handler. -func (h *Handler) SetBroker(b *Broker) { - h.broker = b - - if b != nil { - h.raftHandler = &raft.Handler{Log: b.log} - } else { - h.raftHandler = nil + Broker interface { + LeaderURL() *url.URL + TopicReader(topicID, index uint64, streaming bool) io.ReadCloser + Publish(m *Message) (uint64, error) + SetTopicMaxIndex(topicID, index uint64) error } + + RaftHandler http.Handler } // ServeHTTP serves an HTTP request. func (h *Handler) ServeHTTP(w http.ResponseWriter, r *http.Request) { - // h.broker.Logger.Printf("%s %s", r.Method, r.URL.String()) - // Delegate raft requests to its own handler. if strings.HasPrefix(r.URL.Path, "/raft") { - h.raftHandler.ServeHTTP(w, r) + h.RaftHandler.ServeHTTP(w, r) return } @@ -88,7 +72,7 @@ func (h *Handler) getMessages(w http.ResponseWriter, req *http.Request) { streaming := (req.URL.Query().Get("streaming") == "true") // Create a topic reader. - r := NewTopicReader(h.broker.TopicPath(topicID), index, streaming) + r := h.Broker.TopicReader(topicID, index, streaming) defer r.Close() // Ensure we close the topic reader if the connection is disconnected. @@ -128,7 +112,7 @@ func (h *Handler) postMessages(w http.ResponseWriter, r *http.Request) { } // Publish message to the broker. - index, err := h.broker.Publish(&Message{Type: MessageType(typ), TopicID: topicID, Data: data}) + index, err := h.Broker.Publish(&Message{Type: MessageType(typ), TopicID: topicID, Data: data}) if err == raft.ErrNotLeader { h.redirectToLeader(w, r) return @@ -159,7 +143,7 @@ func (h *Handler) postHeartbeat(w http.ResponseWriter, r *http.Request) { } // Update the topic's highest replicated index. - if err := h.broker.SetTopicMaxIndex(topicID, index); err == raft.ErrNotLeader { + if err := h.Broker.SetTopicMaxIndex(topicID, index); err == raft.ErrNotLeader { h.redirectToLeader(w, r) return } else if err != nil { @@ -178,7 +162,7 @@ func (h *Handler) error(w http.ResponseWriter, err error, code int) { // redirects to the current known leader. // If no leader is found then returns a 500. func (h *Handler) redirectToLeader(w http.ResponseWriter, r *http.Request) { - if u := h.broker.LeaderURL(); u != nil { + if u := h.Broker.LeaderURL(); u != nil { redirectURL := *r.URL redirectURL.Scheme = u.Scheme redirectURL.Host = u.Host diff --git a/messaging/handler_test.go b/messaging/handler_test.go index cdb3790a211..9cf307d23ca 100644 --- a/messaging/handler_test.go +++ b/messaging/handler_test.go @@ -1,119 +1,219 @@ package messaging_test import ( + "bytes" + "io" "net/http" "net/http/httptest" "net/url" + "reflect" + "strings" "testing" - "time" "github.com/influxdb/influxdb/messaging" ) // Ensure a topic can be streamed from an index. func TestHandler_getMessages(t *testing.T) { - s := NewServer() + var hb HandlerBroker + hb.TopicReaderFunc = func(topicID, index uint64, streaming bool) io.ReadCloser { + if topicID != 2000 { + t.Fatalf("unexpected topic id: %d", topicID) + } else if index != 10 { + t.Fatalf("unexpected index: %d", index) + } else if !streaming { + t.Fatalf("unexpected streaming value: %v", streaming) + } + + // Return a reader with one message. + var buf bytes.Buffer + (&messaging.Message{Index: 10, Data: []byte{0, 0, 0, 0}}).WriteTo(&buf) + + return &bytesBufferCloser{buf} + } + s := httptest.NewServer(&messaging.Handler{Broker: &hb}) defer s.Close() // Send request to stream the replica. - resp, err := http.Get(s.URL + `/messaging/messages?topicID=2000`) - defer resp.Body.Close() + resp, err := http.Get(s.URL + `/messaging/messages?topicID=2000&index=10&streaming=true`) if err != nil { t.Fatalf("unexpected error: %s", err) } else if resp.StatusCode != http.StatusOK { t.Fatalf("unexpected status: %d: %s", resp.StatusCode, resp.Header.Get("X-Broker-Error")) } - time.Sleep(10 * time.Millisecond) + defer resp.Body.Close() - // TODO: Decode from body. + // Decode from body. + var m messaging.Message + if err := messaging.NewMessageDecoder(resp.Body).Decode(&m); err != nil { + t.Fatalf("message decode error: %s", err) + } else if !reflect.DeepEqual(&m, &messaging.Message{Index: 10, Data: []byte{0, 0, 0, 0}}) { + t.Fatalf("unexpected message: %#v", &m) + } } -// Ensure an error is returned when requesting a stream with the wrong HTTP method. -func TestHandler_stream_ErrMethodNotAllowed(t *testing.T) { - s := NewServer() +// Ensure a handler returns an error when streaming messages without a topic id. +func TestHandler_getMessages_ErrTopicRequired(t *testing.T) { + s := httptest.NewServer(&messaging.Handler{}) defer s.Close() - resp, _ := http.Head(s.URL + `/messaging/messages`) - defer resp.Body.Close() - if resp.StatusCode != http.StatusMethodNotAllowed { + // Send request to the broker. + resp, err := http.Get(s.URL + `/messaging/messages?index=10`) + if err != nil { + t.Fatal(err) + } else if resp.StatusCode != http.StatusBadRequest { t.Fatalf("unexpected status: %d", resp.StatusCode) + } else if resp.Header.Get("X-Broker-Error") != "topic required" { + t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) } + resp.Body.Close() +} + +// Ensure a handler returns an error when streaming messages without an index. +func TestHandler_getMessages_ErrIndexRequired(t *testing.T) { + s := httptest.NewServer(&messaging.Handler{}) + defer s.Close() + + // Send request to the broker. + resp, err := http.Get(s.URL + `/messaging/messages?topicID=10`) + if err != nil { + t.Fatal(err) + } else if resp.StatusCode != http.StatusBadRequest { + t.Fatalf("unexpected status: %d", resp.StatusCode) + } else if resp.Header.Get("X-Broker-Error") != "index required" { + t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) + } + resp.Body.Close() } // Ensure a handler can receive a message. func TestHandler_postMessages(t *testing.T) { - t.Skip("pending") - /* - s := NewServer() - defer s.Close() - - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/messages?type=100&topicID=200`, "application/octet-stream", strings.NewReader(`abc`)) - defer resp.Body.Close() - if resp.StatusCode != http.StatusOK { - t.Fatalf("unexpected status: %d: %s", resp.StatusCode, resp.Header.Get("X-Broker-Error")) + var hb HandlerBroker + hb.PublishFunc = func(m *messaging.Message) (uint64, error) { + if !reflect.DeepEqual(m, &messaging.Message{Type: 100, TopicID: 200, Data: []byte(`abc`)}) { + t.Fatalf("unexpected message: %#v", m) } - s.Handler.Broker().Sync(4) + return 1, nil + } + s := httptest.NewServer(&messaging.Handler{Broker: &hb}) + defer s.Close() - // Check if the last message received is our new message. - time.Sleep(10 * time.Millisecond) - if !reflect.DeepEqual(&m, &messaging.Message{Type: 100, Index: 4, TopicID: 200, Data: []byte("abc")}) { - t.Fatalf("unexpected message: %#v", &m) - } - */ + // Send request to the broker. + resp, err := http.Post(s.URL+`/messaging/messages?type=100&topicID=200`, "application/octet-stream", strings.NewReader(`abc`)) + if err != nil { + t.Fatal(err) + } else if resp.StatusCode != http.StatusOK { + t.Fatalf("unexpected status: %d: %s", resp.StatusCode, resp.Header.Get("X-Broker-Error")) + } + resp.Body.Close() } // Ensure a handler returns an error when publishing a message without a type. -func TestHandler_publish_ErrMessageTypeRequired(t *testing.T) { - t.Skip("pending") - /* - s := NewServer() - defer s.Close() - - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/messages?topicID=200`, "application/octet-stream", strings.NewReader(`foo`)) - defer resp.Body.Close() - if resp.StatusCode != http.StatusBadRequest { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "message type required" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) - } - */ +func TestHandler_postMessages_ErrMessageTypeRequired(t *testing.T) { + s := httptest.NewServer(&messaging.Handler{}) + defer s.Close() + + // Send request to the broker. + resp, err := http.Post(s.URL+`/messaging/messages?topicID=200`, "application/octet-stream", strings.NewReader(`foo`)) + if err != nil { + t.Fatal(err) + } else if resp.StatusCode != http.StatusBadRequest { + t.Fatalf("unexpected status: %d", resp.StatusCode) + } else if resp.Header.Get("X-Broker-Error") != "message type required" { + t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) + } + resp.Body.Close() } // Ensure a handler returns an error when publishing a message without a topic. -func TestHandler_publish_ErrTopicRequired(t *testing.T) { - t.Skip("pending") - - /* - s := NewServer() - defer s.Close() - - // Send request to the broker. - resp, _ := http.Post(s.URL+`/messaging/messages?type=100`, "application/octet-stream", strings.NewReader(`foo`)) - defer resp.Body.Close() - if resp.StatusCode != http.StatusBadRequest { - t.Fatalf("unexpected status: %d", resp.StatusCode) - } else if resp.Header.Get("X-Broker-Error") != "topic required" { - t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) +func TestHandler_postMessages_ErrTopicRequired(t *testing.T) { + s := httptest.NewServer(&messaging.Handler{}) + defer s.Close() + + // Send request to the broker. + resp, err := http.Post(s.URL+`/messaging/messages?type=100`, "application/octet-stream", strings.NewReader(`foo`)) + if err != nil { + t.Fatal(err) + } else if resp.StatusCode != http.StatusBadRequest { + t.Fatalf("unexpected status: %d", resp.StatusCode) + } else if resp.Header.Get("X-Broker-Error") != "topic required" { + t.Fatalf("unexpected error: %s", resp.Header.Get("X-Broker-Error")) + } + resp.Body.Close() +} + +// Ensure an error is returned when requesting a stream with the wrong HTTP method. +func TestHandler_messages_ErrMethodNotAllowed(t *testing.T) { + s := httptest.NewServer(&messaging.Handler{}) + defer s.Close() + + // Send request to stream the replica. + resp, err := http.Head(s.URL + `/messaging/messages?topicID=2000&index=10&streaming=true`) + if err != nil { + t.Fatalf("unexpected error: %s", err) + } else if resp.StatusCode != http.StatusMethodNotAllowed { + t.Fatalf("unexpected status: %d", resp.StatusCode) + } + resp.Body.Close() +} + +// Ensure a handler can receive a heartbeats. +func TestHandler_postHeartbeat(t *testing.T) { + var hb HandlerBroker + hb.SetTopicMaxIndexFunc = func(topicID, index uint64) error { + if topicID != 1 { + t.Fatalf("unexpected topic id: %d", topicID) + } else if index != 2 { + t.Fatalf("unexpected index: %d", index) } - */ + return nil + } + s := httptest.NewServer(&messaging.Handler{Broker: &hb}) + defer s.Close() + + // Send request to the broker. + resp, err := http.Post(s.URL+`/messaging/heartbeat?topicID=1&index=2`, "application/octet-stream", nil) + if err != nil { + t.Fatal(err) + } else if resp.StatusCode != http.StatusOK { + t.Fatalf("unexpected status: %d: %s", resp.StatusCode, resp.Header.Get("X-Broker-Error")) + } + resp.Body.Close() +} + +// Ensure an error is returned when heartbeating with the wrong HTTP method. +func heartbeat_ErrMethodNotAllowed(t *testing.T) { + s := httptest.NewServer(&messaging.Handler{}) + defer s.Close() + + resp, err := http.Head(s.URL + `/messaging/heartbeat`) + if err != nil { + t.Fatalf("unexpected error: %s", err) + } else if resp.StatusCode != http.StatusMethodNotAllowed { + t.Fatalf("unexpected status: %d", resp.StatusCode) + } + resp.Body.Close() } // Ensure the handler routes raft requests to the raft handler. func TestHandler_raft(t *testing.T) { - s := NewServer() + var h messaging.Handler + h.RaftHandler = http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + w.WriteHeader(http.StatusAccepted) + }) + s := httptest.NewServer(&h) defer s.Close() + resp, _ := http.Get(s.URL + `/raft/ping`) defer resp.Body.Close() - if resp.StatusCode != http.StatusOK { + if resp.StatusCode != http.StatusAccepted { t.Fatalf("unexpected status: %d", resp.StatusCode) } } // Ensure the handler returns an error for an invalid path. func TestHandler_ErrNotFound(t *testing.T) { - s := NewServer() + s := httptest.NewServer(&messaging.Handler{}) defer s.Close() resp, _ := http.Get(s.URL + `/no_such_path`) defer resp.Body.Close() @@ -122,37 +222,23 @@ func TestHandler_ErrNotFound(t *testing.T) { } } -// Server is an test HTTP server that wraps a handler and broker. -type Server struct { - *httptest.Server - Handler *messaging.Handler -} - -// NewServer returns a test server. -func NewServer() *Server { - h := messaging.NewHandler(nil) - s := httptest.NewServer(h) - h.SetBroker(NewBroker(MustParseURL(s.URL)).Broker) - return &Server{s, h} +// HandlerBroker is a mockable type that implements Handler.Broker. +type HandlerBroker struct { + LeaderURLFunc func() *url.URL + PublishFunc func(m *messaging.Message) (uint64, error) + TopicReaderFunc func(topicID, index uint64, streaming bool) io.ReadCloser + SetTopicMaxIndexFunc func(topicID, index uint64) error } -// NewUninitializedServer returns a test server with an uninitialized broker. -func NewUninitializedServer() *Server { - h := messaging.NewHandler(nil) - s := httptest.NewServer(h) - h.SetBroker(NewUninitializedBroker(MustParseURL(s.URL)).Broker) - return &Server{s, h} +func (b *HandlerBroker) LeaderURL() *url.URL { return b.LeaderURLFunc() } +func (b *HandlerBroker) Publish(m *messaging.Message) (uint64, error) { return b.PublishFunc(m) } +func (b *HandlerBroker) TopicReader(topicID, index uint64, streaming bool) io.ReadCloser { + return b.TopicReaderFunc(topicID, index, streaming) } - -// Close stops the server and broker and removes all temp data. -func (s *Server) Close() { - s.Broker().Close() - s.Server.Close() +func (b *HandlerBroker) SetTopicMaxIndex(topicID, index uint64) error { + return b.SetTopicMaxIndexFunc(topicID, index) } -// Broker returns a reference to the broker attached to the handler. -func (s *Server) Broker() *Broker { return &Broker{s.Handler.Broker()} } - // MustParseURL parses a string into a URL. Panic on error. func MustParseURL(s string) *url.URL { u, err := url.Parse(s) @@ -161,3 +247,9 @@ func MustParseURL(s string) *url.URL { } return u } + +type bytesBufferCloser struct { + bytes.Buffer +} + +func (*bytesBufferCloser) Close() error { return nil } diff --git a/messaging/intg_test.go b/messaging/intg_test.go index 1f20d953cb6..a3c59d287ce 100644 --- a/messaging/intg_test.go +++ b/messaging/intg_test.go @@ -1,8 +1,8 @@ package messaging_test import ( - "fmt" - "net/url" + // "fmt" + // "net/url" "testing" //"github.com/influxdb/influxdb/messaging" @@ -93,6 +93,7 @@ func BenchmarkCluster_Publish(b *testing.B) { */ } +/* // Cluster represents a set of joined Servers. type Cluster struct { Servers []*Server @@ -134,3 +135,4 @@ func (c *Cluster) Close() { s.Close() } } +*/ From 9b5aeb1a907ac499537ed34146ce8c4fd8ed140f Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 7 Mar 2015 23:21:44 -0700 Subject: [PATCH 06/20] Refactor messaging client/conn. --- messaging/client.go | 427 ++++++++++++++++++++++++--------------- messaging/client_test.go | 188 +++++++++++++++-- messaging/errors.go | 9 + 3 files changed, 446 insertions(+), 178 deletions(-) diff --git a/messaging/client.go b/messaging/client.go index 679e9c7f5a1..1da2a90f693 100644 --- a/messaging/client.go +++ b/messaging/client.go @@ -21,24 +21,12 @@ import ( // stream disconnects and another connection is retried. const DefaultReconnectTimeout = 100 * time.Millisecond -// ClientConfig represents the Client configuration that must be persisted -// across restarts. -type ClientConfig struct { - Brokers []*url.URL `json:"brokers"` -} - -// NewClientConfig returns a new instance of ClientConfig. -func NewClientConfig(u []*url.URL) *ClientConfig { - return &ClientConfig{ - Brokers: u, - } -} - // Client represents a client for the broker's HTTP API. type Client struct { - mu sync.Mutex - conns []*Conn - config ClientConfig // The Client state that must be persisted to disk. + mu sync.Mutex + conns []*Conn + url url.URL // current known leader URL + urls []url.URL // list of available broker URLs opened bool done chan chan struct{} // disconnection notification @@ -59,21 +47,45 @@ func NewClient() *Client { return c } -// URLs returns a list of broker URLs to connect to. -func (c *Client) URLs() []*url.URL { +// URL returns the current broker leader's URL. +func (c *Client) URL() url.URL { c.mu.Lock() defer c.mu.Unlock() - return c.config.Brokers + return c.url } -// LeaderURL returns the URL of the broker leader. -func (c *Client) LeaderURL() *url.URL { +// SetURL sets the current URL to connect to for the client and its connections. +func (c *Client) SetURL(u url.URL) { c.mu.Lock() defer c.mu.Unlock() + c.setURL(u) +} + +func (c *Client) setURL(u url.URL) { + // Set the client URL. + c.url = u - // TODO(benbjohnson): Actually keep track of the leader. - // HACK(benbjohnson): For testing, just grab a url. - return c.config.Brokers[0] + // Update all connections. + for _, conn := range c.conns { + conn.SetURL(u) + } +} + +// RandomizeURL sets a random URL from the configuration. +func (c *Client) RandomizeURL() { + c.mu.Lock() + defer c.mu.Unlock() + c.randomizeURL() +} + +func (c *Client) randomizeURL() { + // Clear URL if no brokers exist. + if len(c.urls) == 0 { + return + } + + // Otherwise randomly select a URL. + c.setURL(c.urls[rand.Intn(len(c.urls))]) } // SetLogOutput sets writer for all Client log output. @@ -81,13 +93,8 @@ func (c *Client) SetLogOutput(w io.Writer) { c.Logger = log.New(w, "[messaging] ", log.LstdFlags) } -// Open initializes and opens the connection to the cluster. The -// URLs used to contact the cluster are either those supplied to -// the function, or if none are supplied, are read from the file -// at "path". These URLs do need to be URLs of actual Brokers. -// Regardless of URL source, at least 1 URL must be available -// for the client to be successfully opened. -func (c *Client) Open(path string, urls []*url.URL) error { +// Open reads the configuration from the specified path or uses the URLs provided. +func (c *Client) Open(path string, urls []url.URL) error { c.mu.Lock() defer c.mu.Unlock() @@ -98,38 +105,27 @@ func (c *Client) Open(path string, urls []*url.URL) error { // Read URLs from file if no URLs are provided. if len(urls) == 0 { - // Read URLs from config file. There is no guarantee - // that the Brokers URLs in the config file are still - // the Brokers, so we're going to double-check. - b, err := ioutil.ReadFile(path) - if os.IsNotExist(err) { + if b, err := ioutil.ReadFile(path); os.IsNotExist(err) { // nop } else if err != nil { return err } else { - if err := json.Unmarshal(b, &c.config); err != nil { + var config ClientConfig + if err := json.Unmarshal(b, &config); err != nil { return err } - urls = c.config.Brokers + c.urls = config.Brokers } } + // Ensure we have at least one URL. if len(urls) < 1 { return ErrBrokerURLRequired } - // Now that we have the seed URLs, actually use these to - // get the actual Broker URLs. Do that here. - c.config.Brokers = urls // Let's pretend they are the same - - // Create a channel for streaming messages. - c.c = make(chan *Message, 0) - - // Open the streamer if there's an ID set. - if c.replicaID != 0 { - c.done = make(chan chan struct{}) - go c.streamer(c.done) - } + // Set the URLs whether they're from the config or passed in. + c.urls = urls + c.randomizeURL() // Set open flag. c.opened = true @@ -147,18 +143,11 @@ func (c *Client) Close() error { return ErrClientClosed } - // Shutdown streamer. - if c.done != nil { - ch := make(chan struct{}) - c.done <- ch - <-ch - c.done = nil + // Close all connections. + for _, conn := range c.conns { + _ = conn.Close() } - - // Close message stream & clear index. - close(c.c) - c.c = nil - c.index = 0 + c.conns = nil // Unset open flag. c.opened = false @@ -168,59 +157,122 @@ func (c *Client) Close() error { // Publish sends a message to the broker and returns an index or error. func (c *Client) Publish(m *Message) (uint64, error) { - var resp *http.Response - var err error + // Post message to broker. + values := url.Values{ + "type": {strconv.FormatUint(uint64(m.Type), 10)}, + "topicID": {strconv.FormatUint(m.TopicID, 10)}, + } + resp, err := c.do("POST", "/messaging/messages", values, "application/octet-stream", bytes.NewReader(m.Data)) + if err != nil { + return 0, fmt.Errorf("do: %s", err) + } + defer func() { _ = resp.Body.Close() }() + + // Parse broker index. + index, err := strconv.ParseUint(resp.Header.Get("X-Broker-Index"), 10, 64) + if err != nil { + return 0, fmt.Errorf("invalid index: %s", err) + } + + return index, nil +} - u := *c.LeaderURL() +// do sends an HTTP request to the given path with the current leader URL. +// This will automatically retry the request if it is redirected. +func (c *Client) do(method, path string, values url.Values, contentType string, body io.Reader) (*http.Response, error) { for { - // Send the message to the messages endpoint. - u.Path = "/messaging/messages" - u.RawQuery = url.Values{ - "type": {strconv.FormatUint(uint64(m.Type), 10)}, - "topicID": {strconv.FormatUint(m.TopicID, 10)}, - }.Encode() - resp, err = http.Post(u.String(), "application/octet-stream", bytes.NewReader(m.Data)) + // Generate URL. + u := c.URL() + u.Path = path + u.RawQuery = values.Encode() + + // Create request. + req, err := http.NewRequest(method, u.String(), body) if err != nil { - return 0, err + return nil, fmt.Errorf("new request: %s", err) + } + + // Send HTTP request. + // If it cannot connect then select a different URL from the config. + resp, err := http.DefaultClient.Do(req) + if err != nil { + c.randomizeURL() + return nil, err } - defer resp.Body.Close() // If a temporary redirect occurs then update the leader and retry. // If a non-200 status is returned then an error occurred. if resp.StatusCode == http.StatusTemporaryRedirect { redirectURL, err := url.Parse(resp.Header.Get("Location")) if err != nil { - return 0, fmt.Errorf("bad redirect: %s", resp.Header.Get("Location")) + resp.Body.Close() + return nil, fmt.Errorf("invalid redirect location: %s", resp.Header.Get("Location")) } - u = *redirectURL + c.SetURL(*redirectURL) continue } else if resp.StatusCode != http.StatusOK { + resp.Body.Close() if errstr := resp.Header.Get("X-Broker-Error"); errstr != "" { - return 0, errors.New(errstr) + return nil, errors.New(errstr) } - return 0, fmt.Errorf("cannot publish(%d)", resp.StatusCode) - } else { - break + return nil, fmt.Errorf("cannot publish(%d)", resp.StatusCode) } + + return resp, nil } - // Parse broker index. - index, err := strconv.ParseUint(resp.Header.Get("X-Broker-Index"), 10, 64) - if err != nil { - return 0, fmt.Errorf("invalid index: %s", err) +} + +// Conn returns an open connection to the broker for a given topic. +func (c *Client) Conn(topicID, index uint64) (*Conn, error) { + c.mu.Lock() + defer c.mu.Unlock() + + // Create connection and set current URL. + conn := NewConn(topicID, index) + conn.SetURL(c.url) + + // Open connection. + if err := conn.Open(); err != nil { + return nil, err } - return index, nil + // Add to list of client connections. + c.conns = append(c.conns, conn) + + return conn, nil +} + +// ClientConfig represents the configuration that must be persisted across restarts. +type ClientConfig struct { + Brokers []url.URL `json:"brokers"` +} + +// NewClientConfig returns a new instance of ClientConfig. +func NewClientConfig(u []url.URL) *ClientConfig { + return &ClientConfig{ + Brokers: u, + } } // Conn represents a stream over the client for a single topic. type Conn struct { + mu sync.Mutex topicID uint64 // topic identifier index uint64 // highest index sent over the channel url url.URL // current broker url - c chan *Message // channel streams messages from the broker. - reconnect chan struct{} // notification channel for broker change. + opened bool + c chan *Message // channel streams messages from the broker. + + wg sync.WaitGroup + closing chan struct{} + + // The amount of time to wait before reconnecting to a broker stream. + ReconnectTimeout time.Duration + + // The logging interface used by the connection for out-of-band errors. + Logger *log.Logger } // NewConn returns a new connection to the broker for a topic. @@ -228,6 +280,9 @@ func NewConn(topicID uint64, index uint64) *Conn { return &Conn{ topicID: topicID, index: index, + + ReconnectTimeout: DefaultReconnectTimeout, + Logger: log.New(os.Stderr, "", log.LstdFlags), } } @@ -237,13 +292,20 @@ func (c *Conn) TopicID() uint64 { return c.topicID } // C returns streaming channel for the connection. func (c *Conn) C() <-chan *Message { return c.c } -// Index returns the highest index sent over the channel. +// Index returns the highest index replicated to the caller. func (c *Conn) Index() uint64 { c.mu.Lock() defer c.mu.Unlock() return c.index } +// SetIndex sets the highest index replicated to the caller. +func (c *Conn) SetIndex(index uint64) { + c.mu.Lock() + defer c.mu.Unlock() + c.index = index +} + // URL returns the current URL of the connection. func (c *Conn) URL() url.URL { c.mu.Lock() @@ -255,18 +317,63 @@ func (c *Conn) URL() url.URL { func (c *Conn) SetURL(u url.URL) { c.mu.Lock() defer c.mu.Unlock() - c.url = u - - // Notify streamer of change. - select { - case c.reconnect <- struct{}{}: - default: - } } // Open opens a streaming connection to the broker. func (c *Conn) Open() error { + c.mu.Lock() + defer c.mu.Unlock() + + // Exit if aleady open or previously closed. + if c.opened { + return ErrConnOpen + } else if c.c != nil { + return ErrConnCannotReuse + } + c.opened = true + + // Create streaming channel. + c.c = make(chan *Message, 0) + + // Start goroutines. + c.wg.Add(1) + c.closing = make(chan struct{}) + go c.streamer(c.closing) + + return nil +} + +// Close closes a connection. +func (c *Conn) Close() error { + c.mu.Lock() + defer c.mu.Unlock() + return c.close() +} + +func (c *Conn) close() error { + if !c.opened { + return ErrConnClosed + } + + // Notify goroutines that the connection is closing. + if c.closing != nil { + close(c.closing) + c.closing = nil + } + + // Wait for goroutines to finish. + c.mu.Unlock() + c.wg.Wait() + c.mu.Lock() + + // Close channel. + close(c.c) + + // Mark connection as closed. + c.opened = false + + return nil } // Heartbeat sends a heartbeat back to the broker with the client's index. @@ -305,95 +412,89 @@ func (c *Conn) Heartbeat() error { } // streamer connects to a broker server and streams the replica's messages. -func (c *Conn) streamer(done chan chan struct{}) { - for { - // Check for the client disconnection. - select { - case ch := <-done: - close(ch) - return - default: - } +func (c *Conn) streamer(closing <-chan struct{}) { + defer c.wg.Done() - // TODO: Validate that there is at least one broker URL. + // Continually connect and retry streaming from server. + var req *http.Request + var reqlock sync.Mutex - // Connect to broker and stream. - u := c.URL() - u.Path = "/messaging/messages" - if err := c.streamFromURL(&u, done); err == errDone { - return - } else if err != nil { - c.Logger.Print(err) + c.wg.Add(1) + go func() { + defer c.wg.Done() + for { + // Check that the connection is not closing. + select { + case <-closing: + return + default: + } + + // Create URL. + u := c.URL() + u.Path = "/messaging/messages" + u.RawQuery = url.Values{ + "topicID": {strconv.FormatUint(c.topicID, 10)}, + "index": {strconv.FormatUint(c.Index(), 10)}, + }.Encode() + + // Create request. + reqlock.Lock() + req, _ = http.NewRequest("GET", u.String(), nil) + reqlock.Unlock() + + // Begin streaming request. + if err := c.stream(req, closing); err != nil { + c.Logger.Printf("reconnecting to broker: url=%s, err=%s", u, err) + time.Sleep(c.ReconnectTimeout) + } } + }() + + // Wait for the connection to close or the request to close. + <-closing + + // Close in-flight request. + reqlock.Lock() + if req != nil { + http.DefaultTransport.(*http.Transport).CancelRequest(req) } + reqlock.Unlock() } -// streamFromURL connects to a broker server and streams the replica's messages. -func (c *Client) streamFromURL(u *url.URL, done chan chan struct{}) error { - // Set the replica id on the URL and open the stream. - u.RawQuery = url.Values{"replicaID": {strconv.FormatUint(c.replicaID, 10)}}.Encode() - resp, err := http.Get(u.String()) +// stream connects to a broker server and streams the topic messages. +func (c *Conn) stream(req *http.Request, closing <-chan struct{}) error { + resp, err := http.DefaultClient.Do(req) if err != nil { - time.Sleep(c.ReconnectTimeout) - return nil + return err } defer func() { _ = resp.Body.Close() }() // Ensure that we received a 200 OK from the server before streaming. if resp.StatusCode != http.StatusOK { - time.Sleep(c.ReconnectTimeout) - c.Logger.Printf("reconnecting to broker: %s (status=%d)", u, resp.StatusCode) - return nil + return fmt.Errorf("invalid stream status code: %d", resp.StatusCode) } - c.Logger.Printf("connected to broker: %s", u) + c.Logger.Printf("connected to broker: %s", req.URL.String()) // Continuously decode messages from request body in a separate goroutine. - errNotify := make(chan error, 0) - go func() { - dec := NewMessageDecoder(resp.Body) - for { - // Decode message from the stream. - m := &Message{} - if err := dec.Decode(m); err != nil { - errNotify <- err - return - } - - // TODO: Write broker set updates, do not passthrough to channel. - - // Write message to streaming channel. - c.c <- m - - // Update the index on the client. - c.mu.Lock() - if m.Index > c.index { - c.index = m.Index - } - c.mu.Unlock() + dec := NewMessageDecoder(resp.Body) + for { + // Decode message from the stream. + m := &Message{} + if err := dec.Decode(m); err == io.EOF { + return nil + } else if err != nil { + return fmt.Errorf("decode: %s", err) } - }() - // Check for the client disconnect or error from the stream. - select { - case ch := <-done: - // Close body. - _ = resp.Body.Close() + // TODO: Write broker set updates, do not passthrough to channel. - // Clear message buffer. + // Write message to streaming channel. select { - case <-c.c: - default: + case <-closing: + return nil + case c.c <- m: } - - // Notify the close function and return marker error. - close(ch) - return errDone - - case err := <-errNotify: - return err } } - -// marker error for the streamer. -var errDone = errors.New("done") diff --git a/messaging/client_test.go b/messaging/client_test.go index 62f6c0cf010..161ac4c5223 100644 --- a/messaging/client_test.go +++ b/messaging/client_test.go @@ -2,12 +2,183 @@ package messaging_test import ( "io/ioutil" + "net/http" + "net/http/httptest" + "net/url" + "reflect" "testing" - "time" "github.com/influxdb/influxdb/messaging" ) +// Ensure a client can be opened and connections can be created. +func TestClient_Conn(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + switch req.URL.Query().Get("topicID") { + case "1": + (&messaging.Message{Index: 1, Data: []byte{100}}).WriteTo(w) + case "2": + (&messaging.Message{Index: 2, Data: []byte{200}}).WriteTo(w) + } + })) + defer s.Close() + + // Create and open connection to server. + c := messaging.NewClient() + if err := c.Open("", []url.URL{*MustParseURL(s.URL)}); err != nil { + t.Fatal(err) + } + + // Connect on topic #1. + conn1, err := c.Conn(1, 0) + if err != nil { + t.Fatal(err) + } else if m := <-conn1.C(); !reflect.DeepEqual(m, &messaging.Message{Index: 1, Data: []byte{100}}) { + t.Fatalf("unexpected message(1): %#v", m) + } + + // Connect on topic #2. + conn2, err := c.Conn(2, 0) + if err != nil { + t.Fatal(err) + } else if m := <-conn2.C(); !reflect.DeepEqual(m, &messaging.Message{Index: 2, Data: []byte{200}}) { + t.Fatalf("unexpected message(2): %#v", m) + } + + // Close client and all connections. + if err := c.Close(); err != nil { + t.Fatal(err) + } +} + +// Ensure that an error is returned when opening an opened connection. +func TestConn_Open_ErrConnOpen(t *testing.T) { + c := messaging.NewConn(1, 0) + c.Open() + defer c.Close() + if err := c.Open(); err != messaging.ErrConnOpen { + t.Fatalf("unexpected error: %s", err) + } +} + +// Ensure that an error is returned when opening a previously closed connection. +func TestConn_Open_ErrConnCannotReuse(t *testing.T) { + c := messaging.NewConn(1, 0) + c.Open() + c.Close() + if err := c.Open(); err != messaging.ErrConnCannotReuse { + t.Fatalf("unexpected error: %s", err) + } +} + +// Ensure that an error is returned when closing a closed connection. +func TestConn_Close_ErrConnClosed(t *testing.T) { + c := messaging.NewConn(1, 0) + c.Open() + c.Close() + if err := c.Close(); err != messaging.ErrConnClosed { + t.Fatalf("unexpected error: %s", err) + } +} + +// Ensure that a connection can connect and stream from a broker. +func TestConn_Open(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + // Verify incoming parameters. + if req.URL.Path != "/messaging/messages" { + t.Fatalf("unexpected path: %s", req.URL.Path) + } else if topicID := req.URL.Query().Get("topicID"); topicID != "100" { + t.Fatalf("unexpected topic id: %s", topicID) + } else if index := req.URL.Query().Get("index"); index != "200" { + t.Fatalf("unexpected index: %s", index) + } + + // Stream out messages. + (&messaging.Message{Index: 1, Data: []byte{100}}).WriteTo(w) + (&messaging.Message{Index: 2, Data: []byte{200}}).WriteTo(w) + })) + defer s.Close() + + // Create and open connection to server. + c := messaging.NewConn(100, 200) + c.SetURL(*MustParseURL(s.URL)) + if err := c.Open(); err != nil { + t.Fatal(err) + } + + // Receive messages from the stream. + if m := <-c.C(); !reflect.DeepEqual(m, &messaging.Message{Index: 1, Data: []byte{100}}) { + t.Fatalf("unexpected message(0): %#v", m) + } + if m := <-c.C(); !reflect.DeepEqual(m, &messaging.Message{Index: 2, Data: []byte{200}}) { + t.Fatalf("unexpected message(1): %#v", m) + } + + // Close connection. + if err := c.Close(); err != nil { + t.Fatal(err) + } +} + +// Ensure that a connection can reconnect. +func TestConn_Open_Reconnect(t *testing.T) { + var requestN int + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + // Error the first time. + if requestN == 0 { + requestN++ + http.Error(w, "OH NO", http.StatusInternalServerError) + return + } + + // Write a message the second time. + (&messaging.Message{Index: 1, Data: []byte{100}}).WriteTo(w) + })) + defer s.Close() + + // Create and open connection to server. + c := messaging.NewConn(100, 0) + c.SetURL(*MustParseURL(s.URL)) + if err := c.Open(); err != nil { + t.Fatal(err) + } + + // Receive messages from the stream. + if m := <-c.C(); !reflect.DeepEqual(m, &messaging.Message{Index: 1, Data: []byte{100}}) { + t.Fatalf("unexpected message(0): %#v", m) + } + + // Close connection. + if err := c.Close(); err != nil { + t.Fatal(err) + } +} + +// Ensure that a connection can heartbeat to the broker. +func TestConn_Heartbeat(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + // Verify incoming parameters. + if req.Method != "POST" { + t.Fatalf("unexpected method: %s", req.Method) + } else if req.URL.Path != "/messaging/heartbeat" { + t.Fatalf("unexpected path: %s", req.URL.Path) + } else if topicID := req.URL.Query().Get("topicID"); topicID != "100" { + t.Fatalf("unexpected topic id: %s", topicID) + } else if index := req.URL.Query().Get("index"); index != "200" { + t.Fatalf("unexpected index: %s", index) + } + })) + defer s.Close() + + // Create connection and heartbeat. + c := messaging.NewConn(100, 0) + c.SetURL(*MustParseURL(s.URL)) + c.SetIndex(200) + if err := c.Heartbeat(); err != nil { + t.Fatal(err) + } +} + /* // Ensure that a client can open a connect to the broker. func TestClient_Open(t *testing.T) { @@ -126,9 +297,6 @@ func TestClient_Publish_ErrLogClosed(t *testing.T) { } */ -// Ensure that a client can heartbeat its status to the broker. -func TestClient_Heartbeat(t *testing.T) { t.Skip("pending") } - // Client represents a test wrapper for the broker client. type Client struct { *messaging.Client @@ -137,7 +305,7 @@ type Client struct { // NewClient returns a new instance of Client. func NewClient(replicaID uint64) *Client { return &Client{ - Client: messaging.NewClient(replicaID), + Client: messaging.NewClient(), } } @@ -155,16 +323,6 @@ func (c *Client) MustPublish(m *messaging.Message) uint64 { return index } -// Sync waits until the client's index reaches the given index. -func (c *Client) Sync(index uint64) { - for { - if c.Index() == index { - return - } - time.Sleep(1 * time.Millisecond) - } -} - // NewTempFile returns the path of a new temporary file. // It is up to the caller to remove it when finished. func NewTempFile() string { diff --git a/messaging/errors.go b/messaging/errors.go index 13394a9f480..34e09848354 100644 --- a/messaging/errors.go +++ b/messaging/errors.go @@ -37,6 +37,15 @@ var ( // ErrClientClosed is returned when closing an already closed client. ErrClientClosed = errors.New("client closed") + // ErrConnOpen is returned when opening an already open connection. + ErrConnOpen = errors.New("connection already open") + + // ErrConnClosed is returned when closing an already closed connection. + ErrConnClosed = errors.New("connection closed") + + // ErrConnCannotReuse is returned when opening a previously closed connection. + ErrConnCannotReuse = errors.New("cannot reuse connection") + // ErrBrokerURLRequired is returned when opening a broker without URLs. ErrBrokerURLRequired = errors.New("broker url required") From 5f5c6ca297b3338d23c755fc3e986df1aced8132 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 9 Mar 2015 15:47:41 -0600 Subject: [PATCH 07/20] Integrate stateless messaging into influxdb package. --- broker.go | 36 ++-- broker_test.go | 2 + messaging/client.go | 24 +-- messaging/client_test.go | 34 ++-- server.go | 102 +++------- server_test.go | 414 +++++++++++++++++++++++++-------------- shard.go | 90 +++++++-- 7 files changed, 410 insertions(+), 292 deletions(-) diff --git a/broker.go b/broker.go index c1df5a5927b..1060687e611 100644 --- a/broker.go +++ b/broker.go @@ -1,5 +1,6 @@ package influxdb +/* import ( "fmt" "log" @@ -9,6 +10,20 @@ import ( "github.com/influxdb/influxdb/messaging" ) +const ( + // DefaultContinuousQueryCheckTime is how frequently the broker will ask a data node + // in the cluster to run any continuous queries that should be run. + DefaultContinuousQueryCheckTime = 1 * time.Second + + // DefaultDataNodeTimeout is how long the broker will wait before timing out on a data node + // that it has requested process continuous queries. + DefaultDataNodeTimeout = 1 * time.Second + + // DefaultFailureSleep is how long the broker will sleep before trying the next data node in + // the cluster if the current data node failed to respond + DefaultFailureSleep = 100 * time.Millisecond +) + // Broker represents an InfluxDB specific messaging broker. type Broker struct { *messaging.Broker @@ -24,29 +39,14 @@ type Broker struct { TriggerFailurePause time.Duration } -const ( - // DefaultContinuousQueryCheckTime is how frequently the broker will ask a data node - // in the cluster to run any continuous queries that should be run. - DefaultContinuousQueryCheckTime = 1 * time.Second - - // DefaultDataNodeTimeout is how long the broker will wait before timing out on a data node - // that it has requested process continuous queries. - DefaultDataNodeTimeout = 1 * time.Second - - // DefaultFailureSleep is how long the broker will sleep before trying the next data node in - // the cluster if the current data node failed to respond - DefaultFailureSleep = 100 * time.Millisecond -) - // NewBroker returns a new instance of a Broker with default values. func NewBroker() *Broker { - b := &Broker{ + return &Broker{ + Broker: messaging.NewBroker(), TriggerInterval: 5 * time.Second, TriggerTimeout: 20 * time.Second, TriggerFailurePause: 1 * time.Second, } - b.Broker = messaging.NewBroker() - return b } // RunContinuousQueryLoop starts running continuous queries on a background goroutine. @@ -128,3 +128,5 @@ func (b *Broker) requestContinuousQueryProcessing() error { return nil } + +*/ diff --git a/broker_test.go b/broker_test.go index bac0a5f6376..78c45b033fc 100644 --- a/broker_test.go +++ b/broker_test.go @@ -1,5 +1,6 @@ package influxdb_test +/* import ( "net/http" "net/http/httptest" @@ -93,3 +94,4 @@ func (h *BrokerTestHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { http.NotFound(w, r) } } +*/ diff --git a/messaging/client.go b/messaging/client.go index 1da2a90f693..6f90f5c7357 100644 --- a/messaging/client.go +++ b/messaging/client.go @@ -223,24 +223,19 @@ func (c *Client) do(method, path string, values url.Values, contentType string, } -// Conn returns an open connection to the broker for a given topic. -func (c *Client) Conn(topicID, index uint64) (*Conn, error) { +// Conn returns a connection to the broker for a given topic. +func (c *Client) Conn(topicID uint64) *Conn { c.mu.Lock() defer c.mu.Unlock() // Create connection and set current URL. - conn := NewConn(topicID, index) + conn := NewConn(topicID) conn.SetURL(c.url) - // Open connection. - if err := conn.Open(); err != nil { - return nil, err - } - // Add to list of client connections. c.conns = append(c.conns, conn) - return conn, nil + return conn } // ClientConfig represents the configuration that must be persisted across restarts. @@ -276,11 +271,9 @@ type Conn struct { } // NewConn returns a new connection to the broker for a topic. -func NewConn(topicID uint64, index uint64) *Conn { +func NewConn(topicID uint64) *Conn { return &Conn{ - topicID: topicID, - index: index, - + topicID: topicID, ReconnectTimeout: DefaultReconnectTimeout, Logger: log.New(os.Stderr, "", log.LstdFlags), } @@ -321,7 +314,7 @@ func (c *Conn) SetURL(u url.URL) { } // Open opens a streaming connection to the broker. -func (c *Conn) Open() error { +func (c *Conn) Open(index uint64) error { c.mu.Lock() defer c.mu.Unlock() @@ -333,6 +326,9 @@ func (c *Conn) Open() error { } c.opened = true + // Set starting index. + c.index = index + // Create streaming channel. c.c = make(chan *Message, 0) diff --git a/messaging/client_test.go b/messaging/client_test.go index 161ac4c5223..15c6039b726 100644 --- a/messaging/client_test.go +++ b/messaging/client_test.go @@ -30,16 +30,16 @@ func TestClient_Conn(t *testing.T) { } // Connect on topic #1. - conn1, err := c.Conn(1, 0) - if err != nil { + conn1 := c.Conn(1) + if err := conn1.Open(0); err != nil { t.Fatal(err) } else if m := <-conn1.C(); !reflect.DeepEqual(m, &messaging.Message{Index: 1, Data: []byte{100}}) { t.Fatalf("unexpected message(1): %#v", m) } // Connect on topic #2. - conn2, err := c.Conn(2, 0) - if err != nil { + conn2 := c.Conn(2) + if err := conn2.Open(0); err != nil { t.Fatal(err) } else if m := <-conn2.C(); !reflect.DeepEqual(m, &messaging.Message{Index: 2, Data: []byte{200}}) { t.Fatalf("unexpected message(2): %#v", m) @@ -53,28 +53,28 @@ func TestClient_Conn(t *testing.T) { // Ensure that an error is returned when opening an opened connection. func TestConn_Open_ErrConnOpen(t *testing.T) { - c := messaging.NewConn(1, 0) - c.Open() + c := messaging.NewConn(1) + c.Open(0) defer c.Close() - if err := c.Open(); err != messaging.ErrConnOpen { + if err := c.Open(0); err != messaging.ErrConnOpen { t.Fatalf("unexpected error: %s", err) } } // Ensure that an error is returned when opening a previously closed connection. func TestConn_Open_ErrConnCannotReuse(t *testing.T) { - c := messaging.NewConn(1, 0) - c.Open() + c := messaging.NewConn(1) + c.Open(0) c.Close() - if err := c.Open(); err != messaging.ErrConnCannotReuse { + if err := c.Open(0); err != messaging.ErrConnCannotReuse { t.Fatalf("unexpected error: %s", err) } } // Ensure that an error is returned when closing a closed connection. func TestConn_Close_ErrConnClosed(t *testing.T) { - c := messaging.NewConn(1, 0) - c.Open() + c := messaging.NewConn(1) + c.Open(0) c.Close() if err := c.Close(); err != messaging.ErrConnClosed { t.Fatalf("unexpected error: %s", err) @@ -100,9 +100,9 @@ func TestConn_Open(t *testing.T) { defer s.Close() // Create and open connection to server. - c := messaging.NewConn(100, 200) + c := messaging.NewConn(100) c.SetURL(*MustParseURL(s.URL)) - if err := c.Open(); err != nil { + if err := c.Open(200); err != nil { t.Fatal(err) } @@ -137,9 +137,9 @@ func TestConn_Open_Reconnect(t *testing.T) { defer s.Close() // Create and open connection to server. - c := messaging.NewConn(100, 0) + c := messaging.NewConn(100) c.SetURL(*MustParseURL(s.URL)) - if err := c.Open(); err != nil { + if err := c.Open(0); err != nil { t.Fatal(err) } @@ -171,7 +171,7 @@ func TestConn_Heartbeat(t *testing.T) { defer s.Close() // Create connection and heartbeat. - c := messaging.NewConn(100, 0) + c := messaging.NewConn(100) c.SetURL(*MustParseURL(s.URL)) c.SetIndex(200) if err := c.Heartbeat(); err != nil { diff --git a/server.go b/server.go index 72af1c9f6c1..7d617e13260 100644 --- a/server.go +++ b/server.go @@ -254,12 +254,16 @@ func (s *Server) load() error { } } - // Open all shards. + // Open all shards owned by server. for _, db := range s.databases { for _, rp := range db.policies { for _, g := range rp.shardGroups { for _, sh := range g.Shards { - if err := sh.open(s.shardPath(sh.ID)); err != nil { + if !sh.HasDataNodeID(s.id) { + continue + } + + if err := sh.open(s.shardPath(sh.ID), s.client.Conn(sh.ID)); err != nil { return fmt.Errorf("cannot open shard store: id=%d, err=%s", sh.ID, err) } } @@ -347,11 +351,18 @@ func (s *Server) setClient(client MessagingClient) error { // Set the messaging client. s.client = client - // Start goroutine to read messages from the broker. + // Start goroutine to read messages from the broadcast channel. if client != nil { + // Create connection for broadcast channel. + conn := client.Conn(BroadcastTopicID) + if err := conn.Open(s.index); err != nil { + return fmt.Errorf("open conn: %s", err) + } + + // Stream messages done := make(chan struct{}, 0) s.done = done - go s.processor(client, done) + go s.processor(conn, done) } return nil @@ -370,7 +381,7 @@ func (s *Server) broadcast(typ messaging.MessageType, c interface{}) (uint64, er // Publish the message. m := &messaging.Message{ Type: typ, - TopicID: messaging.BroadcastTopicID, + TopicID: BroadcastTopicID, Data: data, } index, err := s.client.Publish(m) @@ -838,7 +849,7 @@ func (s *Server) applyCreateShardGroupIfNotExists(m *messaging.Message) (err err } // Open shard store. Panic if an error occurs and we can retry. - if err := sh.open(s.shardPath(sh.ID)); err != nil { + if err := sh.open(s.shardPath(sh.ID), s.client.Conn(sh.ID)); err != nil { panic("unable to open shard: " + err.Error()) } } @@ -848,21 +859,6 @@ func (s *Server) applyCreateShardGroupIfNotExists(m *messaging.Message) (err err s.shards[sh.ID] = sh } - // Subscribe to shard if it matches the server's index. - // TODO: Move subscription outside of command processing. - // TODO: Retry subscriptions on failure. - for _, sh := range g.Shards { - // Ignore if this server is not assigned. - if !sh.HasDataNodeID(s.id) { - continue - } - - // Subscribe on the broker. - if err := s.client.Subscribe(s.id, sh.ID); err != nil { - log.Printf("unable to subscribe: replica=%d, topic=%d, err=%s", s.id, sh.ID, err) - } - } - return } @@ -1499,29 +1495,6 @@ func (s *Server) WriteSeries(database, retentionPolicy string, points []Point) ( return maxIndex, err } -// applyWriteRawSeries writes raw series data to the database. -// Raw series data has already converted field names to ids so the -// representation is fast and compact. -func (s *Server) applyWriteRawSeries(m *messaging.Message) error { - // Retrieve the shard. - sh := s.Shard(m.TopicID) - if sh == nil { - return ErrShardNotFound - } - if s.WriteTrace { - log.Printf("received write message for application, shard %d", sh.ID) - } - - if err := sh.writeSeries(m.Data); err != nil { - return err - } - if s.WriteTrace { - log.Printf("write message successfully applied to shard %d", sh.ID) - } - - return nil -} - // createMeasurementsIfNotExists walks the "points" and ensures that all new Series are created, and all // new Measurement fields have been created, across the cluster. func (s *Server) createMeasurementsIfNotExists(database, retentionPolicy string, points []Point) error { @@ -2667,7 +2640,7 @@ func (s *Server) normalizeMeasurement(name string, defaultDatabase string) (stri } // processor runs in a separate goroutine and processes all incoming broker messages. -func (s *Server) processor(client MessagingClient, done chan struct{}) { +func (s *Server) processor(conn MessagingConn, done chan struct{}) { for { // Read incoming message. var m *messaging.Message @@ -2675,27 +2648,12 @@ func (s *Server) processor(client MessagingClient, done chan struct{}) { select { case <-done: return - case m, ok = <-client.C(): + case m, ok = <-conn.C(): if !ok { return } } - // Handle write series separately so we don't lock server during shard writes. - if m.Type == writeRawSeriesMessageType { - // Write series to shard without lock. - err := s.applyWriteRawSeries(m) - - // Set index & error under lock. - s.mu.Lock() - s.index = m.Index - if err != nil { - s.errors[m.Index] = err - } - s.mu.Unlock() - continue - } - // All other messages must be processed under lock. func() { s.mu.Lock() @@ -2747,6 +2705,8 @@ func (s *Server) processor(client MessagingClient, done chan struct{}) { err = s.applyCreateContinuousQueryCommand(m) case dropSeriesMessageType: err = s.applyDropSeries(m) + case writeRawSeriesMessageType: + panic("write series not allowed in broadcast topic") } // Sync high water mark and errors. @@ -2854,24 +2814,18 @@ func (r *Results) Error() error { return nil } -// MessagingClient represents the client used to receive messages from brokers. +// MessagingClient represents the client used to connect to brokers. type MessagingClient interface { // Publishes a message to the broker. Publish(m *messaging.Message) (index uint64, err error) - // Creates a new replica with a given ID on the broker. - CreateReplica(replicaID uint64, connectURL *url.URL) error - - // Deletes an existing replica with a given ID from the broker. - DeleteReplica(replicaID uint64) error - - // Creates a subscription for a replica to a topic. - Subscribe(replicaID, topicID uint64) error - - // Removes a subscription from the replica for a topic. - Unsubscribe(replicaID, topicID uint64) error + // Conn returns an open, streaming connection to a topic. + Conn(topicID uint64) MessagingConn +} - // The streaming channel for all subscribed messages. +// MessagingConn represents a streaming connection to a single broker topic. +type MessagingConn interface { + Open(index uint64) error C() <-chan *messaging.Message } diff --git a/server_test.go b/server_test.go index d11d38fb713..8ee3d258a13 100644 --- a/server_test.go +++ b/server_test.go @@ -10,6 +10,7 @@ import ( "os" "reflect" "strings" + "sync" "testing" "time" @@ -39,7 +40,9 @@ func TestServer_Open_ErrPathRequired(t *testing.T) { t.Skip("pending") } // Ensure the server can create a new data node. func TestServer_CreateDataNode(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create a new node. @@ -61,7 +64,9 @@ func TestServer_CreateDataNode(t *testing.T) { // Ensure the server returns an error when creating a duplicate node. func TestServer_CreateDatabase_ErrDataNodeExists(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create a node with the same URL twice. @@ -76,7 +81,9 @@ func TestServer_CreateDatabase_ErrDataNodeExists(t *testing.T) { // Ensure the server can delete a node. func TestServer_DeleteDataNode(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create a data node and verify it exists. @@ -99,7 +106,9 @@ func TestServer_DeleteDataNode(t *testing.T) { // Test unuathorized requests logging func TestServer_UnauthorizedRequests(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.SetAuthenticationEnabled(true) @@ -140,7 +149,9 @@ func TestServer_UnauthorizedRequests(t *testing.T) { // Test user privilege authorization. func TestServer_UserPrivilegeAuthorization(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create cluster admin. @@ -175,7 +186,9 @@ func TestServer_UserPrivilegeAuthorization(t *testing.T) { // Test single statement query authorization. func TestServer_SingleStatementQueryAuthorization(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create cluster admin. @@ -242,7 +255,9 @@ func TestServer_SingleStatementQueryAuthorization(t *testing.T) { // Test multiple statement query authorization. func TestServer_MultiStatementQueryAuthorization(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create cluster admin. @@ -287,7 +302,9 @@ func TestServer_MultiStatementQueryAuthorization(t *testing.T) { // Ensure the server can create a database. func TestServer_CreateDatabase(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create the "foo" database. @@ -304,7 +321,9 @@ func TestServer_CreateDatabase(t *testing.T) { // Ensure the server returns an error when creating a duplicate database. func TestServer_CreateDatabase_ErrDatabaseExists(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create the "foo" database twice. @@ -318,7 +337,9 @@ func TestServer_CreateDatabase_ErrDatabaseExists(t *testing.T) { // Ensure the server can drop a database. func TestServer_DropDatabase(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create the "foo" database and verify it exists. @@ -339,7 +360,9 @@ func TestServer_DropDatabase(t *testing.T) { // Ensure the server returns an error when dropping a database that doesn't exist. func TestServer_DropDatabase_ErrDatabaseNotFound(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Drop a database that doesn't exist. @@ -350,7 +373,9 @@ func TestServer_DropDatabase_ErrDatabaseNotFound(t *testing.T) { // Ensure the server can return a list of all databases. func TestServer_Databases(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create some databases. @@ -370,7 +395,9 @@ func TestServer_Databases(t *testing.T) { // Ensure the server can create a new user. func TestServer_CreateUser(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create a user. @@ -406,7 +433,9 @@ func TestServer_CreateUser(t *testing.T) { // Ensure the server correctly detects when there is an admin user. func TestServer_AdminUserExists(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // A server should start up without any admin user. @@ -435,7 +464,9 @@ func TestServer_AdminUserExists(t *testing.T) { // Ensure the server returns an error when creating an user without a name. func TestServer_CreateUser_ErrUsernameRequired(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() if err := s.CreateUser("", "pass", false); err != influxdb.ErrUsernameRequired { t.Fatal(err) @@ -444,7 +475,9 @@ func TestServer_CreateUser_ErrUsernameRequired(t *testing.T) { // Ensure the server returns an error when creating a duplicate user. func TestServer_CreateUser_ErrUserExists(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() if err := s.CreateUser("susy", "pass", false); err != nil { t.Fatal(err) @@ -456,7 +489,9 @@ func TestServer_CreateUser_ErrUserExists(t *testing.T) { // Ensure the server can delete an existing user. func TestServer_DeleteUser(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create a user. @@ -481,7 +516,9 @@ func TestServer_DeleteUser(t *testing.T) { // Ensure the server can return a list of all users. func TestServer_Users(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create some users. @@ -501,7 +538,9 @@ func TestServer_Users(t *testing.T) { // Ensure the server does not return non-existent users func TestServer_NonExistingUsers(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create some users. @@ -522,7 +561,9 @@ func TestServer_NonExistingUsers(t *testing.T) { // Ensure the database can create a new retention policy. func TestServer_CreateRetentionPolicy(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create a database. @@ -553,7 +594,9 @@ func TestServer_CreateRetentionPolicy(t *testing.T) { // Ensure the server returns an error when creating a retention policy with an invalid db. func TestServer_CreateRetentionPolicy_ErrDatabaseNotFound(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() if err := s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: "bar"}); err != influxdb.ErrDatabaseNotFound { t.Fatal(err) @@ -562,7 +605,9 @@ func TestServer_CreateRetentionPolicy_ErrDatabaseNotFound(t *testing.T) { // Ensure the server returns an error when creating a retention policy without a name. func TestServer_CreateRetentionPolicy_ErrRetentionPolicyNameRequired(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") if err := s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: ""}); err != influxdb.ErrRetentionPolicyNameRequired { @@ -572,7 +617,9 @@ func TestServer_CreateRetentionPolicy_ErrRetentionPolicyNameRequired(t *testing. // Ensure the server returns an error when creating a duplicate retention policy. func TestServer_CreateRetentionPolicy_ErrRetentionPolicyExists(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: "bar"}) @@ -583,7 +630,9 @@ func TestServer_CreateRetentionPolicy_ErrRetentionPolicyExists(t *testing.T) { // Ensure the database can alter an existing retention policy. func TestServer_AlterRetentionPolicy(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create a database. @@ -647,7 +696,9 @@ func TestServer_AlterRetentionPolicy(t *testing.T) { // Ensure the server can delete an existing retention policy. func TestServer_DeleteRetentionPolicy(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create a database and retention policy. @@ -673,7 +724,9 @@ func TestServer_DeleteRetentionPolicy(t *testing.T) { // Ensure the server returns an error when deleting a retention policy on invalid db. func TestServer_DeleteRetentionPolicy_ErrDatabaseNotFound(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() if err := s.DeleteRetentionPolicy("foo", "bar"); err != influxdb.ErrDatabaseNotFound { t.Fatal(err) @@ -682,7 +735,9 @@ func TestServer_DeleteRetentionPolicy_ErrDatabaseNotFound(t *testing.T) { // Ensure the server returns an error when deleting a retention policy without a name. func TestServer_DeleteRetentionPolicy_ErrRetentionPolicyNameRequired(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") if err := s.DeleteRetentionPolicy("foo", ""); err != influxdb.ErrRetentionPolicyNameRequired { @@ -692,7 +747,9 @@ func TestServer_DeleteRetentionPolicy_ErrRetentionPolicyNameRequired(t *testing. // Ensure the server returns an error when deleting a non-existent retention policy. func TestServer_DeleteRetentionPolicy_ErrRetentionPolicyNotFound(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") if err := s.DeleteRetentionPolicy("foo", "no_such_policy"); err != influxdb.ErrRetentionPolicyNotFound { @@ -702,7 +759,9 @@ func TestServer_DeleteRetentionPolicy_ErrRetentionPolicyNotFound(t *testing.T) { // Ensure the server can set the default retention policy func TestServer_SetDefaultRetentionPolicy(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -735,7 +794,9 @@ func TestServer_SetDefaultRetentionPolicy(t *testing.T) { // Ensure the server returns an error when setting the default retention policy to a non-existant one. func TestServer_SetDefaultRetentionPolicy_ErrRetentionPolicyNotFound(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") if err := s.SetDefaultRetentionPolicy("foo", "no_such_policy"); err != influxdb.ErrRetentionPolicyNotFound { @@ -745,7 +806,9 @@ func TestServer_SetDefaultRetentionPolicy_ErrRetentionPolicyNotFound(t *testing. // Ensure the server prohibits a zero check interval for retention policy enforcement. func TestServer_StartRetentionPolicyEnforcement_ErrZeroInterval(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() if err := s.StartRetentionPolicyEnforcement(time.Duration(0)); err == nil { t.Fatal("failed to prohibit retention policies zero check interval") @@ -797,34 +860,22 @@ func TestServer_WriteSeries(t *testing.T) { s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: "mypolicy", Duration: 1 * time.Hour}) s.CreateUser("susy", "pass", false) - // Check if a topic is being subscribed to. - var subscribed bool - c.SubscribeFunc = func(replicaID, topicID uint64) error { - subscribed = true - return nil - } - // Write series with one point to the database. tags := map[string]string{"host": "servera.influx.com", "region": "uswest"} index, err := s.WriteSeries("foo", "mypolicy", []influxdb.Point{{Name: "cpu_load", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:00Z"), Fields: map[string]interface{}{"value": float64(23.2)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) + warn("A") // Write another point 10 seconds later so it goes through "raw series". index, err = s.WriteSeries("foo", "mypolicy", []influxdb.Point{{Name: "cpu_load", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:10Z"), Fields: map[string]interface{}{"value": float64(100)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) - } - - // Verify a subscription was made. - if !subscribed { - t.Fatal("expected subscription") } + c.Sync(index) + warn("B") // Retrieve first series data point. if v, err := s.ReadSeries("foo", "mypolicy", "cpu_load", tags, mustParseTime("2000-01-01T00:00:00Z")); err != nil { @@ -863,9 +914,8 @@ func TestServer_DropMeasurement(t *testing.T) { index, err := s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "cpu", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:00Z"), Fields: map[string]interface{}{"value": float64(23.2)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) // Ensure measurement exists results := s.ExecuteQuery(MustParseQuery(`SHOW MEASUREMENTS`), "foo", nil) @@ -933,9 +983,8 @@ func TestServer_DropMeasurementNoneExists(t *testing.T) { index, err := s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "cpu", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:00Z"), Fields: map[string]interface{}{"value": float64(23.2)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) // Drop measurement after writing data to ensure we still get the same error results = s.ExecuteQuery(MustParseQuery(`DROP MEASUREMENT bar`), "foo", nil) @@ -968,17 +1017,15 @@ func TestServer_DropMeasurementSeriesTagsPreserved(t *testing.T) { index, err := s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "cpu", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:00Z"), Fields: map[string]interface{}{"value": float64(23.2)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) tags = map[string]string{"host": "serverB", "region": "uswest"} index, err = s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "memory", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:01Z"), Fields: map[string]interface{}{"value": float64(33.2)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) // Ensure measurement exists results := s.ExecuteQuery(MustParseQuery(`SHOW MEASUREMENTS`), "foo", nil) @@ -1084,9 +1131,8 @@ func TestServer_DropSeries(t *testing.T) { index, err := s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "cpu", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:00Z"), Fields: map[string]interface{}{"value": float64(23.2)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) // Ensure series exists results := s.ExecuteQuery(MustParseQuery(`SHOW SERIES`), "foo", nil) @@ -1129,16 +1175,15 @@ func TestServer_DropSeriesFromMeasurement(t *testing.T) { index, err := s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "cpu", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:00Z"), Fields: map[string]interface{}{"value": float64(23.2)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) + tags = map[string]string{"host": "serverb", "region": "useast"} index, err = s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "memory", Tags: tags, Timestamp: mustParseTime("2000-01-02T00:00:00Z"), Fields: map[string]interface{}{"value": float64(23465432423)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) // Drop series results := s.ExecuteQuery(MustParseQuery(`DROP SERIES FROM memory`), "foo", nil) @@ -1169,11 +1214,11 @@ func TestServer_MergeManySeries(t *testing.T) { for i := 1; i < 11; i++ { for j := 1; j < 5+i%3; j++ { tags := map[string]string{"host": fmt.Sprintf("server_%d", i)} - if index, err := s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "cpu", Tags: tags, Timestamp: time.Unix(int64(j), int64(0)), Fields: map[string]interface{}{"value": float64(22)}}}); err != nil { + index, err := s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "cpu", Tags: tags, Timestamp: time.Unix(int64(j), int64(0)), Fields: map[string]interface{}{"value": float64(22)}}}) + if err != nil { t.Fatalf("unexpected error: %s", err.Error()) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) } } @@ -1207,17 +1252,15 @@ func TestServer_DropSeriesTagsPreserved(t *testing.T) { index, err := s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "cpu", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:00Z"), Fields: map[string]interface{}{"value": float64(23.2)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) tags = map[string]string{"host": "serverB", "region": "uswest"} index, err = s.WriteSeries("foo", "raw", []influxdb.Point{{Name: "cpu", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:01Z"), Fields: map[string]interface{}{"value": float64(33.2)}}}) if err != nil { t.Fatal(err) - } else if err = s.Sync(index); err != nil { - t.Fatalf("sync error: %s", err) } + c.Sync(index) results := s.ExecuteQuery(MustParseQuery(`SHOW SERIES`), "foo", nil) if res := results.Results[0]; res.Err != nil { @@ -1270,7 +1313,9 @@ func TestServer_DropSeriesTagsPreserved(t *testing.T) { // Ensure the server can execute a query and return the data correctly. func TestServer_ExecuteQuery(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: "raw", Duration: 1 * time.Hour}) @@ -1348,7 +1393,9 @@ func TestServer_ExecuteQuery(t *testing.T) { // Ensure the server respects limit and offset in show series queries func TestServer_ShowSeriesLimitOffset(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: "raw", Duration: 1 * time.Hour}) @@ -1402,7 +1449,9 @@ func TestServer_ShowSeriesLimitOffset(t *testing.T) { // Ensure that when querying for raw data values that they return in time order func TestServer_RawDataReturnsInOrder(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: "raw", Duration: 1 * time.Hour}) @@ -1447,7 +1496,9 @@ func TestServer_RawDataReturnsInOrder(t *testing.T) { // Ensure that limit and offset work func TestServer_LimitAndOffset(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: "raw", Duration: 1 * time.Hour}) @@ -1494,7 +1545,9 @@ func TestServer_LimitAndOffset(t *testing.T) { // Ensure the server can execute a wildcard query and return the data correctly. func TestServer_ExecuteWildcardQuery(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: "raw", Duration: 1 * time.Hour}) @@ -1518,7 +1571,9 @@ func TestServer_ExecuteWildcardQuery(t *testing.T) { // Ensure the server can execute a wildcard GROUP BY func TestServer_ExecuteWildcardGroupBy(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: "raw", Duration: 1 * time.Hour}) @@ -1549,7 +1604,9 @@ func TestServer_ExecuteWildcardGroupBy(t *testing.T) { } func TestServer_CreateShardGroupIfNotExist(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -1572,7 +1629,9 @@ func TestServer_CreateShardGroupIfNotExist(t *testing.T) { } func TestServer_DeleteShardGroup(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -1608,7 +1667,9 @@ func TestServer_DeleteShardGroup(t *testing.T) { /* TODO(benbjohnson): Change test to not expose underlying series ids directly. func TestServer_Measurements(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") s.CreateRetentionPolicy("foo", &influxdb.RetentionPolicy{Name: "mypolicy", Duration: 1 * time.Hour}) @@ -1677,7 +1738,9 @@ func TestServer_NormalizeMeasurement(t *testing.T) { } // Create server with a variety of databases, retention policies, and measurements - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Default database with one policy. @@ -1724,7 +1787,9 @@ func TestServer_NormalizeQuery(t *testing.T) { } // Start server with database & retention policy. - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.CreateDatabase("db0") s.CreateRetentionPolicy("db0", &influxdb.RetentionPolicy{Name: "rp0"}) @@ -1744,7 +1809,9 @@ func TestServer_NormalizeQuery(t *testing.T) { // Ensure the server can create a continuous query func TestServer_CreateContinuousQuery(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create the "foo" database. @@ -1804,7 +1871,9 @@ func TestServer_CreateContinuousQuery_ErrInfinteLoop(t *testing.T) { // Ensure func TestServer_RunContinuousQueries(t *testing.T) { t.Skip() - s := OpenServer(NewMessagingClient()) + c := NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create the "foo" database. @@ -1935,40 +2004,6 @@ func TestbatchWrite_UnmarshalEpoch(t *testing.T) { } -// Ensure the server will skip over replayed log entries and not blow up. -func TestServer_Replay(t *testing.T) { - c := NewMessagingClient() - s := OpenServer(c) - defer s.Close() - - // Record all messages through the client. - var messages []*messaging.Message - c.PublishFunc = func(m *messaging.Message) (uint64, error) { - messages = append(messages, m) - c.c <- m - return m.Index, nil - } - - // Create a new node. - u, _ := url.Parse("http://localhost:80000") - if err := s.CreateDataNode(u); err != nil { - t.Fatal(err) - } - s.Restart() - - // Replay messages through client. - for _, m := range messages { - c.c <- m - } - - // Sleep so it has a moment to process & ignore. - time.Sleep(100 * time.Millisecond) - - // NOTE: There is no way to introspect into the server to see that - // messages are being dropped. This test exists to make sure the server - // doesn't crash on retry. -} - // Server is a wrapping test struct for influxdb.Server. type Server struct { *influxdb.Server @@ -2045,72 +2080,147 @@ func (s *Server) MustWriteSeries(database, retentionPolicy string, points []infl index, err := s.WriteSeries(database, retentionPolicy, points) if err != nil { panic(err.Error()) - } else if err = s.Sync(index); err != nil { - panic("sync error: " + err.Error()) } + s.Client().(*MessagingClient).Sync(index) return index } // MessagingClient represents a test client for the messaging broker. type MessagingClient struct { - index uint64 - c chan *messaging.Message + mu sync.Mutex + index uint64 // highest index + conns []*MessagingConn // list of all connections + + messagesByTopicID map[uint64][]*messaging.Message // message by topic - PublishFunc func(*messaging.Message) (uint64, error) - CreateReplicaFunc func(replicaID uint64, connectURL *url.URL) error - DeleteReplicaFunc func(replicaID uint64) error - SubscribeFunc func(replicaID, topicID uint64) error - UnsubscribeFunc func(replicaID, topicID uint64) error + PublishFunc func(*messaging.Message) (uint64, error) + ConnFunc func(topicID uint64) influxdb.MessagingConn } // NewMessagingClient returns a new instance of MessagingClient. func NewMessagingClient() *MessagingClient { - c := &MessagingClient{c: make(chan *messaging.Message, 1)} - c.PublishFunc = c.send - c.CreateReplicaFunc = func(replicaID uint64, connectURL *url.URL) error { return nil } - c.DeleteReplicaFunc = func(replicaID uint64) error { return nil } - c.SubscribeFunc = func(replicaID, topicID uint64) error { return nil } - c.UnsubscribeFunc = func(replicaID, topicID uint64) error { return nil } + c := &MessagingClient{ + messagesByTopicID: make(map[uint64][]*messaging.Message), + } + c.PublishFunc = c.DefaultPublishFunc + c.ConnFunc = c.DefaultConnFunc return c } -// Publish attaches an autoincrementing index to the message. -// This function also execute's the client's PublishFunc mock function. -func (c *MessagingClient) Publish(m *messaging.Message) (uint64, error) { +// Close closes all open connections. +func (c *MessagingClient) Close() error { + c.mu.Lock() + defer c.mu.Unlock() + + for _, conn := range c.conns { + conn.Close() + } + + return nil +} + +func (c *MessagingClient) Publish(m *messaging.Message) (uint64, error) { return c.PublishFunc(m) } + +// DefaultPublishFunc sets an autoincrementing index on the message and sends it to each topic connection. +func (c *MessagingClient) DefaultPublishFunc(m *messaging.Message) (uint64, error) { + c.mu.Lock() + defer c.mu.Unlock() + + // Increment index and assign it to message. c.index++ m.Index = c.index - return c.PublishFunc(m) -} -// send sends the message through to the channel. -// This is the default value of PublishFunc. -func (c *MessagingClient) send(m *messaging.Message) (uint64, error) { - c.c <- m + // Append message to the topic. + c.messagesByTopicID[m.TopicID] = append(c.messagesByTopicID[m.TopicID], m) + + // Send to each connection for the topic. + for _, conn := range c.conns { + if conn.topicID == m.TopicID { + conn.Send(m) + } + } + return m.Index, nil } -// Creates a new replica with a given ID on the broker. -func (c *MessagingClient) CreateReplica(replicaID uint64, connectURL *url.URL) error { - return c.CreateReplicaFunc(replicaID, connectURL) +func (c *MessagingClient) Conn(topicID uint64) influxdb.MessagingConn { + return c.ConnFunc(topicID) +} + +// DefaultConnFunc returns a connection for a specific topic. +func (c *MessagingClient) DefaultConnFunc(topicID uint64) influxdb.MessagingConn { + c.mu.Lock() + defer c.mu.Unlock() + + // Create new connection. + conn := NewMessagingConn(topicID) + + // Track connections. + c.conns = append(c.conns, conn) + + return conn +} + +// Sync blocks until a given index has been sent through the client. +func (c *MessagingClient) Sync(index uint64) { + for { + c.mu.Lock() + if c.index >= index { + c.mu.Unlock() + time.Sleep(10 * time.Millisecond) + return + } + c.mu.Unlock() + + // Otherwise wait momentarily and check again. + time.Sleep(1 * time.Millisecond) + } +} + +// MessagingConn represents a mockable connection implementing influxdb.MessagingConn. +type MessagingConn struct { + mu sync.Mutex + topicID uint64 + index uint64 + c chan *messaging.Message } -// Deletes an existing replica with a given ID from the broker. -func (c *MessagingClient) DeleteReplica(replicaID uint64) error { - return c.DeleteReplicaFunc(replicaID) +// NewMessagingConn returns a new instance of MessagingConn. +func NewMessagingConn(topicID uint64) *MessagingConn { + return &MessagingConn{ + topicID: topicID, + } } -// Subscribe adds a subscription to a replica for a topic on the broker. -func (c *MessagingClient) Subscribe(replicaID, topicID uint64) error { - return c.SubscribeFunc(replicaID, topicID) +// Open starts the stream from a given index. +func (c *MessagingConn) Open(index uint64) error { + // TODO: Fill connection stream with existing messages. + c.c = make(chan *messaging.Message, 1024) + return nil } -// Unsubscribe removes a subscrition from a replica for a topic on the broker. -func (c *MessagingClient) Unsubscribe(replicaID, topicID uint64) error { - return c.UnsubscribeFunc(replicaID, topicID) +// Close closes the streaming channel. +func (c *MessagingConn) Close() error { + close(c.c) + return nil } // C returns a channel for streaming message. -func (c *MessagingClient) C() <-chan *messaging.Message { return c.c } +func (c *MessagingConn) C() <-chan *messaging.Message { return c.c } + +func (c *MessagingConn) Send(m *messaging.Message) { + // Ignore any old messages. + c.mu.Lock() + if m.Index <= c.index { + c.mu.Unlock() + return + } + c.index = m.Index + c.mu.Unlock() + + // Send message to channel. + c.c <- m +} // tempfile returns a temporary path. func tempfile() string { diff --git a/shard.go b/shard.go index 301bfe4050e..a8dcd459ce2 100644 --- a/shard.go +++ b/shard.go @@ -17,6 +17,9 @@ type ShardGroup struct { Shards []*Shard `json:"shards,omitempty"` } +// newShardGroup returns a new initialized ShardGroup instance. +func newShardGroup() *ShardGroup { return &ShardGroup{} } + // close closes all shards. func (g *ShardGroup) close() { for _, sh := range g.Shards { @@ -29,19 +32,6 @@ func (g *ShardGroup) ShardBySeriesID(seriesID uint32) *Shard { return g.Shards[int(seriesID)%len(g.Shards)] } -// Shard represents the logical storage for a given time range. -// The instance on a local server may contain the raw data in "store" if the -// shard is assigned to the server's data node id. -type Shard struct { - ID uint64 `json:"id,omitempty"` - DataNodeIDs []uint64 `json:"nodeIDs,omitempty"` // owners - - store *bolt.DB -} - -// newShardGroup returns a new initialized ShardGroup instance. -func newShardGroup() *ShardGroup { return &ShardGroup{} } - // Duration returns the duration between the shard group's start and end time. func (g *ShardGroup) Duration() time.Duration { return g.EndTime.Sub(g.StartTime) } @@ -63,11 +53,23 @@ func (g *ShardGroup) dropSeries(seriesID uint32) error { return nil } +// Shard represents the logical storage for a given time range. +// The instance on a local server may contain the raw data in "store" if the +// shard is assigned to the server's data node id. +type Shard struct { + ID uint64 `json:"id,omitempty"` + DataNodeIDs []uint64 `json:"nodeIDs,omitempty"` // owners + + index uint64 // highest replicated index + store *bolt.DB // underlying data store + conn MessagingConn // streaming connection to broker +} + // newShard returns a new initialized Shard instance. func newShard() *Shard { return &Shard{} } // open initializes and opens the shard's store. -func (s *Shard) open(path string) error { +func (s *Shard) open(path string, conn MessagingConn) error { // Return an error if the shard is already open. if s.store != nil { return errors.New("shard already open") @@ -81,23 +83,40 @@ func (s *Shard) open(path string) error { s.store = store // Initialize store. + s.index = 0 if err := s.store.Update(func(tx *bolt.Tx) error { _, _ = tx.CreateBucketIfNotExists([]byte("values")) + + // Find highest replicated index. + b, _ := tx.CreateBucketIfNotExists([]byte("meta")) + if buf := b.Get([]byte("index")); len(buf) > 0 { + s.index = btou64(buf) + } + return nil }); err != nil { _ = s.close() return fmt.Errorf("init: %s", err) } + // Open connection. + if err := conn.Open(s.index); err != nil { + _ = s.close() + return fmt.Errorf("open shard conn: id=%d, idx=%d, err=%s", s.ID, s.index, err) + } + + // Start importing from connection. + go s.processor(conn) + return nil } // close shuts down the shard's store. func (s *Shard) close() error { - if s.store == nil { - return nil + if s.store != nil { + _ = s.store.Close() } - return s.store.Close() + return nil } // HasDataNodeID return true if the data node owns the shard. @@ -127,7 +146,7 @@ func (s *Shard) readSeries(seriesID uint32, timestamp int64) (values []byte, err } // writeSeries writes series batch to a shard. -func (s *Shard) writeSeries(batch []byte) error { +func (s *Shard) writeSeries(index uint64, batch []byte) error { return s.store.Update(func(tx *bolt.Tx) error { for { if pointHeaderSize > len(batch) { @@ -159,6 +178,11 @@ func (s *Shard) writeSeries(batch []byte) error { } } + // Set index. + if err := tx.Bucket([]byte("meta")).Put([]byte("index"), u64tob(index)); err != nil { + return fmt.Errorf("write shard index: %s", err) + } + return nil }) } @@ -176,6 +200,36 @@ func (s *Shard) dropSeries(seriesID uint32) error { }) } +// processor runs in a separate goroutine and processes all incoming broker messages. +func (s *Shard) processor(conn MessagingConn) { + for { + // Read incoming message. + // Exit if the connection has been closed. + m, ok := <-conn.C() + if !ok { + return + } + + // Ignore any writes that are from an old index. + if m.Index < s.index { + continue + } + + // Handle write series separately so we don't lock server during shard writes. + switch m.Type { + case writeRawSeriesMessageType: + if err := s.writeSeries(m.Index, m.Data); err != nil { + panic(fmt.Errorf("apply shard: id=%d, idx=%d, err=%s", s.ID, m.Index, err)) + } + default: + panic(fmt.Sprintf("invalid shard message type: %d", m.Type)) + } + + // Track last index. + s.index = m.Index + } +} + // Shards represents a list of shards. type Shards []*Shard From 4160d0b785d13d374f1a9ebc1c2f4c99a6671724 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 9 Mar 2015 22:32:12 -0600 Subject: [PATCH 08/20] Add continuously streaming topic readers. --- messaging/broker.go | 65 +++++++++++++++---- messaging/broker_test.go | 134 +++++++++++++++++++++++++++++++++++++++ messaging/errors.go | 3 + 3 files changed, 188 insertions(+), 14 deletions(-) diff --git a/messaging/broker.go b/messaging/broker.go index 7260723a4d6..28b3b765ce2 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -17,6 +17,11 @@ import ( "github.com/influxdb/influxdb/raft" ) +// DefaultPollInterval is the default amount of time a topic reader will wait +// between checks for new segments or new data on an existing segment. This +// only occurs when the reader is at the end of all the data. +const DefaultPollInterval = 100 * time.Millisecond + // Broker represents distributed messaging system segmented into topics. // Each topic represents a linear series of events. type Broker struct { @@ -892,6 +897,9 @@ type TopicReader struct { file *os.File // current segment file handler closed bool + + // The time between file system polling to check for new segments. + PollInterval time.Duration } // NewTopicReader returns a new instance of TopicReader that reads segments @@ -901,6 +909,8 @@ func NewTopicReader(path string, index uint64, streaming bool) *TopicReader { path: path, index: index, streaming: streaming, + + PollInterval: DefaultPollInterval, } } @@ -908,19 +918,33 @@ func NewTopicReader(path string, index uint64, streaming bool) *TopicReader { func (r *TopicReader) Read(p []byte) (int, error) { for { // Retrieve current segment file handle. + // If the reader is closed then return EOF. + // If we don't have a file and we're streaming then sleep and retry. f, err := r.File() - if err != nil { + if err == ErrReaderClosed { + return 0, io.EOF + } else if err != nil { return 0, fmt.Errorf("file: %s", err) } else if f == nil { + if r.streaming { + time.Sleep(r.PollInterval) + continue + } return 0, io.EOF } - // Write data to buffer. + // Read under lock so the underlying file cannot be closed. + r.mu.Lock() + n, err := f.Read(p) + r.mu.Unlock() + + // Read into buffer. // If no more data is available, then retry with the next segment. - if n, err := r.file.Read(p); err == io.EOF { + if err == io.EOF { if err := r.nextSegment(); err != nil { return 0, fmt.Errorf("next segment: %s", err) } + time.Sleep(r.PollInterval) continue } else { return n, err @@ -936,7 +960,7 @@ func (r *TopicReader) File() (*os.File, error) { // Exit if closed. if r.closed { - return nil, nil + return nil, ErrReaderClosed } // If the first file hasn't been opened then open it and seek. @@ -993,30 +1017,42 @@ func (r *TopicReader) nextSegment() error { r.mu.Lock() defer r.mu.Unlock() + // Skip if the reader is closed. + if r.closed { + return nil + } + // Find current segment index. index, err := strconv.ParseUint(filepath.Base(r.file.Name()), 10, 64) if err != nil { return fmt.Errorf("parse current segment index: %s", err) } - // Clear file. - if r.file != nil { - r.file.Close() - r.file = nil - } - // Read current segment list. // If no segments exist then exit. + // If current segment is the last segment then ignore. segments, err := ReadSegments(r.path) if err != nil { return fmt.Errorf("read segments: %s", err) } else if len(segments) == 0 { return nil + } else if segments[len(segments)-1].Index == index { + if !r.streaming { + r.closed = true + } + return nil } // Loop over segments and find the next one. for i := range segments[:len(segments)-1] { if segments[i].Index == index { + // Clear current file. + if r.file != nil { + r.file.Close() + r.file = nil + } + + // Open next segment. f, err := os.Open(segments[i+1].Path) if err != nil { return fmt.Errorf("open next segment: %s", err) @@ -1026,8 +1062,7 @@ func (r *TopicReader) nextSegment() error { } } - // If we didn't find the current segment or the current segment is the - // last segment then mark the reader as closed. + // This should only occur if our current segment was deleted. r.closed = true return nil } @@ -1135,14 +1170,16 @@ func NewMessageDecoder(r io.Reader) *MessageDecoder { func (dec *MessageDecoder) Decode(m *Message) error { // Read header bytes. var b [messageHeaderSize]byte - if _, err := io.ReadFull(dec.r, b[:]); err != nil { + if _, err := io.ReadFull(dec.r, b[:]); err == io.EOF { return err + } else if err != nil { + return fmt.Errorf("read header: %s", err) } m.unmarshalHeader(b[:]) // Read data. if _, err := io.ReadFull(dec.r, m.Data); err != nil { - return err + return fmt.Errorf("read body: %s", err) } return nil diff --git a/messaging/broker_test.go b/messaging/broker_test.go index 8838c57dc78..78ac8d22c56 100644 --- a/messaging/broker_test.go +++ b/messaging/broker_test.go @@ -5,11 +5,14 @@ import ( "fmt" "io" "io/ioutil" + "math/rand" "os" "path/filepath" "reflect" "strings" + "sync" "testing" + "time" "github.com/influxdb/influxdb/messaging" "github.com/influxdb/influxdb/raft" @@ -500,7 +503,138 @@ func TestTopicReader(t *testing.T) { t.Fatalf("%d. %v: result mismatch:\n\nexp=%#v\n\ngot=%#v", i, tt.index, tt.results, results) } } +} + +// Ensure a topic reader can stream new messages. +func TestTopicReader_streaming(t *testing.T) { + path, _ := ioutil.TempDir("", "") + defer os.RemoveAll(path) + + // Start topic reader from the beginning. + r := messaging.NewTopicReader(path, 0, true) + r.PollInterval = 1 * time.Millisecond + + // Write a segments with delays. + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + + time.Sleep(2 * time.Millisecond) + MustWriteFile(filepath.Join(path, "6"), + MustMarshalMessages([]*messaging.Message{ + {Index: 6}, + {Index: 7}, + {Index: 10}, + }), + ) + + // Write two more segments. + time.Sleep(5 * time.Millisecond) + MustWriteFile(filepath.Join(path, "12"), + MustMarshalMessages([]*messaging.Message{ + {Index: 12}, + }), + ) + + MustWriteFile(filepath.Join(path, "13"), + MustMarshalMessages([]*messaging.Message{ + {Index: 13}, + {Index: 14}, + }), + ) + + // Close reader. + time.Sleep(5 * time.Millisecond) + r.Close() + }() + + // Slurp all message ids from the reader. + indices := make([]uint64, 0) + dec := messaging.NewMessageDecoder(r) + for { + m := &messaging.Message{} + if err := dec.Decode(m); err == io.EOF { + break + } else if err != nil { + t.Fatalf("decode error: %s", err) + } else { + indices = append(indices, m.Index) + } + } + + // Verify we received the correct indices. + if !reflect.DeepEqual(indices, []uint64{6, 7, 10, 12, 13, 14}) { + t.Fatalf("unexpected indices: %#v", indices) + } + + wg.Wait() +} + +// Ensure multiple topic readers can read from the same topic directory. +func BenchmarkTopicReaderStreaming(b *testing.B) { + path, _ := ioutil.TempDir("", "") + defer os.RemoveAll(path) + + // Configurable settings. + readerN := 10 // number of readers + messageN := b.N // total message count + dataSize := 50 // per message data size + pollInterval := 1 * time.Millisecond + + // Create a topic to write into. + topic := messaging.NewTopic(1, path) + topic.MaxSegmentSize = 64 * 1024 // 64KB + if err := topic.Open(); err != nil { + b.Fatal(err) + } + defer topic.Close() + + // Stream from multiple readers in parallel. + var wg sync.WaitGroup + wg.Add(readerN) + readers := make([]*messaging.TopicReader, readerN) + for i := range readers { + r := messaging.NewTopicReader(path, 0, true) + r.PollInterval = pollInterval + readers[i] = r + + // Read messages in sequence. + go func(r *messaging.TopicReader) { + defer r.Close() + defer wg.Done() + + var index uint64 + dec := messaging.NewMessageDecoder(r) + for { + var m messaging.Message + if err := dec.Decode(&m); err == io.EOF { + b.Fatalf("unexpected EOF") + } else if err != nil { + b.Fatalf("decode error: %s", err) + } else if index+1 != m.Index { + b.Fatalf("out of order: %d..%d", index, m.Index) + } + index = m.Index + + if index == uint64(messageN) { + break + } + } + }(r) + } + + // Write messages into topic but stagger them by small, random intervals. + for i := 0; i < messageN; i++ { + time.Sleep(time.Duration(rand.Intn(int(pollInterval)))) + + index := uint64(i) + 1 + if err := topic.WriteMessage(&messaging.Message{Index: index, Data: make([]byte, dataSize)}); err != nil { + b.Fatalf("write message error: %s", err) + } + } + wg.Wait() } // Broker is a wrapper for broker.Broker that creates the broker in a temporary location. diff --git a/messaging/errors.go b/messaging/errors.go index 34e09848354..4dc1927679a 100644 --- a/messaging/errors.go +++ b/messaging/errors.go @@ -69,4 +69,7 @@ var ( // ErrStaleWrite is returned when writing a message with an old index to a topic. ErrStaleWrite = errors.New("stale write") + + // ErrReaderClosed is returned when reading from a closed topic reader. + ErrReaderClosed = errors.New("reader closed") ) From 27e91327968b8cae51b512f7ebe3bcab9428b11b Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 10 Mar 2015 14:53:45 -0600 Subject: [PATCH 09/20] Integrate stateless broker into remaining packages. --- broker.go | 14 +- cmd/influxd/config.go | 8 +- cmd/influxd/run.go | 96 +++++++----- httpd/handler.go | 6 - httpd/handler_test.go | 315 ++++++++++++++++++++++---------------- messaging/broker.go | 46 ++++-- messaging/broker_test.go | 20 ++- messaging/client.go | 25 ++- messaging/client_test.go | 136 ++-------------- messaging/handler.go | 51 +++++- messaging/handler_test.go | 4 +- raft/config.go | 16 +- raft/config_test.go | 32 ++-- raft/handler.go | 4 +- raft/handler_test.go | 15 +- raft/log.go | 66 +++++--- raft/log_test.go | 43 +++--- raft/transport.go | 20 +-- raft/transport_test.go | 48 +++--- server.go | 27 +++- server_test.go | 252 +++++++----------------------- shard.go | 2 +- tx_test.go | 5 +- 23 files changed, 603 insertions(+), 648 deletions(-) diff --git a/broker.go b/broker.go index 1060687e611..2c5fb3cf613 100644 --- a/broker.go +++ b/broker.go @@ -1,10 +1,6 @@ package influxdb -/* import ( - "fmt" - "log" - "net/http" "time" "github.com/influxdb/influxdb/messaging" @@ -31,7 +27,7 @@ type Broker struct { done chan struct{} // send CQ processing requests to the same data node - currentCQProcessingNode *messaging.Replica + // currentCQProcessingNode *messaging.Replica // FIX(benbjohnson) // variables to control when to trigger processing and when to timeout TriggerInterval time.Duration @@ -51,10 +47,14 @@ func NewBroker() *Broker { // RunContinuousQueryLoop starts running continuous queries on a background goroutine. func (b *Broker) RunContinuousQueryLoop() { - b.done = make(chan struct{}) - go b.continuousQueryLoop(b.done) + // FIX(benbjohnson) + // b.done = make(chan struct{}) + // go b.continuousQueryLoop(b.done) } +/* + + // Close closes the broker. func (b *Broker) Close() error { if b.done != nil { diff --git a/cmd/influxd/config.go b/cmd/influxd/config.go index 7485dda0294..4452654e988 100644 --- a/cmd/influxd/config.go +++ b/cmd/influxd/config.go @@ -180,8 +180,8 @@ func (c *Config) DataAddrUDP() string { } // DataURL returns the URL required to contact the data server. -func (c *Config) DataURL() *url.URL { - return &url.URL{ +func (c *Config) DataURL() url.URL { + return url.URL{ Scheme: "http", Host: net.JoinHostPort(c.Hostname, strconv.Itoa(c.Data.Port)), } @@ -193,8 +193,8 @@ func (c *Config) BrokerAddr() string { } // BrokerURL returns the URL required to contact the Broker server. -func (c *Config) BrokerURL() *url.URL { - return &url.URL{ +func (c *Config) BrokerURL() url.URL { + return url.URL{ Scheme: "http", Host: net.JoinHostPort(c.Hostname, strconv.Itoa(c.Broker.Port)), } diff --git a/cmd/influxd/run.go b/cmd/influxd/run.go index 712177c20d5..38671ce3e64 100644 --- a/cmd/influxd/run.go +++ b/cmd/influxd/run.go @@ -20,6 +20,7 @@ import ( "github.com/influxdb/influxdb/graphite" "github.com/influxdb/influxdb/httpd" "github.com/influxdb/influxdb/messaging" + "github.com/influxdb/influxdb/raft" "github.com/influxdb/influxdb/udp" ) @@ -34,20 +35,26 @@ func Run(config *Config, join, version string, logWriter *os.File) (*messaging.B initializing := !fileExists(config.BrokerDir()) && !fileExists(config.DataDir()) // Parse join urls from the --join flag. - var joinURLs []*url.URL + var joinURLs []url.URL if join == "" { joinURLs = parseURLs(config.JoinURLs()) } else { joinURLs = parseURLs(join) } - // Open broker, initialize or join as necessary. - b := openBroker(config.BrokerDir(), config.BrokerURL(), initializing, joinURLs, logWriter) + // Open broker & raft log, initialize or join as necessary. + b, l := openBroker(config.BrokerDir(), config.BrokerURL(), initializing, joinURLs, logWriter) // Start the broker handler. var h *Handler if b != nil { - h = &Handler{brokerHandler: messaging.NewHandler(b.Broker)} + h = &Handler{ + brokerHandler: &messaging.Handler{ + Broker: b.Broker, + RaftHandler: &raft.Handler{Log: l}, + }, + } + // We want to make sure we are spun up before we exit this function, so we manually listen and serve listener, err := net.Listen("tcp", config.BrokerAddr()) if err != nil { @@ -158,7 +165,7 @@ func Run(config *Config, join, version string, logWriter *os.File) (*messaging.B // unless disabled, start the loop to report anonymous usage stats every 24h if !config.ReportingDisabled { - clusterID := b.Broker.Log().Config().ClusterID + clusterID := b.Broker.ClusterID() go s.StartReportingLoop(version, clusterID) } @@ -206,57 +213,67 @@ func parseConfig(path, hostname string) *Config { } // creates and initializes a broker. -func openBroker(path string, u *url.URL, initializing bool, joinURLs []*url.URL, w io.Writer) *influxdb.Broker { +func openBroker(path string, u url.URL, initializing bool, joinURLs []url.URL, w io.Writer) (*influxdb.Broker, *raft.Log) { // Ignore if there's no existing broker and we're not initializing or joining. if !fileExists(path) && !initializing && len(joinURLs) == 0 { - return nil + return nil, nil } + // Create raft log. + l := raft.NewLog() + l.SetURL(u) + l.SetLogOutput(w) + // Create broker. b := influxdb.NewBroker() + b.Log = l b.SetLogOutput(w) - if err := b.Open(path, u); err != nil { + // Open broker so it can feed last index data to the log. + if err := b.Open(path); err != nil { log.Fatalf("failed to open broker: %s", err) } - // If this is a new broker then we can initialie two ways: + // Attach the broker as the finite state machine of the raft log. + l.FSM = &messaging.RaftFSM{Broker: b} + + // Open raft log inside broker directory. + if err := l.Open(filepath.Join(path, "raft")); err != nil { + log.Fatalf("raft: %s", err) + } + + // If this is a new raft log then we can initialie two ways: // 1) Start a brand new cluster. // 2) Join an existing cluster. if initializing { if len(joinURLs) == 0 { - initializeBroker(b) + if err := l.Initialize(); err != nil { + log.Fatalf("initialize raft log: %s", err) + } } else { - joinBroker(b, joinURLs) + joinLog(l, joinURLs) } } - return b -} - -// initializes a new broker. -func initializeBroker(b *influxdb.Broker) { - if err := b.Initialize(); err != nil { - log.Fatalf("initialize: %s", err) - } + return b, l } -// joins a broker to an existing cluster. -func joinBroker(b *influxdb.Broker, joinURLs []*url.URL) { +// joins a raft log to an existing cluster. +func joinLog(l *raft.Log, joinURLs []url.URL) { // Attempts to join each server until successful. for _, u := range joinURLs { - if err := b.Join(u); err != nil { - log.Printf("join: failed to connect to broker: %s: %s", u, err) + if err := l.Join(u); err != nil { + log.Printf("join: failed to connect to raft cluster: %s: %s", u, err) } else { - log.Printf("join: connected broker to %s", u) + log.Printf("join: connected raft log to %s", u) return } } - log.Fatalf("join: failed to connect broker to any specified server") + log.Fatalf("join: failed to connect raft log to any specified server") } // creates and initializes a server. -func openServer(config *Config, b *influxdb.Broker, initializing, configExists bool, joinURLs []*url.URL, w io.Writer) *influxdb.Server { +func openServer(config *Config, b *influxdb.Broker, initializing, configExists bool, joinURLs []url.URL, w io.Writer) *influxdb.Server { // Ignore if there's no existing server and we're not initializing or joining. if !fileExists(config.Data.Dir) && !initializing && len(joinURLs) == 0 { return nil @@ -286,13 +303,13 @@ func openServer(config *Config, b *influxdb.Broker, initializing, configExists b } else if !configExists { // We are spining up a server that has no config, // but already has an initialized data directory - joinURLs = []*url.URL{b.URL()} + joinURLs = []url.URL{b.URL()} openServerClient(s, joinURLs, w) } else { if len(joinURLs) == 0 { // If a config exists, but no joinUrls are specified, fall back to the broker URL // TODO: Make sure we have a leader, and then spin up the server - joinURLs = []*url.URL{b.URL()} + joinURLs = []url.URL{b.URL()} } openServerClient(s, joinURLs, w) } @@ -301,18 +318,13 @@ func openServer(config *Config, b *influxdb.Broker, initializing, configExists b } // initializes a new server that does not yet have an ID. -func initializeServer(u *url.URL, s *influxdb.Server, b *influxdb.Broker, w io.Writer) { +func initializeServer(u url.URL, s *influxdb.Server, b *influxdb.Broker, w io.Writer) { // TODO: Create replica using the messaging client. - // Create replica on broker. - if err := b.CreateReplica(1, u); err != nil { - log.Fatalf("replica creation error: %s", err) - } - // Create messaging client. - c := messaging.NewClient(1) + c := influxdb.NewMessagingClient() c.SetLogOutput(w) - if err := c.Open(filepath.Join(s.Path(), messagingClientFile), []*url.URL{b.URL()}); err != nil { + if err := c.Open(filepath.Join(s.Path(), messagingClientFile), []url.URL{b.URL()}); err != nil { log.Fatalf("messaging client error: %s", err) } if err := s.SetClient(c); err != nil { @@ -326,12 +338,12 @@ func initializeServer(u *url.URL, s *influxdb.Server, b *influxdb.Broker, w io.W } // joins a server to an existing cluster. -func joinServer(s *influxdb.Server, u *url.URL, joinURLs []*url.URL) { +func joinServer(s *influxdb.Server, u url.URL, joinURLs []url.URL) { // TODO: Use separate broker and data join urls. // Create data node on an existing data node. for _, joinURL := range joinURLs { - if err := s.Join(u, joinURL); err != nil { + if err := s.Join(&u, &joinURL); err != nil { log.Printf("join: failed to connect data node: %s: %s", u, err) } else { log.Printf("join: connected data node to %s", u) @@ -342,8 +354,8 @@ func joinServer(s *influxdb.Server, u *url.URL, joinURLs []*url.URL) { } // opens the messaging client and attaches it to the server. -func openServerClient(s *influxdb.Server, joinURLs []*url.URL, w io.Writer) { - c := messaging.NewClient(s.ID()) +func openServerClient(s *influxdb.Server, joinURLs []url.URL, w io.Writer) { + c := influxdb.NewMessagingClient() c.SetLogOutput(w) if err := c.Open(filepath.Join(s.Path(), messagingClientFile), joinURLs); err != nil { log.Fatalf("messaging client error: %s", err) @@ -354,7 +366,7 @@ func openServerClient(s *influxdb.Server, joinURLs []*url.URL, w io.Writer) { } // parses a comma-delimited list of URLs. -func parseURLs(s string) (a []*url.URL) { +func parseURLs(s string) (a []url.URL) { if s == "" { return nil } @@ -364,7 +376,7 @@ func parseURLs(s string) (a []*url.URL) { if err != nil { log.Fatalf("cannot parse urls: %s", err) } - a = append(a, u) + a = append(a, *u) } return } diff --git a/httpd/handler.go b/httpd/handler.go index 2cb601e2bb7..842681a2814 100644 --- a/httpd/handler.go +++ b/httpd/handler.go @@ -383,12 +383,6 @@ func (h *Handler) serveCreateDataNode(w http.ResponseWriter, r *http.Request) { // Retrieve data node reference. node := h.server.DataNodeByURL(u) - // Create a new replica on the broker. - if err := h.server.Client().CreateReplica(node.ID, node.URL); err != nil { - httpError(w, err.Error(), false, http.StatusBadGateway) - return - } - // Write new node back to client. w.WriteHeader(http.StatusCreated) w.Header().Add("content-type", "application/json") diff --git a/httpd/handler_test.go b/httpd/handler_test.go index f975650913b..8911fac1804 100644 --- a/httpd/handler_test.go +++ b/httpd/handler_test.go @@ -11,14 +11,13 @@ import ( "net/url" "os" "strings" - "sync" "testing" "time" "github.com/influxdb/influxdb" "github.com/influxdb/influxdb/httpd" "github.com/influxdb/influxdb/influxql" - "github.com/influxdb/influxdb/messaging" + "github.com/influxdb/influxdb/test" ) func init() { @@ -137,7 +136,9 @@ func TestBatchWrite_UnmarshalRFC(t *testing.T) { } func TestHandler_Databases(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateDatabase("bar") s := NewHTTPServer(srvr) @@ -152,7 +153,9 @@ func TestHandler_Databases(t *testing.T) { } func TestHandler_DatabasesPrettyPrinted(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateDatabase("bar") s := NewHTTPServer(srvr) @@ -187,7 +190,9 @@ func TestHandler_DatabasesPrettyPrinted(t *testing.T) { } func TestHandler_CreateDatabase(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -200,7 +205,9 @@ func TestHandler_CreateDatabase(t *testing.T) { } func TestHandler_CreateDatabase_BadRequest_NoName(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -211,7 +218,9 @@ func TestHandler_CreateDatabase_BadRequest_NoName(t *testing.T) { } func TestHandler_CreateDatabase_Conflict(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") s := NewHTTPServer(srvr) defer s.Close() @@ -225,7 +234,9 @@ func TestHandler_CreateDatabase_Conflict(t *testing.T) { } func TestHandler_DropDatabase(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") s := NewHTTPServer(srvr) defer s.Close() @@ -239,7 +250,9 @@ func TestHandler_DropDatabase(t *testing.T) { } func TestHandler_DropDatabase_NotFound(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -252,7 +265,9 @@ func TestHandler_DropDatabase_NotFound(t *testing.T) { } func TestHandler_RetentionPolicies(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) s := NewHTTPServer(srvr) @@ -268,7 +283,9 @@ func TestHandler_RetentionPolicies(t *testing.T) { } func TestHandler_RetentionPolicies_DatabaseNotFound(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -282,7 +299,9 @@ func TestHandler_RetentionPolicies_DatabaseNotFound(t *testing.T) { } func TestHandler_CreateRetentionPolicy(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") s := NewHTTPServer(srvr) defer s.Close() @@ -298,7 +317,9 @@ func TestHandler_CreateRetentionPolicy(t *testing.T) { } func TestHandler_CreateRetentionPolicyAsDefault(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") s := NewHTTPServer(srvr) defer s.Close() @@ -321,7 +342,9 @@ func TestHandler_CreateRetentionPolicyAsDefault(t *testing.T) { } func TestHandler_CreateRetentionPolicy_DatabaseNotFound(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -334,7 +357,9 @@ func TestHandler_CreateRetentionPolicy_DatabaseNotFound(t *testing.T) { } func TestHandler_CreateRetentionPolicy_Conflict(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") s := NewHTTPServer(srvr) defer s.Close() @@ -350,7 +375,9 @@ func TestHandler_CreateRetentionPolicy_Conflict(t *testing.T) { } func TestHandler_CreateRetentionPolicy_BadRequest(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") s := NewHTTPServer(srvr) defer s.Close() @@ -364,7 +391,9 @@ func TestHandler_CreateRetentionPolicy_BadRequest(t *testing.T) { } func TestHandler_UpdateRetentionPolicy(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) s := NewHTTPServer(srvr) @@ -394,7 +423,9 @@ func TestHandler_UpdateRetentionPolicy(t *testing.T) { } func TestHandler_UpdateRetentionPolicy_BadRequest(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) s := NewHTTPServer(srvr) @@ -410,7 +441,9 @@ func TestHandler_UpdateRetentionPolicy_BadRequest(t *testing.T) { } func TestHandler_UpdateRetentionPolicy_DatabaseNotFound(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -424,7 +457,9 @@ func TestHandler_UpdateRetentionPolicy_DatabaseNotFound(t *testing.T) { } func TestHandler_UpdateRetentionPolicy_NotFound(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) s := NewHTTPServer(srvr) @@ -440,7 +475,9 @@ func TestHandler_UpdateRetentionPolicy_NotFound(t *testing.T) { } func TestHandler_DeleteRetentionPolicy(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) s := NewHTTPServer(srvr) @@ -457,7 +494,9 @@ func TestHandler_DeleteRetentionPolicy(t *testing.T) { } func TestHandler_DeleteRetentionPolicy_DatabaseNotFound(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -472,7 +511,9 @@ func TestHandler_DeleteRetentionPolicy_DatabaseNotFound(t *testing.T) { } func TestHandler_DeleteRetentionPolicy_NotFound(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") s := NewHTTPServer(srvr) defer s.Close() @@ -488,7 +529,9 @@ func TestHandler_DeleteRetentionPolicy_NotFound(t *testing.T) { } func TestHandler_GzipEnabled(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -512,7 +555,9 @@ func TestHandler_GzipEnabled(t *testing.T) { } func TestHandler_GzipDisabled(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -536,7 +581,9 @@ func TestHandler_GzipDisabled(t *testing.T) { } func TestHandler_Index(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -552,7 +599,9 @@ func TestHandler_Index(t *testing.T) { } func TestHandler_Wait(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -568,7 +617,9 @@ func TestHandler_Wait(t *testing.T) { } func TestHandler_WaitIncrement(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) @@ -586,7 +637,9 @@ func TestHandler_WaitIncrement(t *testing.T) { } func TestHandler_WaitNoIndexSpecified(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -598,7 +651,9 @@ func TestHandler_WaitNoIndexSpecified(t *testing.T) { } func TestHandler_WaitInvalidIndexSpecified(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -610,7 +665,9 @@ func TestHandler_WaitInvalidIndexSpecified(t *testing.T) { } func TestHandler_WaitExpectTimeout(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -622,7 +679,9 @@ func TestHandler_WaitExpectTimeout(t *testing.T) { } func TestHandler_Ping(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -634,7 +693,9 @@ func TestHandler_Ping(t *testing.T) { } func TestHandler_PingHead(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -646,7 +707,9 @@ func TestHandler_PingHead(t *testing.T) { } func TestHandler_Users_MultipleUsers(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateUser("jdoe", "1337", false) srvr.CreateUser("mclark", "1337", true) srvr.CreateUser("csmith", "1337", false) @@ -664,7 +727,9 @@ func TestHandler_Users_MultipleUsers(t *testing.T) { func TestHandler_UpdateUser(t *testing.T) { t.Skip() - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateUser("jdoe", "1337", false) s := NewHTTPServer(srvr) defer s.Close() @@ -685,7 +750,9 @@ func TestHandler_UpdateUser(t *testing.T) { func TestHandler_UpdateUser_PasswordBadRequest(t *testing.T) { t.Skip() - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateUser("jdoe", "1337", false) s := NewHTTPServer(srvr) defer s.Close() @@ -700,7 +767,9 @@ func TestHandler_UpdateUser_PasswordBadRequest(t *testing.T) { func TestHandler_DataNodes(t *testing.T) { t.Skip() - srvr := OpenUninitializedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenUninitializedServer(c) srvr.CreateDataNode(MustParseURL("http://localhost:1000")) srvr.CreateDataNode(MustParseURL("http://localhost:2000")) srvr.CreateDataNode(MustParseURL("http://localhost:3000")) @@ -717,7 +786,9 @@ func TestHandler_DataNodes(t *testing.T) { func TestHandler_CreateDataNode(t *testing.T) { t.Skip() - srvr := OpenUninitializedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenUninitializedServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -731,7 +802,9 @@ func TestHandler_CreateDataNode(t *testing.T) { func TestHandler_CreateDataNode_BadRequest(t *testing.T) { t.Skip() - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -745,7 +818,9 @@ func TestHandler_CreateDataNode_BadRequest(t *testing.T) { func TestHandler_CreateDataNode_InternalServerError(t *testing.T) { t.Skip() - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -759,7 +834,9 @@ func TestHandler_CreateDataNode_InternalServerError(t *testing.T) { func TestHandler_DeleteDataNode(t *testing.T) { t.Skip() - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDataNode(MustParseURL("http://localhost:1000")) s := NewHTTPServer(srvr) defer s.Close() @@ -774,7 +851,9 @@ func TestHandler_DeleteDataNode(t *testing.T) { func TestHandler_DeleteUser_DataNodeNotFound(t *testing.T) { t.Skip() - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -789,7 +868,9 @@ func TestHandler_DeleteUser_DataNodeNotFound(t *testing.T) { // Perform a subset of endpoint testing, with authentication enabled. func TestHandler_AuthenticatedCreateAdminUser(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) s := NewAuthenticatedHTTPServer(srvr) defer s.Close() @@ -810,7 +891,9 @@ func TestHandler_AuthenticatedCreateAdminUser(t *testing.T) { } func TestHandler_AuthenticatedDatabases_Unauthorized(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) s := NewAuthenticatedHTTPServer(srvr) defer s.Close() @@ -821,7 +904,9 @@ func TestHandler_AuthenticatedDatabases_Unauthorized(t *testing.T) { } func TestHandler_AuthenticatedDatabases_AuthorizedQueryParams(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) srvr.CreateUser("lisa", "password", true) s := NewAuthenticatedHTTPServer(srvr) defer s.Close() @@ -834,7 +919,9 @@ func TestHandler_AuthenticatedDatabases_AuthorizedQueryParams(t *testing.T) { } func TestHandler_AuthenticatedDatabases_UnauthorizedQueryParams(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) srvr.CreateUser("lisa", "password", true) s := NewAuthenticatedHTTPServer(srvr) defer s.Close() @@ -847,7 +934,9 @@ func TestHandler_AuthenticatedDatabases_UnauthorizedQueryParams(t *testing.T) { } func TestHandler_AuthenticatedDatabases_AuthorizedBasicAuth(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) srvr.CreateUser("lisa", "password", true) s := NewAuthenticatedHTTPServer(srvr) defer s.Close() @@ -862,7 +951,9 @@ func TestHandler_AuthenticatedDatabases_AuthorizedBasicAuth(t *testing.T) { } func TestHandler_AuthenticatedDatabases_UnauthorizedBasicAuth(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) srvr.CreateUser("lisa", "password", true) s := NewAuthenticatedHTTPServer(srvr) defer s.Close() @@ -877,7 +968,9 @@ func TestHandler_AuthenticatedDatabases_UnauthorizedBasicAuth(t *testing.T) { } func TestHandler_GrantDBPrivilege(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) // Create a cluster admin that will grant privilege to "john". srvr.CreateUser("lisa", "password", true) // Create user that will be granted a privilege. @@ -914,7 +1007,9 @@ func TestHandler_GrantDBPrivilege(t *testing.T) { } func TestHandler_RevokeAdmin(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) // Create a cluster admin that will revoke admin from "john". srvr.CreateUser("lisa", "password", true) // Create user that will have cluster admin revoked. @@ -946,7 +1041,9 @@ func TestHandler_RevokeAdmin(t *testing.T) { } func TestHandler_RevokeDBPrivilege(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) // Create a cluster admin that will revoke privilege from "john". srvr.CreateUser("lisa", "password", true) // Create user that will have privilege revoked. @@ -980,7 +1077,9 @@ func TestHandler_RevokeDBPrivilege(t *testing.T) { } func TestHandler_DropSeries(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) s := NewHTTPServer(srvr) @@ -1001,7 +1100,9 @@ func TestHandler_DropSeries(t *testing.T) { } func TestHandler_serveWriteSeries(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) s := NewHTTPServer(srvr) @@ -1015,7 +1116,9 @@ func TestHandler_serveWriteSeries(t *testing.T) { } func TestHandler_serveWriteSeriesWithNoFields(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) s := NewHTTPServer(srvr) @@ -1033,7 +1136,9 @@ func TestHandler_serveWriteSeriesWithNoFields(t *testing.T) { } func TestHandler_serveWriteSeriesWithAuthNilUser(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) s := NewAuthenticatedHTTPServer(srvr) @@ -1052,7 +1157,9 @@ func TestHandler_serveWriteSeriesWithAuthNilUser(t *testing.T) { } func TestHandler_serveWriteSeries_noDatabaseExists(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -1070,7 +1177,9 @@ func TestHandler_serveWriteSeries_noDatabaseExists(t *testing.T) { } func TestHandler_serveWriteSeries_invalidJSON(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -1087,7 +1196,9 @@ func TestHandler_serveWriteSeries_invalidJSON(t *testing.T) { } func TestHandler_serveWriteSeries_noDatabaseSpecified(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) s := NewHTTPServer(srvr) defer s.Close() @@ -1104,7 +1215,9 @@ func TestHandler_serveWriteSeries_noDatabaseSpecified(t *testing.T) { } func TestHandler_serveWriteSeriesNonZeroTime(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) srvr.SetDefaultRetentionPolicy("foo", "bar") @@ -1145,7 +1258,9 @@ func TestHandler_serveWriteSeriesNonZeroTime(t *testing.T) { } func TestHandler_serveWriteSeriesZeroTime(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) srvr.SetDefaultRetentionPolicy("foo", "bar") @@ -1198,7 +1313,9 @@ func TestHandler_serveWriteSeriesZeroTime(t *testing.T) { } func TestHandler_serveWriteSeriesBatch(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) srvr.SetDefaultRetentionPolicy("foo", "bar") @@ -1281,7 +1398,9 @@ func TestHandler_serveWriteSeriesBatch(t *testing.T) { } func TestHandler_serveWriteSeriesFieldTypeConflict(t *testing.T) { - srvr := OpenAuthlessServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthlessServer(c) srvr.CreateDatabase("foo") srvr.CreateRetentionPolicy("foo", influxdb.NewRetentionPolicy("bar")) srvr.SetDefaultRetentionPolicy("foo", "bar") @@ -1322,7 +1441,9 @@ func str2iface(strs []string) []interface{} { } func TestHandler_ProcessContinousQueries(t *testing.T) { - srvr := OpenAuthenticatedServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + srvr := OpenAuthenticatedServer(c) s := NewAuthenticatedHTTPServer(srvr) defer s.Close() @@ -1408,7 +1529,7 @@ func NewServer() *Server { // OpenAuthenticatedServer returns a new, open test server instance with authentication enabled. func OpenAuthenticatedServer(client influxdb.MessagingClient) *Server { s := OpenUninitializedServer(client) - if err := s.Initialize(&url.URL{Host: "127.0.0.1:8080"}); err != nil { + if err := s.Initialize(url.URL{Host: "127.0.0.1:8080"}); err != nil { panic(err.Error()) } return s @@ -1457,72 +1578,6 @@ func OpenUninitializedServer(client influxdb.MessagingClient) *Server { return s } -// TODO corylanou: evaluate how much of this should be in this package -// vs. how much should be a mocked out interface -// MessagingClient represents a test client for the messaging broker. -type MessagingClient struct { - index uint64 - c chan *messaging.Message - mu sync.Mutex // Ensure all publishing is serialized. - - PublishFunc func(*messaging.Message) (uint64, error) - CreateReplicaFunc func(replicaID uint64, connectURL *url.URL) error - DeleteReplicaFunc func(replicaID uint64) error - SubscribeFunc func(replicaID, topicID uint64) error - UnsubscribeFunc func(replicaID, topicID uint64) error -} - -// NewMessagingClient returns a new instance of MessagingClient. -func NewMessagingClient() *MessagingClient { - c := &MessagingClient{c: make(chan *messaging.Message, 1)} - c.PublishFunc = c.send - c.CreateReplicaFunc = func(replicaID uint64, connectURL *url.URL) error { return nil } - c.DeleteReplicaFunc = func(replicaID uint64) error { return nil } - c.SubscribeFunc = func(replicaID, topicID uint64) error { return nil } - c.UnsubscribeFunc = func(replicaID, topicID uint64) error { return nil } - return c -} - -// Publish attaches an autoincrementing index to the message. -// This function also execute's the client's PublishFunc mock function. -func (c *MessagingClient) Publish(m *messaging.Message) (uint64, error) { - c.mu.Lock() - defer c.mu.Unlock() - c.index++ - m.Index = c.index - return c.PublishFunc(m) -} - -// send sends the message through to the channel. -// This is the default value of PublishFunc. -func (c *MessagingClient) send(m *messaging.Message) (uint64, error) { - c.c <- m - return m.Index, nil -} - -// Creates a new replica with a given ID on the broker. -func (c *MessagingClient) CreateReplica(replicaID uint64, connectURL *url.URL) error { - return c.CreateReplicaFunc(replicaID, connectURL) -} - -// Deletes an existing replica with a given ID from the broker. -func (c *MessagingClient) DeleteReplica(replicaID uint64) error { - return c.DeleteReplicaFunc(replicaID) -} - -// Subscribe adds a subscription to a replica for a topic on the broker. -func (c *MessagingClient) Subscribe(replicaID, topicID uint64) error { - return c.SubscribeFunc(replicaID, topicID) -} - -// Unsubscribe removes a subscrition from a replica for a topic on the broker. -func (c *MessagingClient) Unsubscribe(replicaID, topicID uint64) error { - return c.UnsubscribeFunc(replicaID, topicID) -} - -// C returns a channel for streaming message. -func (c *MessagingClient) C() <-chan *messaging.Message { return c.c } - // tempfile returns a temporary path. func tempfile() string { f, _ := ioutil.TempFile("", "influxdb-") diff --git a/messaging/broker.go b/messaging/broker.go index 28b3b765ce2..c1fc503494a 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -6,6 +6,7 @@ import ( "fmt" "io" "log" + "net/url" "os" "path/filepath" "sort" @@ -34,6 +35,9 @@ type Broker struct { // Log is the distributed raft log that commands are applied to. Log interface { + URL() url.URL + Leader() (uint64, url.URL) + ClusterID() uint64 Apply(data []byte) (index uint64, err error) } @@ -61,6 +65,18 @@ func (b *Broker) metaPath() string { return filepath.Join(b.path, "meta") } +// URL returns the URL of the broker. +func (b *Broker) URL() url.URL { return b.Log.URL() } + +// LeaderURL returns the URL to the leader broker. +func (b *Broker) LeaderURL() url.URL { + _, u := b.Log.Leader() + return u +} + +// ClusterID returns the identifier for the cluster. +func (b *Broker) ClusterID() uint64 { return b.Log.ClusterID() } + // TopicPath returns the file path to a topic's data. // Returns a blank string if the broker is closed. func (b *Broker) TopicPath(id uint64) string { @@ -86,10 +102,10 @@ func (b *Broker) Topic(id uint64) *Topic { // Index returns the highest index seen by the broker across all topics. // Returns 0 if the broker is closed. -func (b *Broker) Index() uint64 { +func (b *Broker) Index() (uint64, error) { b.mu.RLock() defer b.mu.RUnlock() - return b.index + return b.index, nil } // opened returns true if the broker is in an open and running state. @@ -268,7 +284,7 @@ func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) { // Read segments from disk, not topic. segments, err := ReadSegments(t.path) - if err != nil { + if err != nil && !os.IsNotExist(err) { return nil, fmt.Errorf("read segments: %s", err) } @@ -401,7 +417,7 @@ func (b *Broker) Publish(m *Message) (uint64, error) { } // TopicReader returns a new topic reader for a topic starting from a given index. -func (b *Broker) TopicReader(topicID, index uint64, streaming bool) *TopicReader { +func (b *Broker) TopicReader(topicID, index uint64, streaming bool) io.ReadCloser { return NewTopicReader(b.TopicPath(topicID), index, streaming) } @@ -617,7 +633,7 @@ func (t *Topic) Open() error { // Read available segments. segments, err := ReadSegments(t.path) - if err != nil { + if err != nil && !os.IsNotExist(err) { t.close() return fmt.Errorf("read segments: %s", err) } @@ -669,7 +685,7 @@ func (t *Topic) close() error { func (t *Topic) ReadIndex() (uint64, error) { // Read a list of all segments. segments, err := ReadSegments(t.path) - if err != nil { + if err != nil && !os.IsNotExist(err) { return 0, fmt.Errorf("read segments: %s", err) } @@ -839,7 +855,9 @@ func ReadSegments(path string) (Segments, error) { func ReadSegmentByIndex(path string, index uint64) (*Segment, error) { // Find a list of all segments. segments, err := ReadSegments(path) - if err != nil { + if os.IsNotExist(err) { + return nil, err + } else if err != nil { return nil, fmt.Errorf("read segments: %s", err) } @@ -869,7 +887,9 @@ func ReadSegmentByIndex(path string, index uint64) (*Segment, error) { func ReadSegmentMaxIndex(path string) (uint64, error) { // Open segment file. f, err := os.Open(path) - if err != nil { + if os.IsNotExist(err) { + return 0, err + } else if err != nil { return 0, fmt.Errorf("open: %s", err) } defer func() { _ = f.Close() }() @@ -966,9 +986,11 @@ func (r *TopicReader) File() (*os.File, error) { // If the first file hasn't been opened then open it and seek. if r.file == nil { // Find the segment containing the index. - // Exit if no segments are available. + // Exit if no segments are available or if path not found. segment, err := ReadSegmentByIndex(r.path, r.index) - if err != nil { + if os.IsNotExist(err) { + return nil, nil + } else if err != nil { return nil, fmt.Errorf("segment by index: %s", err) } else if segment == nil { return nil, nil @@ -1032,7 +1054,9 @@ func (r *TopicReader) nextSegment() error { // If no segments exist then exit. // If current segment is the last segment then ignore. segments, err := ReadSegments(r.path) - if err != nil { + if os.IsNotExist(err) { + return nil + } else if err != nil { return fmt.Errorf("read segments: %s", err) } else if len(segments) == 0 { return nil diff --git a/messaging/broker_test.go b/messaging/broker_test.go index 78ac8d22c56..2f7d30984a6 100644 --- a/messaging/broker_test.go +++ b/messaging/broker_test.go @@ -6,6 +6,7 @@ import ( "io" "io/ioutil" "math/rand" + "net/url" "os" "path/filepath" "reflect" @@ -18,6 +19,11 @@ import ( "github.com/influxdb/influxdb/raft" ) +func init() { + // Ensure the broker matches the handler's interface. + _ = messaging.Handler{Broker: messaging.NewBroker()} +} + // Ensure that opening a broker without a path returns an error. func TestBroker_Open_ErrPathRequired(t *testing.T) { b := messaging.NewBroker() @@ -97,7 +103,7 @@ func TestBroker_Apply(t *testing.T) { } // Verify broker high water mark. - if index := b.Index(); index != 4 { + if index, _ := b.Index(); index != 4 { t.Fatalf("unexpected broker index: %d", index) } } @@ -152,7 +158,7 @@ func TestBroker_Reopen(t *testing.T) { } // Verify broker high water mark. - if index := b.Index(); index != 4 { + if index, _ := b.Index(); index != 4 { t.Fatalf("unexpected broker index: %d", index) } @@ -218,7 +224,7 @@ func TestBroker_Snapshot(t *testing.T) { } // Verify broker high water mark. - if index := b1.Index(); index != 4 { + if index, _ := b1.Index(); index != 4 { t.Fatalf("unexpected broker index: %d", index) } } @@ -710,10 +716,16 @@ func (b *Broker) MustReadAllTopic(topicID uint64) (a []*messaging.Message) { // BrokerLog is a mockable object that implements Broker.Log. type BrokerLog struct { - ApplyFunc func(data []byte) (uint64, error) + ApplyFunc func(data []byte) (uint64, error) + ClusterIDFunc func() uint64 + LeaderFunc func() (uint64, url.URL) + URLFunc func() url.URL } func (l *BrokerLog) Apply(data []byte) (uint64, error) { return l.ApplyFunc(data) } +func (l *BrokerLog) ClusterID() uint64 { return l.ClusterIDFunc() } +func (l *BrokerLog) Leader() (uint64, url.URL) { return l.LeaderFunc() } +func (l *BrokerLog) URL() url.URL { return l.URLFunc() } // Messages represents a collection of messages. // This type provides helper functions. diff --git a/messaging/client.go b/messaging/client.go index 6f90f5c7357..5acd3da9bff 100644 --- a/messaging/client.go +++ b/messaging/client.go @@ -252,10 +252,11 @@ func NewClientConfig(u []url.URL) *ClientConfig { // Conn represents a stream over the client for a single topic. type Conn struct { - mu sync.Mutex - topicID uint64 // topic identifier - index uint64 // highest index sent over the channel - url url.URL // current broker url + mu sync.Mutex + topicID uint64 // topic identifier + index uint64 // highest index sent over the channel + streaming bool // use streaming reader, if true + url url.URL // current broker url opened bool c chan *Message // channel streams messages from the broker. @@ -299,6 +300,13 @@ func (c *Conn) SetIndex(index uint64) { c.index = index } +// Streaming returns true if the connection streams messages continuously. +func (c *Conn) Streaming() bool { + c.mu.Lock() + defer c.mu.Unlock() + return c.streaming +} + // URL returns the current URL of the connection. func (c *Conn) URL() url.URL { c.mu.Lock() @@ -314,7 +322,7 @@ func (c *Conn) SetURL(u url.URL) { } // Open opens a streaming connection to the broker. -func (c *Conn) Open(index uint64) error { +func (c *Conn) Open(index uint64, streaming bool) error { c.mu.Lock() defer c.mu.Unlock() @@ -328,6 +336,7 @@ func (c *Conn) Open(index uint64) error { // Set starting index. c.index = index + c.streaming = streaming // Create streaming channel. c.c = make(chan *Message, 0) @@ -430,8 +439,9 @@ func (c *Conn) streamer(closing <-chan struct{}) { u := c.URL() u.Path = "/messaging/messages" u.RawQuery = url.Values{ - "topicID": {strconv.FormatUint(c.topicID, 10)}, - "index": {strconv.FormatUint(c.Index(), 10)}, + "topicID": {strconv.FormatUint(c.topicID, 10)}, + "index": {strconv.FormatUint(c.Index(), 10)}, + "streaming": {strconv.FormatBool(c.Streaming())}, }.Encode() // Create request. @@ -479,6 +489,7 @@ func (c *Conn) stream(req *http.Request, closing <-chan struct{}) error { // Decode message from the stream. m := &Message{} if err := dec.Decode(m); err == io.EOF { + warn("EOF!!!") return nil } else if err != nil { return fmt.Errorf("decode: %s", err) diff --git a/messaging/client_test.go b/messaging/client_test.go index 15c6039b726..1b7f0fdf92e 100644 --- a/messaging/client_test.go +++ b/messaging/client_test.go @@ -31,7 +31,7 @@ func TestClient_Conn(t *testing.T) { // Connect on topic #1. conn1 := c.Conn(1) - if err := conn1.Open(0); err != nil { + if err := conn1.Open(0, false); err != nil { t.Fatal(err) } else if m := <-conn1.C(); !reflect.DeepEqual(m, &messaging.Message{Index: 1, Data: []byte{100}}) { t.Fatalf("unexpected message(1): %#v", m) @@ -39,7 +39,7 @@ func TestClient_Conn(t *testing.T) { // Connect on topic #2. conn2 := c.Conn(2) - if err := conn2.Open(0); err != nil { + if err := conn2.Open(0, false); err != nil { t.Fatal(err) } else if m := <-conn2.C(); !reflect.DeepEqual(m, &messaging.Message{Index: 2, Data: []byte{200}}) { t.Fatalf("unexpected message(2): %#v", m) @@ -54,9 +54,9 @@ func TestClient_Conn(t *testing.T) { // Ensure that an error is returned when opening an opened connection. func TestConn_Open_ErrConnOpen(t *testing.T) { c := messaging.NewConn(1) - c.Open(0) + c.Open(0, false) defer c.Close() - if err := c.Open(0); err != messaging.ErrConnOpen { + if err := c.Open(0, false); err != messaging.ErrConnOpen { t.Fatalf("unexpected error: %s", err) } } @@ -64,9 +64,9 @@ func TestConn_Open_ErrConnOpen(t *testing.T) { // Ensure that an error is returned when opening a previously closed connection. func TestConn_Open_ErrConnCannotReuse(t *testing.T) { c := messaging.NewConn(1) - c.Open(0) + c.Open(0, false) c.Close() - if err := c.Open(0); err != messaging.ErrConnCannotReuse { + if err := c.Open(0, false); err != messaging.ErrConnCannotReuse { t.Fatalf("unexpected error: %s", err) } } @@ -74,7 +74,7 @@ func TestConn_Open_ErrConnCannotReuse(t *testing.T) { // Ensure that an error is returned when closing a closed connection. func TestConn_Close_ErrConnClosed(t *testing.T) { c := messaging.NewConn(1) - c.Open(0) + c.Open(0, false) c.Close() if err := c.Close(); err != messaging.ErrConnClosed { t.Fatalf("unexpected error: %s", err) @@ -102,7 +102,7 @@ func TestConn_Open(t *testing.T) { // Create and open connection to server. c := messaging.NewConn(100) c.SetURL(*MustParseURL(s.URL)) - if err := c.Open(200); err != nil { + if err := c.Open(200, false); err != nil { t.Fatal(err) } @@ -139,7 +139,7 @@ func TestConn_Open_Reconnect(t *testing.T) { // Create and open connection to server. c := messaging.NewConn(100) c.SetURL(*MustParseURL(s.URL)) - if err := c.Open(0); err != nil { + if err := c.Open(0, false); err != nil { t.Fatal(err) } @@ -179,124 +179,6 @@ func TestConn_Heartbeat(t *testing.T) { } } -/* -// Ensure that a client can open a connect to the broker. -func TestClient_Open(t *testing.T) { - c := NewClient() - defer c.Close() - - // Create replica on broker. - c.Server.Handler.Broker().PublishSync() - - // Open client to broker. - f := NewTempFile() - defer os.Remove(f) - u, _ := url.Parse(c.Server.URL) - if err := c.Open(f, []*url.URL{u}); err != nil { - t.Fatalf("unexpected error: %s", err) - } - - // Receive messages from the stream. - if m := <-c.C(); m.Type != messaging.InternalMessageType { - t.Fatalf("message type mismatch(internal): %x", m.Type) - } else if m = <-c.C(); m.Type != messaging.CreateReplicaMessageType { - t.Fatalf("message type mismatch(create replica): %x", m.Type) - } - - // Close connection to the broker. - if err := c.Client.Close(); err != nil { - t.Fatalf("unexpected error: %s", err) - } -} - -// Ensure that opening an already open client returns an error. -func TestClient_Open_ErrClientOpen(t *testing.T) { - c := NewClient(1000) - defer c.Close() - - // Open client to broker. - f := NewTempFile() - defer os.Remove(f) - u, _ := url.Parse(c.Server.URL) - c.Open(f, []*url.URL{u}) - if err := c.Open(f, []*url.URL{u}); err != messaging.ErrClientOpen { - t.Fatalf("unexpected error: %s", err) - } -} - -// Ensure that opening a client without a broker URL returns an error. -func TestClient_Open_ErrBrokerURLRequired(t *testing.T) { - t.Skip() - c := NewClient(1000) - defer c.Close() - f := NewTempFile() - defer os.Remove(f) - if err := c.Open(f, []*url.URL{}); err != messaging.ErrBrokerURLRequired { - t.Fatalf("unexpected error: %s", err) - } -} - -// Ensure that a client can close while a message is pending. -func TestClient_Close(t *testing.T) { - c := NewClient(1000) - defer c.Close() - - // Create replica on broker. - c.Server.Handler.Broker().CreateReplica(1000, &url.URL{Host: "localhost"}) - - // Open client to broker. - f := NewTempFile() - defer os.Remove(f) - u, _ := url.Parse(c.Server.URL) - if err := c.Open(f, []*url.URL{u}); err != nil { - t.Fatalf("unexpected error: %s", err) - } - time.Sleep(10 * time.Millisecond) - - // Close connection to the broker. - if err := c.Client.Close(); err != nil { - t.Fatalf("unexpected error: %s", err) - } -} - -// Ensure that a client can publish messages to the broker. -func TestClient_Publish(t *testing.T) { - c := OpenClient(1000) - defer c.Close() - - // Publish message to the broker. - if index, err := c.Publish(&messaging.Message{Type: 100, TopicID: messaging.BroadcastTopicID, Data: []byte{0}}); err != nil { - t.Fatalf("unexpected error: %v", err) - } else if index != 3 { - t.Fatalf("unexpected index: %d", index) - } -} - -// Ensure that a client receives an error when publishing to a stopped server. -func TestClient_Publish_ErrConnectionRefused(t *testing.T) { - c := OpenClient(1000) - c.Server.Close() - defer c.Close() - - // Publish message to the broker. - if _, err := c.Publish(&messaging.Message{Type: 100, TopicID: 0, Data: []byte{0}}); err == nil || !strings.Contains(err.Error(), "connection refused") { - t.Fatalf("unexpected error: %v", err) - } -} - -// Ensure that a client receives an error when publishing to a closed broker. -func TestClient_Publish_ErrLogClosed(t *testing.T) { - c := OpenClient(1000) - c.Server.Handler.Broker().Close() - defer c.Close() - - // Publish message to the broker. - if _, err := c.Publish(&messaging.Message{Type: 100, TopicID: 0, Data: []byte{0}}); err == nil || err.Error() != "log closed" { - t.Fatalf("unexpected error: %v", err) - } -} -*/ - // Client represents a test wrapper for the broker client. type Client struct { *messaging.Client diff --git a/messaging/handler.go b/messaging/handler.go index c302da83357..9c7bef5186e 100644 --- a/messaging/handler.go +++ b/messaging/handler.go @@ -3,6 +3,7 @@ package messaging import ( "io" "io/ioutil" + "log" "net/http" "net/url" "strconv" @@ -14,7 +15,7 @@ import ( // Handler represents an HTTP handler by the broker. type Handler struct { Broker interface { - LeaderURL() *url.URL + LeaderURL() url.URL TopicReader(topicID, index uint64, streaming bool) io.ReadCloser Publish(m *Message) (uint64, error) SetTopicMaxIndex(topicID, index uint64) error @@ -76,16 +77,24 @@ func (h *Handler) getMessages(w http.ResponseWriter, req *http.Request) { defer r.Close() // Ensure we close the topic reader if the connection is disconnected. + done := make(chan struct{}, 0) + defer close(done) if w, ok := w.(http.CloseNotifier); ok { go func() { select { case <-w.CloseNotify(): + _ = r.Close() + case <-done: + return } }() } // Write out all data from the topic reader. - io.Copy(w, r) + // Automatically flush as reads come in. + if _, err := CopyFlush(w, r); err != nil { + log.Printf("message stream error: %s", err) + } } // postMessages publishes a message to the broker. @@ -162,7 +171,7 @@ func (h *Handler) error(w http.ResponseWriter, err error, code int) { // redirects to the current known leader. // If no leader is found then returns a 500. func (h *Handler) redirectToLeader(w http.ResponseWriter, r *http.Request) { - if u := h.Broker.LeaderURL(); u != nil { + if u := h.Broker.LeaderURL(); u.Host != "" { redirectURL := *r.URL redirectURL.Scheme = u.Scheme redirectURL.Host = u.Host @@ -172,3 +181,39 @@ func (h *Handler) redirectToLeader(w http.ResponseWriter, r *http.Request) { h.error(w, raft.ErrNotLeader, http.StatusInternalServerError) } + +// CopyFlush copies from src to dst until EOF or an error occurs. +// Each write is proceeded by a flush, if the writer implements http.Flusher. +// +// This implementation is copied from io.Copy(). +func CopyFlush(dst io.Writer, src io.Reader) (written int64, err error) { + buf := make([]byte, 32*1024) + for { + nr, er := src.Read(buf) + if nr > 0 { + nw, ew := dst.Write(buf[0:nr]) + if nw > 0 { + written += int64(nw) + } + + // Flush after write. + if dst, ok := dst.(http.Flusher); ok { + dst.Flush() + } + + if ew != nil { + err = ew + break + } else if nr != nw { + err = io.ErrShortWrite + break + } + } else if er == io.EOF { + break + } else if er != nil { + err = er + break + } + } + return written, err +} diff --git a/messaging/handler_test.go b/messaging/handler_test.go index 9cf307d23ca..328d717eda1 100644 --- a/messaging/handler_test.go +++ b/messaging/handler_test.go @@ -224,13 +224,13 @@ func TestHandler_ErrNotFound(t *testing.T) { // HandlerBroker is a mockable type that implements Handler.Broker. type HandlerBroker struct { - LeaderURLFunc func() *url.URL + LeaderURLFunc func() url.URL PublishFunc func(m *messaging.Message) (uint64, error) TopicReaderFunc func(topicID, index uint64, streaming bool) io.ReadCloser SetTopicMaxIndexFunc func(topicID, index uint64) error } -func (b *HandlerBroker) LeaderURL() *url.URL { return b.LeaderURLFunc() } +func (b *HandlerBroker) LeaderURL() url.URL { return b.LeaderURLFunc() } func (b *HandlerBroker) Publish(m *messaging.Message) (uint64, error) { return b.PublishFunc(m) } func (b *HandlerBroker) TopicReader(topicID, index uint64, streaming bool) io.ReadCloser { return b.TopicReaderFunc(topicID, index, streaming) diff --git a/raft/config.go b/raft/config.go index 4b81e3702b0..a8d6253e789 100644 --- a/raft/config.go +++ b/raft/config.go @@ -33,7 +33,7 @@ func (c *Config) NodeByID(id uint64) *ConfigNode { } // NodeByURL returns a node by URL. -func (c *Config) NodeByURL(u *url.URL) *ConfigNode { +func (c *Config) NodeByURL(u url.URL) *ConfigNode { for _, n := range c.Nodes { if n.URL.String() == u.String() { return n @@ -43,11 +43,11 @@ func (c *Config) NodeByURL(u *url.URL) *ConfigNode { } // AddNode adds a new node to the config. -func (c *Config) AddNode(id uint64, u *url.URL) error { +func (c *Config) AddNode(id uint64, u url.URL) error { // Validate that the id is non-zero and the url exists. if id == 0 { return ErrInvalidNodeID - } else if u == nil { + } else if u.Host == "" { return ErrNodeURLRequired } @@ -97,13 +97,13 @@ func (c *Config) Clone() *Config { // ConfigNode represents a single machine in the raft configuration. type ConfigNode struct { ID uint64 - URL *url.URL + URL url.URL } // clone returns a deep copy of the node. func (n *ConfigNode) clone() *ConfigNode { - other := &ConfigNode{ID: n.ID, URL: &url.URL{}} - *other.URL = *n.URL + other := &ConfigNode{ID: n.ID} + other.URL = n.URL return other } @@ -162,11 +162,11 @@ func (dec *ConfigDecoder) Decode(c *Config) error { if err != nil { return err } else if n.URL == "" { - u = nil + u = &url.URL{} } // Append node to config. - if err := c.AddNode(n.ID, u); err != nil { + if err := c.AddNode(n.ID, *u); err != nil { return err } } diff --git a/raft/config_test.go b/raft/config_test.go index 451aa5ce58b..d40cd2af48e 100644 --- a/raft/config_test.go +++ b/raft/config_test.go @@ -14,8 +14,8 @@ import ( func TestConfig_NodeByID(t *testing.T) { c := &raft.Config{ Nodes: []*raft.ConfigNode{ - {ID: 1, URL: &url.URL{Host: "localhost:8000"}}, - {ID: 2, URL: &url.URL{Host: "localhost:9000"}}, + {ID: 1, URL: url.URL{Host: "localhost:8000"}}, + {ID: 2, URL: url.URL{Host: "localhost:9000"}}, }, } @@ -34,18 +34,18 @@ func TestConfig_NodeByID(t *testing.T) { func TestConfig_NodeByURL(t *testing.T) { c := &raft.Config{ Nodes: []*raft.ConfigNode{ - {ID: 1, URL: &url.URL{Host: "localhost:8000"}}, - {ID: 2, URL: &url.URL{Host: "localhost:9000"}}, + {ID: 1, URL: url.URL{Host: "localhost:8000"}}, + {ID: 2, URL: url.URL{Host: "localhost:9000"}}, }, } // Matching nodes should return the correct node. - if n := c.NodeByURL(&url.URL{Host: "localhost:8000"}); n != c.Nodes[0] { + if n := c.NodeByURL(url.URL{Host: "localhost:8000"}); n != c.Nodes[0] { t.Fatalf("unexpected node: %#v", n) } // Non-existent nodes should return nil. - if n := c.NodeByURL(&url.URL{Scheme: "http", Host: "localhost:8000"}); n != nil { + if n := c.NodeByURL(url.URL{Scheme: "http", Host: "localhost:8000"}); n != nil { t.Fatalf("expected nil node: %#v", n) } } @@ -53,11 +53,11 @@ func TestConfig_NodeByURL(t *testing.T) { // Ensure that the config can add nodes. func TestConfig_AddNode(t *testing.T) { var c raft.Config - c.AddNode(1, &url.URL{Host: "localhost:8000"}) - c.AddNode(2, &url.URL{Host: "localhost:9000"}) - if n := c.Nodes[0]; !reflect.DeepEqual(n, &raft.ConfigNode{ID: 1, URL: &url.URL{Host: "localhost:8000"}}) { + c.AddNode(1, url.URL{Host: "localhost:8000"}) + c.AddNode(2, url.URL{Host: "localhost:9000"}) + if n := c.Nodes[0]; !reflect.DeepEqual(n, &raft.ConfigNode{ID: 1, URL: url.URL{Host: "localhost:8000"}}) { t.Fatalf("unexpected node(0): %#v", n) - } else if n = c.Nodes[1]; !reflect.DeepEqual(n, &raft.ConfigNode{ID: 2, URL: &url.URL{Host: "localhost:9000"}}) { + } else if n = c.Nodes[1]; !reflect.DeepEqual(n, &raft.ConfigNode{ID: 2, URL: url.URL{Host: "localhost:9000"}}) { t.Fatalf("unexpected node(1): %#v", n) } } @@ -65,8 +65,8 @@ func TestConfig_AddNode(t *testing.T) { // Ensure that the config can remove nodes. func TestConfig_RemoveNode(t *testing.T) { var c raft.Config - c.AddNode(1, &url.URL{Host: "localhost:8000"}) - c.AddNode(2, &url.URL{Host: "localhost:9000"}) + c.AddNode(1, url.URL{Host: "localhost:8000"}) + c.AddNode(2, url.URL{Host: "localhost:9000"}) if err := c.RemoveNode(1); err != nil { t.Fatalf("unexpected error(0): %s", err) } else if err = c.RemoveNode(2); err != nil { @@ -83,8 +83,8 @@ func TestConfigEncoder_Encode(t *testing.T) { Index: 20, MaxNodeID: 3, Nodes: []*raft.ConfigNode{ - {ID: 1, URL: &url.URL{Host: "localhost:8000"}}, - {ID: 2, URL: &url.URL{Host: "localhost:9000"}}, + {ID: 1, URL: url.URL{Host: "localhost:8000"}}, + {ID: 2, URL: url.URL{Host: "localhost:9000"}}, }, } @@ -103,8 +103,8 @@ func TestConfigDecoder_Decode(t *testing.T) { Index: 20, MaxNodeID: 3, Nodes: []*raft.ConfigNode{ - {ID: 1, URL: &url.URL{Host: "localhost:8000"}}, - {ID: 2, URL: &url.URL{Host: "localhost:9000"}}, + {ID: 1, URL: url.URL{Host: "localhost:8000"}}, + {ID: 2, URL: url.URL{Host: "localhost:9000"}}, }, } diff --git a/raft/handler.go b/raft/handler.go index 0e271e60a45..2d517165e16 100644 --- a/raft/handler.go +++ b/raft/handler.go @@ -12,7 +12,7 @@ import ( // Handler represents an HTTP endpoint for Raft to communicate over. type Handler struct { Log interface { - AddPeer(u *url.URL) (id uint64, leaderID uint64, config *Config, err error) + AddPeer(u url.URL) (id uint64, leaderID uint64, config *Config, err error) RemovePeer(id uint64) error Heartbeat(term, commitIndex, leaderID uint64) (currentIndex uint64, err error) WriteEntriesTo(w io.Writer, id, term, index uint64) error @@ -60,7 +60,7 @@ func (h *Handler) serveJoin(w http.ResponseWriter, r *http.Request) { } // Add peer to the log. - id, leaderID, config, err := h.Log.AddPeer(u) + id, leaderID, config, err := h.Log.AddPeer(*u) if err != nil { w.Header().Set("X-Raft-Error", err.Error()) w.WriteHeader(http.StatusInternalServerError) diff --git a/raft/handler_test.go b/raft/handler_test.go index ed5e64f49c5..bb36ac19cf1 100644 --- a/raft/handler_test.go +++ b/raft/handler_test.go @@ -11,10 +11,15 @@ import ( "github.com/influxdb/influxdb/raft" ) +func init() { + // Ensure Log implements the Handler.Log interface. + _ = raft.Handler{Log: raft.NewLog()} +} + // Ensure a node can join a cluster over HTTP. func TestHandler_HandleJoin(t *testing.T) { h := NewHandler() - h.AddPeerFunc = func(u *url.URL) (uint64, uint64, *raft.Config, error) { + h.AddPeerFunc = func(u url.URL) (uint64, uint64, *raft.Config, error) { if u.String() != "http://localhost:1000" { t.Fatalf("unexpected url: %s", u) } @@ -42,7 +47,7 @@ func TestHandler_HandleJoin(t *testing.T) { // Ensure that joining with an invalid query string with return an error. func TestHandler_HandleJoin_Error(t *testing.T) { h := NewHandler() - h.AddPeerFunc = func(u *url.URL) (uint64, uint64, *raft.Config, error) { + h.AddPeerFunc = func(u url.URL) (uint64, uint64, *raft.Config, error) { return 0, 0, nil, raft.ErrClosed } s := httptest.NewServer(h) @@ -364,7 +369,7 @@ func TestHandler_Ping(t *testing.T) { // Handler represents a test wrapper for the raft.Handler. type Handler struct { *raft.Handler - AddPeerFunc func(u *url.URL) (uint64, uint64, *raft.Config, error) + AddPeerFunc func(u url.URL) (uint64, uint64, *raft.Config, error) RemovePeerFunc func(id uint64) error HeartbeatFunc func(term, commitIndex, leaderID uint64) (currentIndex uint64, err error) WriteEntriesToFunc func(w io.Writer, id, term, index uint64) error @@ -378,8 +383,8 @@ func NewHandler() *Handler { return h } -func (h *Handler) AddPeer(u *url.URL) (uint64, uint64, *raft.Config, error) { return h.AddPeerFunc(u) } -func (h *Handler) RemovePeer(id uint64) error { return h.RemovePeerFunc(id) } +func (h *Handler) AddPeer(u url.URL) (uint64, uint64, *raft.Config, error) { return h.AddPeerFunc(u) } +func (h *Handler) RemovePeer(id uint64) error { return h.RemovePeerFunc(id) } func (h *Handler) Heartbeat(term, commitIndex, leaderID uint64) (currentIndex uint64, err error) { return h.HeartbeatFunc(term, commitIndex, leaderID) diff --git a/raft/log.go b/raft/log.go index 9c160258046..f41f4864d98 100644 --- a/raft/log.go +++ b/raft/log.go @@ -104,18 +104,18 @@ type Log struct { closing chan struct{} // close notification // Network address to the reach the log. - URL *url.URL + url url.URL // The state machine that log entries will be applied to. FSM FSM // The transport used to communicate with other nodes in the cluster. Transport interface { - Join(u *url.URL, nodeURL *url.URL) (id uint64, leaderID uint64, config *Config, err error) - Leave(u *url.URL, id uint64) error - Heartbeat(u *url.URL, term, commitIndex, leaderID uint64) (lastIndex uint64, err error) - ReadFrom(u *url.URL, id, term, index uint64) (io.ReadCloser, error) - RequestVote(u *url.URL, term, candidateID, lastLogIndex, lastLogTerm uint64) error + Join(u url.URL, nodeURL url.URL) (id uint64, leaderID uint64, config *Config, err error) + Leave(u url.URL, id uint64) error + Heartbeat(u url.URL, term, commitIndex, leaderID uint64) (lastIndex uint64, err error) + ReadFrom(u url.URL, id, term, index uint64) (io.ReadCloser, error) + RequestVote(u url.URL, term, candidateID, lastLogIndex, lastLogTerm uint64) error } // Clock is an abstraction of time. @@ -154,6 +154,25 @@ func NewLog() *Log { // Returns an empty string if the log is closed. func (l *Log) Path() string { return l.path } +// URL returns the URL for the log. +func (l *Log) URL() url.URL { + l.mu.Lock() + defer l.mu.Unlock() + return l.url +} + +// SetURL sets the URL for the log. This must be set before opening. +func (l *Log) SetURL(u url.URL) { + l.mu.Lock() + defer l.mu.Unlock() + + if l.opened() { + panic("url cannot be set while log is open") + } + + l.url = u +} + func (l *Log) idPath() string { return filepath.Join(l.path, "id") } func (l *Log) termPath() string { return filepath.Join(l.path, "term") } func (l *Log) configPath() string { return filepath.Join(l.path, "config") } @@ -459,7 +478,7 @@ func (l *Log) Initialize() error { // Generate a new configuration with one node. config = &Config{MaxNodeID: id} - config.AddNode(id, l.URL) + config.AddNode(id, l.url) // Generate new 8-hex digit cluster identifier. config.ClusterID = uint64(l.Rand()) @@ -511,8 +530,8 @@ func (l *Log) SetLogOutput(w io.Writer) { func (l *Log) updateLogPrefix() { var host string - if l.URL != nil { - host = l.URL.Host + if l.url.Host != "" { + host = l.url.Host } l.Logger.SetPrefix(fmt.Sprintf("[raft] %s ", host)) } @@ -533,13 +552,24 @@ func (l *Log) tracef(msg string, v ...interface{}) { // Leader returns the id and URL associated with the current leader. // Returns zero if there is no current leader. -func (l *Log) Leader() (id uint64, u *url.URL) { +func (l *Log) Leader() (id uint64, u url.URL) { l.mu.Lock() defer l.mu.Unlock() return l.leader() } -func (l *Log) leader() (id uint64, u *url.URL) { +// ClusterID returns the identifier for the cluster. +// Returns zero if the cluster has not been initialized yet. +func (l *Log) ClusterID() uint64 { + l.mu.Lock() + defer l.mu.Unlock() + if l.config == nil { + return 0 + } + return l.config.ClusterID +} + +func (l *Log) leader() (id uint64, u url.URL) { // Ignore if there's no configuration set. if l.config == nil { return @@ -556,9 +586,9 @@ func (l *Log) leader() (id uint64, u *url.URL) { // Join contacts a node in the cluster to request membership. // A log cannot join a cluster if it has already been initialized. -func (l *Log) Join(u *url.URL) error { +func (l *Log) Join(u url.URL) error { // Validate under lock. - var nodeURL *url.URL + var nodeURL url.URL if err := func() error { l.mu.Lock() defer l.mu.Unlock() @@ -567,11 +597,11 @@ func (l *Log) Join(u *url.URL) error { return ErrClosed } else if l.id != 0 { return ErrInitialized - } else if l.URL == nil { + } else if l.url.Host == "" { return ErrURLRequired } - nodeURL = l.URL + nodeURL = l.url return nil }(); err != nil { return err @@ -727,7 +757,7 @@ func (l *Log) readFromLeader(wg *sync.WaitGroup, transitioning <-chan struct{}) l.mu.Unlock() // If no leader exists then wait momentarily and retry. - if u == nil { + if u.Host == "" { l.tracef("readFromLeader: no leader") time.Sleep(100 * time.Millisecond) continue @@ -1309,9 +1339,9 @@ func (l *Log) mustApplyRemovePeer(e *LogEntry) error { // AddPeer creates a new peer in the cluster. // Returns the new peer's identifier and the current configuration. -func (l *Log) AddPeer(u *url.URL) (uint64, uint64, *Config, error) { +func (l *Log) AddPeer(u url.URL) (uint64, uint64, *Config, error) { // Validate URL. - if u == nil { + if u.Host == "" { return 0, 0, nil, fmt.Errorf("peer url required") } diff --git a/raft/log_test.go b/raft/log_test.go index 6d18fc89455..92145b8cdd9 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -19,7 +19,7 @@ import ( // Ensure that opening an already open log returns an error. func TestLog_Open_ErrOpen(t *testing.T) { - l := NewInitializedLog(&url.URL{Host: "log0"}) + l := NewInitializedLog(url.URL{Host: "log0"}) defer l.Close() if err := l.Open(tempfile()); err != raft.ErrOpen { t.Fatal("expected error") @@ -28,7 +28,7 @@ func TestLog_Open_ErrOpen(t *testing.T) { // Ensure that a log can be checked for being open. func TestLog_Opened(t *testing.T) { - l := NewInitializedLog(&url.URL{Host: "log0"}) + l := NewInitializedLog(url.URL{Host: "log0"}) if l.Opened() != true { t.Fatalf("expected open") } @@ -40,7 +40,7 @@ func TestLog_Opened(t *testing.T) { // Ensure that reopening an existing log will restore its ID. func TestLog_Reopen(t *testing.T) { - l := NewInitializedLog(&url.URL{Host: "log0"}) + l := NewInitializedLog(url.URL{Host: "log0"}) if l.ID() != 1 { t.Fatalf("expected id == 1") } @@ -64,7 +64,7 @@ func TestLog_Reopen(t *testing.T) { // Ensure that a single node-cluster can apply a log entry. func TestLog_Apply(t *testing.T) { - l := NewInitializedLog(&url.URL{Host: "log0"}) + l := NewInitializedLog(url.URL{Host: "log0"}) defer l.Close() // Apply a command. @@ -87,7 +87,7 @@ func TestLog_Apply(t *testing.T) { // Ensure that a node has no configuration after it's closed. func TestLog_Config_Closed(t *testing.T) { - l := NewInitializedLog(&url.URL{Host: "log0"}) + l := NewInitializedLog(url.URL{Host: "log0"}) defer l.Close() l.Log.Close() if l.Config() != nil { @@ -353,12 +353,13 @@ func NewCluster(fsmFn func() raft.FSM) *Cluster { logN := 3 for i := 0; i < logN; i++ { - l := NewLog(&url.URL{Host: fmt.Sprintf("log%d", i)}) + l := NewLog(url.URL{Host: fmt.Sprintf("log%d", i)}) l.Log.FSM = fsmFn() l.Transport = t c.Logs = append(c.Logs, l) t.register(l.Log) - warnf("Log %s: %p", l.URL.String(), l.Log) + u := l.URL() + warnf("Log %s: %p", u.String(), l.Log) } warn("") @@ -372,7 +373,7 @@ func NewCluster(fsmFn func() raft.FSM) *Cluster { c.Logs[0].MustWaitUncommitted(2) c.Logs[0].Clock.apply() }() - if err := c.Logs[1].Join(c.Logs[0].URL); err != nil { + if err := c.Logs[1].Join(c.Logs[0].URL()); err != nil { panic("join: " + err.Error()) } c.Logs[0].Clock.heartbeat() @@ -390,7 +391,7 @@ func NewCluster(fsmFn func() raft.FSM) *Cluster { c.Logs[1].Clock.apply() c.Logs[2].Clock.apply() }() - if err := c.Logs[2].Log.Join(c.Logs[0].Log.URL); err != nil { + if err := c.Logs[2].Log.Join(c.Logs[0].Log.URL()); err != nil { panic("join: " + err.Error()) } @@ -409,14 +410,15 @@ func NewRealTimeCluster(logN int, fsmFn func() raft.FSM) *Cluster { t := NewTransport() for i := 0; i < logN; i++ { - l := NewLog(&url.URL{Host: fmt.Sprintf("log%d", i)}) + l := NewLog(url.URL{Host: fmt.Sprintf("log%d", i)}) l.Log.FSM = fsmFn() l.Clock = nil l.Log.Clock = raft.NewClock() l.Transport = t c.Logs = append(c.Logs, l) t.register(l.Log) - warnf("Log %s: %p", l.URL.String(), l.Log) + u := l.URL() + warnf("Log %s: %p", u.String(), l.Log) } warn("") @@ -427,7 +429,7 @@ func NewRealTimeCluster(logN int, fsmFn func() raft.FSM) *Cluster { // Join remaining nodes. for i := 1; i < logN; i++ { c.Logs[i].MustOpen() - c.Logs[i].MustJoin(c.Logs[0].URL) + c.Logs[i].MustJoin(c.Logs[0].URL()) } // Ensure nodes are ready. @@ -494,9 +496,9 @@ type Log struct { } // NewLog returns a new instance of Log. -func NewLog(u *url.URL) *Log { +func NewLog(u url.URL) *Log { l := &Log{Log: raft.NewLog(), Clock: NewClock()} - l.URL = u + l.SetURL(u) l.Log.Clock = l.Clock l.Rand = seq() l.DebugEnabled = true @@ -507,7 +509,7 @@ func NewLog(u *url.URL) *Log { } // NewInitializedLog returns a new initialized Node. -func NewInitializedLog(u *url.URL) *Log { +func NewInitializedLog(u url.URL) *Log { l := NewLog(u) l.Log.FSM = &FSM{} l.MustOpen() @@ -536,7 +538,7 @@ func (l *Log) MustInitialize() { } // MustJoin joins the log to another log. Panic on error. -func (l *Log) MustJoin(u *url.URL) { +func (l *Log) MustJoin(u url.URL) { if err := l.Join(u); err != nil { panic("join: " + err.Error()) } @@ -552,21 +554,24 @@ func (l *Log) Close() error { // MustWaits waits for at least a given applied index. Panic on error. func (l *Log) MustWait(index uint64) { if err := l.Log.Wait(index); err != nil { - panic(l.URL.String() + " wait: " + err.Error()) + u := l.URL() + panic(u.String() + " wait: " + err.Error()) } } // MustCommitted waits for at least a given committed index. Panic on error. func (l *Log) MustWaitCommitted(index uint64) { if err := l.Log.WaitCommitted(index); err != nil { - panic(l.URL.String() + " wait committed: " + err.Error()) + u := l.URL() + panic(u.String() + " wait committed: " + err.Error()) } } // MustWaitUncommitted waits for at least a given uncommitted index. Panic on error. func (l *Log) MustWaitUncommitted(index uint64) { if err := l.Log.WaitUncommitted(index); err != nil { - panic(l.URL.String() + " wait uncommitted: " + err.Error()) + u := l.URL() + panic(u.String() + " wait uncommitted: " + err.Error()) } } diff --git a/raft/transport.go b/raft/transport.go index 5ea0c4d624f..ab35f5ed367 100644 --- a/raft/transport.go +++ b/raft/transport.go @@ -15,9 +15,9 @@ import ( type HTTPTransport struct{} // Join requests membership into a node's cluster. -func (t *HTTPTransport) Join(uri *url.URL, nodeURL *url.URL) (uint64, uint64, *Config, error) { +func (t *HTTPTransport) Join(uri url.URL, nodeURL url.URL) (uint64, uint64, *Config, error) { // Construct URL. - u := *uri + u := uri u.Path = path.Join(u.Path, "raft/join") u.RawQuery = (&url.Values{"url": {nodeURL.String()}}).Encode() @@ -55,9 +55,9 @@ func (t *HTTPTransport) Join(uri *url.URL, nodeURL *url.URL) (uint64, uint64, *C } // Leave removes a node from a cluster's membership. -func (t *HTTPTransport) Leave(uri *url.URL, id uint64) error { +func (t *HTTPTransport) Leave(uri url.URL, id uint64) error { // Construct URL. - u := *uri + u := uri u.Path = path.Join(u.Path, "raft/leave") u.RawQuery = (&url.Values{"id": {strconv.FormatUint(id, 10)}}).Encode() @@ -77,9 +77,9 @@ func (t *HTTPTransport) Leave(uri *url.URL, id uint64) error { } // Heartbeat checks the status of a follower. -func (t *HTTPTransport) Heartbeat(uri *url.URL, term, commitIndex, leaderID uint64) (uint64, error) { +func (t *HTTPTransport) Heartbeat(uri url.URL, term, commitIndex, leaderID uint64) (uint64, error) { // Construct URL. - u := *uri + u := uri u.Path = path.Join(u.Path, "raft/heartbeat") // Set URL parameters. @@ -112,9 +112,9 @@ func (t *HTTPTransport) Heartbeat(uri *url.URL, term, commitIndex, leaderID uint } // ReadFrom streams the log from a leader. -func (t *HTTPTransport) ReadFrom(uri *url.URL, id, term, index uint64) (io.ReadCloser, error) { +func (t *HTTPTransport) ReadFrom(uri url.URL, id, term, index uint64) (io.ReadCloser, error) { // Construct URL. - u := *uri + u := uri u.Path = path.Join(u.Path, "raft/stream") // Set URL parameters. @@ -140,9 +140,9 @@ func (t *HTTPTransport) ReadFrom(uri *url.URL, id, term, index uint64) (io.ReadC } // RequestVote requests a vote for a candidate in a given term. -func (t *HTTPTransport) RequestVote(uri *url.URL, term, candidateID, lastLogIndex, lastLogTerm uint64) error { +func (t *HTTPTransport) RequestVote(uri url.URL, term, candidateID, lastLogIndex, lastLogTerm uint64) error { // Construct URL. - u := *uri + u := uri u.Path = path.Join(u.Path, "raft/vote") // Set URL parameters. diff --git a/raft/transport_test.go b/raft/transport_test.go index 52618a57f53..9fdbe55b46d 100644 --- a/raft/transport_test.go +++ b/raft/transport_test.go @@ -34,7 +34,7 @@ func TestHTTPTransport_Join(t *testing.T) { // Execute join against test server. u, _ := url.Parse(s.URL) - id, leaderID, config, err := (&raft.HTTPTransport{}).Join(u, &url.URL{Host: "local"}) + id, leaderID, config, err := (&raft.HTTPTransport{}).Join(*u, url.URL{Host: "local"}) if err != nil { t.Fatalf("unexpected error: %s", err) } else if id != 1 { @@ -48,7 +48,7 @@ func TestHTTPTransport_Join(t *testing.T) { // Ensure that joining a server that doesn't exist returns an error. func TestHTTPTransport_Join_ErrConnectionRefused(t *testing.T) { - _, _, _, err := (&raft.HTTPTransport{}).Join(&url.URL{Scheme: "http", Host: "localhost:27322"}, &url.URL{Host: "local"}) + _, _, _, err := (&raft.HTTPTransport{}).Join(url.URL{Scheme: "http", Host: "localhost:27322"}, url.URL{Host: "local"}) if err == nil || !strings.Contains(err.Error(), "connection refused") { t.Fatalf("unexpected error: %s", err) } @@ -64,7 +64,7 @@ func TestHTTPTransport_Join_ErrInvalidID(t *testing.T) { // Execute join against test server. u, _ := url.Parse(s.URL) - _, _, _, err := (&raft.HTTPTransport{}).Join(u, &url.URL{Host: "local"}) + _, _, _, err := (&raft.HTTPTransport{}).Join(*u, url.URL{Host: "local"}) if err == nil || err.Error() != `invalid id: "xxx"` { t.Fatalf("unexpected error: %s", err) } @@ -82,7 +82,7 @@ func TestHTTPTransport_Join_ErrInvalidConfig(t *testing.T) { // Execute join against test server. u, _ := url.Parse(s.URL) - _, _, _, err := (&raft.HTTPTransport{}).Join(u, &url.URL{Host: "local"}) + _, _, _, err := (&raft.HTTPTransport{}).Join(*u, url.URL{Host: "local"}) if err == nil || err.Error() != `config unmarshal: unexpected EOF` { t.Fatalf("unexpected error: %s", err) } @@ -99,7 +99,7 @@ func TestHTTPTransport_Join_Err(t *testing.T) { // Execute join against test server. u, _ := url.Parse(s.URL) - _, _, _, err := (&raft.HTTPTransport{}).Join(u, &url.URL{Host: "local"}) + _, _, _, err := (&raft.HTTPTransport{}).Join(*u, url.URL{Host: "local"}) if err == nil || err.Error() != `oh no` { t.Fatalf("unexpected error: %s", err) } @@ -119,14 +119,14 @@ func TestHTTPTransport_Leave(t *testing.T) { // Execute leave against test server. u, _ := url.Parse(s.URL) - if err := (&raft.HTTPTransport{}).Leave(u, 1); err != nil { + if err := (&raft.HTTPTransport{}).Leave(*u, 1); err != nil { t.Fatalf("unexpected error: %s", err) } } // Ensure that leaving a server that doesn't exist returns an error. func TestHTTPTransport_Leave_ErrConnectionRefused(t *testing.T) { - err := (&raft.HTTPTransport{}).Leave(&url.URL{Scheme: "http", Host: "localhost:27322"}, 1) + err := (&raft.HTTPTransport{}).Leave(url.URL{Scheme: "http", Host: "localhost:27322"}, 1) if err == nil || !strings.Contains(err.Error(), "connection refused") { t.Fatalf("unexpected error: %s", err) } @@ -142,7 +142,7 @@ func TestHTTPTransport_Leave_Err(t *testing.T) { // Execute leave against test server. u, _ := url.Parse(s.URL) - err := (&raft.HTTPTransport{}).Leave(u, 1) + err := (&raft.HTTPTransport{}).Leave(*u, 1) if err == nil || err.Error() != `oh no` { t.Fatalf("unexpected error: %s", err) } @@ -171,7 +171,7 @@ func TestHTTPTransport_Heartbeat(t *testing.T) { // Execute heartbeat against test server. u, _ := url.Parse(s.URL) - newIndex, err := (&raft.HTTPTransport{}).Heartbeat(u, 1, 2, 3) + newIndex, err := (&raft.HTTPTransport{}).Heartbeat(*u, 1, 2, 3) if err != nil { t.Fatalf("unexpected error: %s", err) } else if newIndex != 4 { @@ -198,7 +198,7 @@ func TestHTTPTransport_Heartbeat_Err(t *testing.T) { })) u, _ := url.Parse(s.URL) - _, err := (&raft.HTTPTransport{}).Heartbeat(u, 1, 2, 3) + _, err := (&raft.HTTPTransport{}).Heartbeat(*u, 1, 2, 3) if err == nil { t.Errorf("%d. expected error", i) } else if tt.err != err.Error() { @@ -211,7 +211,7 @@ func TestHTTPTransport_Heartbeat_Err(t *testing.T) { // Ensure an HTTP heartbeat to a stopped server returns an error. func TestHTTPTransport_Heartbeat_ErrConnectionRefused(t *testing.T) { u, _ := url.Parse("http://localhost:41932") - _, err := (&raft.HTTPTransport{}).Heartbeat(u, 0, 0, 0) + _, err := (&raft.HTTPTransport{}).Heartbeat(*u, 0, 0, 0) if err == nil { t.Fatal("expected error") } else if !strings.Contains(err.Error(), `connection refused`) { @@ -241,7 +241,7 @@ func TestHTTPTransport_ReadFrom(t *testing.T) { // Execute stream against test server. u, _ := url.Parse(s.URL) - r, err := (&raft.HTTPTransport{}).ReadFrom(u, 1, 2, 3) + r, err := (&raft.HTTPTransport{}).ReadFrom(*u, 1, 2, 3) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -261,7 +261,7 @@ func TestHTTPTransport_ReadFrom_Err(t *testing.T) { // Execute stream against test server. u, _ := url.Parse(s.URL) - r, err := (&raft.HTTPTransport{}).ReadFrom(u, 0, 0, 0) + r, err := (&raft.HTTPTransport{}).ReadFrom(*u, 0, 0, 0) if err == nil { t.Fatalf("expected error") } else if err.Error() != `bad stream` { @@ -274,7 +274,7 @@ func TestHTTPTransport_ReadFrom_Err(t *testing.T) { // Ensure an streaming over HTTP to a stopped server returns an error. func TestHTTPTransport_ReadFrom_ErrConnectionRefused(t *testing.T) { u, _ := url.Parse("http://localhost:41932") - _, err := (&raft.HTTPTransport{}).ReadFrom(u, 0, 0, 0) + _, err := (&raft.HTTPTransport{}).ReadFrom(*u, 0, 0, 0) if err == nil { t.Fatal("expected error") } else if !strings.Contains(err.Error(), `connection refused`) { @@ -307,7 +307,7 @@ func TestHTTPTransport_RequestVote(t *testing.T) { // Execute heartbeat against test server. u, _ := url.Parse(s.URL) - if err := (&raft.HTTPTransport{}).RequestVote(u, 1, 2, 3, 4); err != nil { + if err := (&raft.HTTPTransport{}).RequestVote(*u, 1, 2, 3, 4); err != nil { t.Fatalf("unexpected error: %s", err) } } @@ -322,7 +322,7 @@ func TestHTTPTransport_RequestVote_Error(t *testing.T) { defer s.Close() u, _ := url.Parse(s.URL) - if err := (&raft.HTTPTransport{}).RequestVote(u, 0, 0, 0, 0); err == nil { + if err := (&raft.HTTPTransport{}).RequestVote(*u, 0, 0, 0, 0); err == nil { t.Errorf("expected error") } else if err.Error() != `already voted` { t.Errorf("unexpected error: %s", err) @@ -332,7 +332,7 @@ func TestHTTPTransport_RequestVote_Error(t *testing.T) { // Ensure that requesting a vote over HTTP to a stopped server returns an error. func TestHTTPTransport_RequestVote_ErrConnectionRefused(t *testing.T) { u, _ := url.Parse("http://localhost:41932") - if err := (&raft.HTTPTransport{}).RequestVote(u, 0, 0, 0, 0); err == nil { + if err := (&raft.HTTPTransport{}).RequestVote(*u, 0, 0, 0, 0); err == nil { t.Fatal("expected error") } else if !strings.Contains(err.Error(), `connection refused`) { t.Fatalf("unexpected error: %s", err) @@ -352,11 +352,11 @@ func NewTransport() *Transport { // register registers a log by hostname. func (t *Transport) register(l *raft.Log) { - t.logs[l.URL.Host] = l + t.logs[l.URL().Host] = l } // log returns a log registered by hostname. -func (t *Transport) log(u *url.URL) (*raft.Log, error) { +func (t *Transport) log(u url.URL) (*raft.Log, error) { if l := t.logs[u.Host]; l != nil { return l, nil } @@ -364,7 +364,7 @@ func (t *Transport) log(u *url.URL) (*raft.Log, error) { } // Join calls the AddPeer method on the target log. -func (t *Transport) Join(u *url.URL, nodeURL *url.URL) (uint64, uint64, *raft.Config, error) { +func (t *Transport) Join(u url.URL, nodeURL url.URL) (uint64, uint64, *raft.Config, error) { l, err := t.log(u) if err != nil { return 0, 0, nil, err @@ -373,7 +373,7 @@ func (t *Transport) Join(u *url.URL, nodeURL *url.URL) (uint64, uint64, *raft.Co } // Leave calls the RemovePeer method on the target log. -func (t *Transport) Leave(u *url.URL, id uint64) error { +func (t *Transport) Leave(u url.URL, id uint64) error { l, err := t.log(u) if err != nil { return err @@ -382,7 +382,7 @@ func (t *Transport) Leave(u *url.URL, id uint64) error { } // Heartbeat calls the Heartbeat method on the target log. -func (t *Transport) Heartbeat(u *url.URL, term, commitIndex, leaderID uint64) (lastIndex uint64, err error) { +func (t *Transport) Heartbeat(u url.URL, term, commitIndex, leaderID uint64) (lastIndex uint64, err error) { l, err := t.log(u) if err != nil { return 0, err @@ -391,7 +391,7 @@ func (t *Transport) Heartbeat(u *url.URL, term, commitIndex, leaderID uint64) (l } // ReadFrom streams entries from the target log. -func (t *Transport) ReadFrom(u *url.URL, id, term, index uint64) (io.ReadCloser, error) { +func (t *Transport) ReadFrom(u url.URL, id, term, index uint64) (io.ReadCloser, error) { l, err := t.log(u) if err != nil { return nil, err @@ -409,7 +409,7 @@ func (t *Transport) ReadFrom(u *url.URL, id, term, index uint64) (io.ReadCloser, } // RequestVote calls RequestVote() on the target log. -func (t *Transport) RequestVote(u *url.URL, term, candidateID, lastLogIndex, lastLogTerm uint64) error { +func (t *Transport) RequestVote(u url.URL, term, candidateID, lastLogIndex, lastLogTerm uint64) error { l, err := t.log(u) if err != nil { return err diff --git a/server.go b/server.go index 7d617e13260..212136c147f 100644 --- a/server.go +++ b/server.go @@ -355,7 +355,7 @@ func (s *Server) setClient(client MessagingClient) error { if client != nil { // Create connection for broadcast channel. conn := client.Conn(BroadcastTopicID) - if err := conn.Open(s.index); err != nil { + if err := conn.Open(s.index, true); err != nil { return fmt.Errorf("open conn: %s", err) } @@ -417,16 +417,16 @@ func (s *Server) Sync(index uint64) error { } // Initialize creates a new data node and initializes the server's id to 1. -func (s *Server) Initialize(u *url.URL) error { +func (s *Server) Initialize(u url.URL) error { // Create a new data node. - if err := s.CreateDataNode(u); err != nil { + if err := s.CreateDataNode(&u); err != nil { return err } // Ensure the data node returns with an ID of 1. // If it doesn't then something went really wrong. We have to panic because // the messaging client relies on the first server being assigned ID 1. - n := s.DataNodeByURL(u) + n := s.DataNodeByURL(&u) assert(n != nil && n.ID == 1, "invalid initial server id: %d", n.ID) // Set the ID on the metastore. @@ -2816,16 +2816,33 @@ func (r *Results) Error() error { // MessagingClient represents the client used to connect to brokers. type MessagingClient interface { + Open(path string, urls []url.URL) error + Close() error + // Publishes a message to the broker. Publish(m *messaging.Message) (index uint64, err error) // Conn returns an open, streaming connection to a topic. Conn(topicID uint64) MessagingConn + + // Sets the logging destination. + SetLogOutput(w io.Writer) +} + +type messagingClient struct { + *messaging.Client } +// NewMessagingClient returns an instance of MessagingClient. +func NewMessagingClient() MessagingClient { + return &messagingClient{messaging.NewClient()} +} + +func (c *messagingClient) Conn(topicID uint64) MessagingConn { return c.Client.Conn(topicID) } + // MessagingConn represents a streaming connection to a single broker topic. type MessagingConn interface { - Open(index uint64) error + Open(index uint64, streaming bool) error C() <-chan *messaging.Message } diff --git a/server_test.go b/server_test.go index 8ee3d258a13..9235fcaa206 100644 --- a/server_test.go +++ b/server_test.go @@ -10,13 +10,12 @@ import ( "os" "reflect" "strings" - "sync" "testing" "time" "github.com/influxdb/influxdb" "github.com/influxdb/influxdb/influxql" - "github.com/influxdb/influxdb/messaging" + "github.com/influxdb/influxdb/test" "golang.org/x/crypto/bcrypt" ) @@ -40,7 +39,7 @@ func TestServer_Open_ErrPathRequired(t *testing.T) { t.Skip("pending") } // Ensure the server can create a new data node. func TestServer_CreateDataNode(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -64,7 +63,7 @@ func TestServer_CreateDataNode(t *testing.T) { // Ensure the server returns an error when creating a duplicate node. func TestServer_CreateDatabase_ErrDataNodeExists(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -81,7 +80,7 @@ func TestServer_CreateDatabase_ErrDataNodeExists(t *testing.T) { // Ensure the server can delete a node. func TestServer_DeleteDataNode(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -106,7 +105,7 @@ func TestServer_DeleteDataNode(t *testing.T) { // Test unuathorized requests logging func TestServer_UnauthorizedRequests(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -149,7 +148,7 @@ func TestServer_UnauthorizedRequests(t *testing.T) { // Test user privilege authorization. func TestServer_UserPrivilegeAuthorization(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -186,7 +185,7 @@ func TestServer_UserPrivilegeAuthorization(t *testing.T) { // Test single statement query authorization. func TestServer_SingleStatementQueryAuthorization(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -255,7 +254,7 @@ func TestServer_SingleStatementQueryAuthorization(t *testing.T) { // Test multiple statement query authorization. func TestServer_MultiStatementQueryAuthorization(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -302,7 +301,7 @@ func TestServer_MultiStatementQueryAuthorization(t *testing.T) { // Ensure the server can create a database. func TestServer_CreateDatabase(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -321,7 +320,7 @@ func TestServer_CreateDatabase(t *testing.T) { // Ensure the server returns an error when creating a duplicate database. func TestServer_CreateDatabase_ErrDatabaseExists(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -337,7 +336,7 @@ func TestServer_CreateDatabase_ErrDatabaseExists(t *testing.T) { // Ensure the server can drop a database. func TestServer_DropDatabase(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -360,7 +359,7 @@ func TestServer_DropDatabase(t *testing.T) { // Ensure the server returns an error when dropping a database that doesn't exist. func TestServer_DropDatabase_ErrDatabaseNotFound(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -373,7 +372,7 @@ func TestServer_DropDatabase_ErrDatabaseNotFound(t *testing.T) { // Ensure the server can return a list of all databases. func TestServer_Databases(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -395,7 +394,7 @@ func TestServer_Databases(t *testing.T) { // Ensure the server can create a new user. func TestServer_CreateUser(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -433,7 +432,7 @@ func TestServer_CreateUser(t *testing.T) { // Ensure the server correctly detects when there is an admin user. func TestServer_AdminUserExists(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -464,7 +463,7 @@ func TestServer_AdminUserExists(t *testing.T) { // Ensure the server returns an error when creating an user without a name. func TestServer_CreateUser_ErrUsernameRequired(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -475,7 +474,7 @@ func TestServer_CreateUser_ErrUsernameRequired(t *testing.T) { // Ensure the server returns an error when creating a duplicate user. func TestServer_CreateUser_ErrUserExists(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -489,7 +488,7 @@ func TestServer_CreateUser_ErrUserExists(t *testing.T) { // Ensure the server can delete an existing user. func TestServer_DeleteUser(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -516,7 +515,7 @@ func TestServer_DeleteUser(t *testing.T) { // Ensure the server can return a list of all users. func TestServer_Users(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -538,7 +537,7 @@ func TestServer_Users(t *testing.T) { // Ensure the server does not return non-existent users func TestServer_NonExistingUsers(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -561,7 +560,7 @@ func TestServer_NonExistingUsers(t *testing.T) { // Ensure the database can create a new retention policy. func TestServer_CreateRetentionPolicy(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -594,7 +593,7 @@ func TestServer_CreateRetentionPolicy(t *testing.T) { // Ensure the server returns an error when creating a retention policy with an invalid db. func TestServer_CreateRetentionPolicy_ErrDatabaseNotFound(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -605,7 +604,7 @@ func TestServer_CreateRetentionPolicy_ErrDatabaseNotFound(t *testing.T) { // Ensure the server returns an error when creating a retention policy without a name. func TestServer_CreateRetentionPolicy_ErrRetentionPolicyNameRequired(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -617,7 +616,7 @@ func TestServer_CreateRetentionPolicy_ErrRetentionPolicyNameRequired(t *testing. // Ensure the server returns an error when creating a duplicate retention policy. func TestServer_CreateRetentionPolicy_ErrRetentionPolicyExists(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -630,7 +629,7 @@ func TestServer_CreateRetentionPolicy_ErrRetentionPolicyExists(t *testing.T) { // Ensure the database can alter an existing retention policy. func TestServer_AlterRetentionPolicy(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -696,7 +695,7 @@ func TestServer_AlterRetentionPolicy(t *testing.T) { // Ensure the server can delete an existing retention policy. func TestServer_DeleteRetentionPolicy(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -724,7 +723,7 @@ func TestServer_DeleteRetentionPolicy(t *testing.T) { // Ensure the server returns an error when deleting a retention policy on invalid db. func TestServer_DeleteRetentionPolicy_ErrDatabaseNotFound(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -735,7 +734,7 @@ func TestServer_DeleteRetentionPolicy_ErrDatabaseNotFound(t *testing.T) { // Ensure the server returns an error when deleting a retention policy without a name. func TestServer_DeleteRetentionPolicy_ErrRetentionPolicyNameRequired(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -747,7 +746,7 @@ func TestServer_DeleteRetentionPolicy_ErrRetentionPolicyNameRequired(t *testing. // Ensure the server returns an error when deleting a non-existent retention policy. func TestServer_DeleteRetentionPolicy_ErrRetentionPolicyNotFound(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -759,7 +758,7 @@ func TestServer_DeleteRetentionPolicy_ErrRetentionPolicyNotFound(t *testing.T) { // Ensure the server can set the default retention policy func TestServer_SetDefaultRetentionPolicy(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -794,7 +793,7 @@ func TestServer_SetDefaultRetentionPolicy(t *testing.T) { // Ensure the server returns an error when setting the default retention policy to a non-existant one. func TestServer_SetDefaultRetentionPolicy_ErrRetentionPolicyNotFound(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -806,7 +805,7 @@ func TestServer_SetDefaultRetentionPolicy_ErrRetentionPolicyNotFound(t *testing. // Ensure the server prohibits a zero check interval for retention policy enforcement. func TestServer_StartRetentionPolicyEnforcement_ErrZeroInterval(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -816,7 +815,7 @@ func TestServer_StartRetentionPolicyEnforcement_ErrZeroInterval(t *testing.T) { } func TestServer_EnforceRetentionPolices(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -853,7 +852,7 @@ func TestServer_EnforceRetentionPolices(t *testing.T) { // Ensure the database can write data to the database. func TestServer_WriteSeries(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -867,7 +866,6 @@ func TestServer_WriteSeries(t *testing.T) { t.Fatal(err) } c.Sync(index) - warn("A") // Write another point 10 seconds later so it goes through "raw series". index, err = s.WriteSeries("foo", "mypolicy", []influxdb.Point{{Name: "cpu_load", Tags: tags, Timestamp: mustParseTime("2000-01-01T00:00:10Z"), Fields: map[string]interface{}{"value": float64(100)}}}) @@ -875,7 +873,6 @@ func TestServer_WriteSeries(t *testing.T) { t.Fatal(err) } c.Sync(index) - warn("B") // Retrieve first series data point. if v, err := s.ReadSeries("foo", "mypolicy", "cpu_load", tags, mustParseTime("2000-01-01T00:00:00Z")); err != nil { @@ -901,7 +898,7 @@ func TestServer_WriteSeries(t *testing.T) { // Ensure the server can drop a measurement. func TestServer_DropMeasurement(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -964,7 +961,7 @@ func TestServer_DropMeasurement(t *testing.T) { // Ensure the server can handles drop measurement if none exists. func TestServer_DropMeasurementNoneExists(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -1004,7 +1001,7 @@ func TestServer_DropMeasurementNoneExists(t *testing.T) { // select * from memory where host=serverb // select * from memory where region=uswest func TestServer_DropMeasurementSeriesTagsPreserved(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -1118,7 +1115,7 @@ func TestServer_DropMeasurementSeriesTagsPreserved(t *testing.T) { // Ensure the server can drop a series. func TestServer_DropSeries(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -1162,7 +1159,7 @@ func TestServer_DropSeries(t *testing.T) { // Ensure the server can drop a series from measurement when more than one shard exists. func TestServer_DropSeriesFromMeasurement(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -1204,7 +1201,7 @@ func TestServer_DropSeriesFromMeasurement(t *testing.T) { // Ensure that when merging many series together and some of them have a different number of points than others // in a group by interval the results are correct func TestServer_MergeManySeries(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -1239,7 +1236,7 @@ func TestServer_MergeManySeries(t *testing.T) { // ensure that the dropped series is gone // ensure that we can still query: select value from cpu where region=uswest func TestServer_DropSeriesTagsPreserved(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() s := OpenServer(c) defer s.Close() s.CreateDatabase("foo") @@ -1313,7 +1310,7 @@ func TestServer_DropSeriesTagsPreserved(t *testing.T) { // Ensure the server can execute a query and return the data correctly. func TestServer_ExecuteQuery(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1393,7 +1390,7 @@ func TestServer_ExecuteQuery(t *testing.T) { // Ensure the server respects limit and offset in show series queries func TestServer_ShowSeriesLimitOffset(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1449,7 +1446,7 @@ func TestServer_ShowSeriesLimitOffset(t *testing.T) { // Ensure that when querying for raw data values that they return in time order func TestServer_RawDataReturnsInOrder(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1496,7 +1493,7 @@ func TestServer_RawDataReturnsInOrder(t *testing.T) { // Ensure that limit and offset work func TestServer_LimitAndOffset(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1545,7 +1542,7 @@ func TestServer_LimitAndOffset(t *testing.T) { // Ensure the server can execute a wildcard query and return the data correctly. func TestServer_ExecuteWildcardQuery(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1571,7 +1568,7 @@ func TestServer_ExecuteWildcardQuery(t *testing.T) { // Ensure the server can execute a wildcard GROUP BY func TestServer_ExecuteWildcardGroupBy(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1604,7 +1601,7 @@ func TestServer_ExecuteWildcardGroupBy(t *testing.T) { } func TestServer_CreateShardGroupIfNotExist(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1629,7 +1626,7 @@ func TestServer_CreateShardGroupIfNotExist(t *testing.T) { } func TestServer_DeleteShardGroup(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1667,7 +1664,7 @@ func TestServer_DeleteShardGroup(t *testing.T) { /* TODO(benbjohnson): Change test to not expose underlying series ids directly. func TestServer_Measurements(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1738,7 +1735,7 @@ func TestServer_NormalizeMeasurement(t *testing.T) { } // Create server with a variety of databases, retention policies, and measurements - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1787,7 +1784,7 @@ func TestServer_NormalizeQuery(t *testing.T) { } // Start server with database & retention policy. - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1809,7 +1806,7 @@ func TestServer_NormalizeQuery(t *testing.T) { // Ensure the server can create a continuous query func TestServer_CreateContinuousQuery(t *testing.T) { - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -1871,7 +1868,7 @@ func TestServer_CreateContinuousQuery_ErrInfinteLoop(t *testing.T) { // Ensure func TestServer_RunContinuousQueries(t *testing.T) { t.Skip() - c := NewMessagingClient() + c := test.NewMessagingClient() defer c.Close() s := OpenServer(c) defer s.Close() @@ -2019,7 +2016,7 @@ func NewServer() *Server { // OpenServer returns a new, open test server instance. func OpenServer(client influxdb.MessagingClient) *Server { s := OpenUninitializedServer(client) - if err := s.Initialize(&url.URL{Host: "127.0.0.1:8080"}); err != nil { + if err := s.Initialize(url.URL{Host: "127.0.0.1:8080"}); err != nil { panic(err.Error()) } return s @@ -2081,147 +2078,10 @@ func (s *Server) MustWriteSeries(database, retentionPolicy string, points []infl if err != nil { panic(err.Error()) } - s.Client().(*MessagingClient).Sync(index) + s.Client().(*test.MessagingClient).Sync(index) return index } -// MessagingClient represents a test client for the messaging broker. -type MessagingClient struct { - mu sync.Mutex - index uint64 // highest index - conns []*MessagingConn // list of all connections - - messagesByTopicID map[uint64][]*messaging.Message // message by topic - - PublishFunc func(*messaging.Message) (uint64, error) - ConnFunc func(topicID uint64) influxdb.MessagingConn -} - -// NewMessagingClient returns a new instance of MessagingClient. -func NewMessagingClient() *MessagingClient { - c := &MessagingClient{ - messagesByTopicID: make(map[uint64][]*messaging.Message), - } - c.PublishFunc = c.DefaultPublishFunc - c.ConnFunc = c.DefaultConnFunc - return c -} - -// Close closes all open connections. -func (c *MessagingClient) Close() error { - c.mu.Lock() - defer c.mu.Unlock() - - for _, conn := range c.conns { - conn.Close() - } - - return nil -} - -func (c *MessagingClient) Publish(m *messaging.Message) (uint64, error) { return c.PublishFunc(m) } - -// DefaultPublishFunc sets an autoincrementing index on the message and sends it to each topic connection. -func (c *MessagingClient) DefaultPublishFunc(m *messaging.Message) (uint64, error) { - c.mu.Lock() - defer c.mu.Unlock() - - // Increment index and assign it to message. - c.index++ - m.Index = c.index - - // Append message to the topic. - c.messagesByTopicID[m.TopicID] = append(c.messagesByTopicID[m.TopicID], m) - - // Send to each connection for the topic. - for _, conn := range c.conns { - if conn.topicID == m.TopicID { - conn.Send(m) - } - } - - return m.Index, nil -} - -func (c *MessagingClient) Conn(topicID uint64) influxdb.MessagingConn { - return c.ConnFunc(topicID) -} - -// DefaultConnFunc returns a connection for a specific topic. -func (c *MessagingClient) DefaultConnFunc(topicID uint64) influxdb.MessagingConn { - c.mu.Lock() - defer c.mu.Unlock() - - // Create new connection. - conn := NewMessagingConn(topicID) - - // Track connections. - c.conns = append(c.conns, conn) - - return conn -} - -// Sync blocks until a given index has been sent through the client. -func (c *MessagingClient) Sync(index uint64) { - for { - c.mu.Lock() - if c.index >= index { - c.mu.Unlock() - time.Sleep(10 * time.Millisecond) - return - } - c.mu.Unlock() - - // Otherwise wait momentarily and check again. - time.Sleep(1 * time.Millisecond) - } -} - -// MessagingConn represents a mockable connection implementing influxdb.MessagingConn. -type MessagingConn struct { - mu sync.Mutex - topicID uint64 - index uint64 - c chan *messaging.Message -} - -// NewMessagingConn returns a new instance of MessagingConn. -func NewMessagingConn(topicID uint64) *MessagingConn { - return &MessagingConn{ - topicID: topicID, - } -} - -// Open starts the stream from a given index. -func (c *MessagingConn) Open(index uint64) error { - // TODO: Fill connection stream with existing messages. - c.c = make(chan *messaging.Message, 1024) - return nil -} - -// Close closes the streaming channel. -func (c *MessagingConn) Close() error { - close(c.c) - return nil -} - -// C returns a channel for streaming message. -func (c *MessagingConn) C() <-chan *messaging.Message { return c.c } - -func (c *MessagingConn) Send(m *messaging.Message) { - // Ignore any old messages. - c.mu.Lock() - if m.Index <= c.index { - c.mu.Unlock() - return - } - c.index = m.Index - c.mu.Unlock() - - // Send message to channel. - c.c <- m -} - // tempfile returns a temporary path. func tempfile() string { f, _ := ioutil.TempFile("", "influxdb-") diff --git a/shard.go b/shard.go index a8dcd459ce2..fa6635e2faa 100644 --- a/shard.go +++ b/shard.go @@ -100,7 +100,7 @@ func (s *Shard) open(path string, conn MessagingConn) error { } // Open connection. - if err := conn.Open(s.index); err != nil { + if err := conn.Open(s.index, true); err != nil { _ = s.close() return fmt.Errorf("open shard conn: id=%d, idx=%d, err=%s", s.ID, s.index, err) } diff --git a/tx_test.go b/tx_test.go index 4d9815a8c8c..12b8d0d680b 100644 --- a/tx_test.go +++ b/tx_test.go @@ -7,12 +7,15 @@ import ( "github.com/influxdb/influxdb" "github.com/influxdb/influxdb/influxql" + "github.com/influxdb/influxdb/test" ) // Ensure a transaction can retrieve a list of iterators for a simple SELECT statement. func TestTx_CreateIterators(t *testing.T) { t.Skip() - s := OpenDefaultServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + s := OpenDefaultServer(c) defer s.Close() // Write to us-east From 5f6bcf523f952b64c8c7f279a483fe9eb250d0d3 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Wed, 11 Mar 2015 12:00:45 -0600 Subject: [PATCH 10/20] Fix broker integration bugs. --- cmd/influxd/run.go | 68 ++++++++------------------ httpd/handler_test.go | 10 +--- messaging/broker.go | 53 +++++++++++++++++++-- messaging/broker_test.go | 14 +++--- messaging/client.go | 70 +++++++++++++++++++++++---- messaging/client_test.go | 26 ++++++++++ messaging/handler.go | 7 +++ messaging/handler_test.go | 17 ++++++- raft/log.go | 9 +--- server.go | 76 +++++++++++++----------------- server_test.go | 14 ++---- tests/create_write_single_query.sh | 1 + 12 files changed, 226 insertions(+), 139 deletions(-) diff --git a/cmd/influxd/run.go b/cmd/influxd/run.go index 91917fe842e..22981bd1257 100644 --- a/cmd/influxd/run.go +++ b/cmd/influxd/run.go @@ -288,6 +288,19 @@ func joinLog(l *raft.Log, joinURLs []url.URL) { // creates and initializes a server. func openServer(config *Config, b *influxdb.Broker, initServer, initBroker, configExists bool, joinURLs []url.URL, w io.Writer) *influxdb.Server { + // Use broker URL is there is no config and there are no join URLs passed. + clientJoinURLs := joinURLs + if !configExists || len(joinURLs) == 0 { + clientJoinURLs = []url.URL{b.URL()} + } + + // Create messaging client to the brokers. + c := influxdb.NewMessagingClient() + c.SetLogOutput(w) + if err := c.Open(filepath.Join(config.Data.Dir, messagingClientFile), clientJoinURLs); err != nil { + log.Fatalf("messaging client error: %s", err) + } + // Create and open the server. s := influxdb.NewServer() s.SetLogOutput(w) @@ -298,56 +311,27 @@ func openServer(config *Config, b *influxdb.Broker, initServer, initBroker, conf s.ComputeRunsPerInterval = config.ContinuousQuery.ComputeRunsPerInterval s.ComputeNoMoreThan = time.Duration(config.ContinuousQuery.ComputeNoMoreThan) - if err := s.Open(config.Data.Dir); err != nil { + // Open server with data directory and broker client. + if err := s.Open(config.Data.Dir, c); err != nil { log.Fatalf("failed to open data server: %v", err.Error()) } // If the server is uninitialized then initialize or join it. if initServer { if len(joinURLs) == 0 { - initializeServer(config.DataURL(), s, b, w, initBroker) + if initBroker { + if err := s.Initialize(b.URL()); err != nil { + log.Fatalf("server initialization error: %s", err) + } + } } else { joinServer(s, config.DataURL(), joinURLs) } } - if !configExists { - // We are spining up a server that has no config, - // but already has an initialized data directory - joinURLs = []url.URL{b.URL()} - openServerClient(s, joinURLs, w) - } else { - if len(joinURLs) == 0 { - // If a config exists, but no joinUrls are specified, fall back to the broker URL - // TODO: Make sure we have a leader, and then spin up the server - joinURLs = []url.URL{b.URL()} - } - openServerClient(s, joinURLs, w) - } - return s } -// initializes a new server that does not yet have an ID. -func initializeServer(u url.URL, s *influxdb.Server, b *influxdb.Broker, w io.Writer, initBroker bool) { - // Create messaging client. - c := influxdb.NewMessagingClient() - c.SetLogOutput(w) - if err := c.Open(filepath.Join(s.Path(), messagingClientFile), []url.URL{b.URL()}); err != nil { - log.Fatalf("messaging client error: %s", err) - } - if err := s.SetClient(c); err != nil { - log.Fatalf("set client error: %s", err) - } - - if initBroker { - // Initialize the server. - if err := s.Initialize(b.URL()); err != nil { - log.Fatalf("server initialization error: %s", err) - } - } -} - // joins a server to an existing cluster. func joinServer(s *influxdb.Server, u url.URL, joinURLs []url.URL) { // TODO: Use separate broker and data join urls. @@ -364,18 +348,6 @@ func joinServer(s *influxdb.Server, u url.URL, joinURLs []url.URL) { log.Fatalf("join: failed to connect data node to any specified server") } -// opens the messaging client and attaches it to the server. -func openServerClient(s *influxdb.Server, joinURLs []url.URL, w io.Writer) { - c := influxdb.NewMessagingClient() - c.SetLogOutput(w) - if err := c.Open(filepath.Join(s.Path(), messagingClientFile), joinURLs); err != nil { - log.Fatalf("messaging client error: %s", err) - } - if err := s.SetClient(c); err != nil { - log.Fatalf("set client error: %s", err) - } -} - // parses a comma-delimited list of URLs. func parseURLs(s string) (a []url.URL) { if s == "" { diff --git a/httpd/handler_test.go b/httpd/handler_test.go index 941f1c5b78b..2c6eb748968 100644 --- a/httpd/handler_test.go +++ b/httpd/handler_test.go @@ -1559,21 +1559,15 @@ func (s *Server) Restart() { } // Open and reset the client. - if err := s.Server.Open(path); err != nil { + if err := s.Server.Open(path, client); err != nil { panic("open: " + err.Error()) } - if err := s.Server.SetClient(client); err != nil { - panic("client: " + err.Error()) - } } // OpenUninitializedServer returns a new, uninitialized, open test server instance. func OpenUninitializedServer(client influxdb.MessagingClient) *Server { s := NewServer() - if err := s.Open(tempfile()); err != nil { - panic(err.Error()) - } - if err := s.SetClient(client); err != nil { + if err := s.Open(tempfile(), client); err != nil { panic(err.Error()) } return s diff --git a/messaging/broker.go b/messaging/broker.go index c1fc503494a..14f2abf40b4 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -228,9 +228,23 @@ func (b *Broker) closeTopics() { // SetMaxIndex sets the highest index seen by the broker. // This is only used for internal log messages and topics may have a higher index. func (b *Broker) SetMaxIndex(index uint64) error { - return b.meta.Update(func(tx *bolt.Tx) error { + b.mu.Lock() + defer b.mu.Unlock() + return b.setMaxIndex(index) +} + +func (b *Broker) setMaxIndex(index uint64) error { + // Update index in meta database. + if err := b.meta.Update(func(tx *bolt.Tx) error { return tx.Bucket([]byte("meta")).Put([]byte("index"), u64tob(index)) - }) + }); err != nil { + return err + } + + // Set in-memory index. + b.index = index + + return nil } // Snapshot streams the current state of the broker and returns the index. @@ -335,8 +349,8 @@ func (b *Broker) Restore(r io.Reader) error { defer b.mu.Unlock() // Remove and recreate broker path. - if err := os.RemoveAll(b.path); err != nil && !os.IsNotExist(err) { - return fmt.Errorf("remove all: %s", err) + if err := b.reset(); err != nil && !os.IsNotExist(err) { + return fmt.Errorf("reset: %s", err) } else if err = os.MkdirAll(b.path, 0700); err != nil { return fmt.Errorf("mkdir: %s", err) } @@ -400,7 +414,7 @@ func (b *Broker) Restore(r io.Reader) error { } // Set the highest seen index. - if err := b.SetMaxIndex(sh.Index); err != nil { + if err := b.setMaxIndex(sh.Index); err != nil { return fmt.Errorf("set max index: %s", err) } b.index = sh.Index @@ -408,6 +422,35 @@ func (b *Broker) Restore(r io.Reader) error { return nil } +// reset removes all files in the broker directory besides the raft directory. +func (b *Broker) reset() error { + // Open handle to directory. + f, err := os.Open(b.path) + if err != nil { + return err + } + defer func() { _ = f.Close() }() + + // Read directory items. + fis, err := f.Readdir(0) + if err != nil { + return err + } + + // Remove all files & directories besides raft. + for _, fi := range fis { + if fi.Name() == "raft" { + continue + } + + if err := os.RemoveAll(fi.Name()); err != nil { + return fmt.Errorf("remove: %s", fi.Name()) + } + } + + return nil +} + // Publish writes a message. // Returns the index of the message. Otherwise returns an error. func (b *Broker) Publish(m *Message) (uint64, error) { diff --git a/messaging/broker_test.go b/messaging/broker_test.go index 223b0dc62a9..edb03eec89e 100644 --- a/messaging/broker_test.go +++ b/messaging/broker_test.go @@ -524,8 +524,6 @@ func TestTopicReader_streaming(t *testing.T) { var wg sync.WaitGroup wg.Add(1) go func() { - defer wg.Done() - time.Sleep(2 * time.Millisecond) MustWriteFile(filepath.Join(path, "6"), MustMarshalMessages([]*messaging.Message{ @@ -549,10 +547,6 @@ func TestTopicReader_streaming(t *testing.T) { {Index: 14}, }), ) - - // Close reader. - time.Sleep(5 * time.Millisecond) - r.Close() }() // Slurp all message ids from the reader. @@ -561,12 +555,16 @@ func TestTopicReader_streaming(t *testing.T) { for { m := &messaging.Message{} if err := dec.Decode(m); err == io.EOF { - break + t.Fatalf("unexpected EOF") } else if err != nil { t.Fatalf("decode error: %s", err) } else { indices = append(indices, m.Index) } + + if m.Index == 14 { + break + } } // Verify we received the correct indices. @@ -574,7 +572,7 @@ func TestTopicReader_streaming(t *testing.T) { t.Fatalf("unexpected indices: %#v", indices) } - wg.Wait() + r.Close() } // Ensure multiple topic readers can read from the same topic directory. diff --git a/messaging/client.go b/messaging/client.go index 4904f0f2480..2cf9423c4e9 100644 --- a/messaging/client.go +++ b/messaging/client.go @@ -19,7 +19,10 @@ import ( // DefaultReconnectTimeout is the default time to wait between when a broker // stream disconnects and another connection is retried. -const DefaultReconnectTimeout = 100 * time.Millisecond +const DefaultReconnectTimeout = 1000 * time.Millisecond + +// DefaultPingInterval is the default time to wait between checks to the broker. +const DefaultPingInterval = 1000 * time.Millisecond // Client represents a client for the broker's HTTP API. type Client struct { @@ -29,11 +32,16 @@ type Client struct { urls []url.URL // list of available broker URLs opened bool - done chan chan struct{} // disconnection notification + + wg sync.WaitGroup + closing chan struct{} // The amount of time to wait before reconnecting to a broker stream. ReconnectTimeout time.Duration + // The amount of time between pings to verify the broker is alive. + PingInterval time.Duration + // The logging interface used by the client for out-of-band errors. Logger *log.Logger } @@ -42,6 +50,7 @@ type Client struct { func NewClient() *Client { c := &Client{ ReconnectTimeout: DefaultReconnectTimeout, + PingInterval: DefaultPingInterval, } c.SetLogOutput(os.Stderr) return c @@ -130,6 +139,11 @@ func (c *Client) Open(path string, urls []url.URL) error { // Set open flag. c.opened = true + // Start background ping. + c.closing = make(chan struct{}, 0) + c.wg.Add(1) + go c.pinger(c.closing) + return nil } @@ -149,6 +163,17 @@ func (c *Client) Close() error { } c.conns = nil + // Close goroutines. + if c.closing != nil { + close(c.closing) + c.closing = nil + } + + // Wait for goroutines to finish. + c.mu.Unlock() + c.wg.Wait() + c.mu.Lock() + // Unset open flag. c.opened = false @@ -168,6 +193,14 @@ func (c *Client) Publish(m *Message) (uint64, error) { } defer func() { _ = resp.Body.Close() }() + // Check response code. + if resp.StatusCode != http.StatusOK { + if errstr := resp.Header.Get("X-Broker-Error"); errstr != "" { + return 0, errors.New(errstr) + } + return 0, fmt.Errorf("cannot publish(%d)", resp.StatusCode) + } + // Parse broker index. index, err := strconv.ParseUint(resp.Header.Get("X-Broker-Index"), 10, 64) if err != nil { @@ -177,6 +210,18 @@ func (c *Client) Publish(m *Message) (uint64, error) { return index, nil } +// Ping sends a request to the current broker to check if it is alive. +// If the broker is down then a new URL is tried. +func (c *Client) Ping() error { + // Post message to broker. + resp, err := c.do("POST", "/messaging/ping", nil, "application/octet-stream", nil) + if err != nil { + return err + } + resp.Body.Close() + return nil +} + // do sends an HTTP request to the given path with the current leader URL. // This will automatically retry the request if it is redirected. func (c *Client) do(method, path string, values url.Values, contentType string, body io.Reader) (*http.Response, error) { @@ -210,12 +255,6 @@ func (c *Client) do(method, path string, values url.Values, contentType string, } c.SetURL(*redirectURL) continue - } else if resp.StatusCode != http.StatusOK { - resp.Body.Close() - if errstr := resp.Header.Get("X-Broker-Error"); errstr != "" { - return nil, errors.New(errstr) - } - return nil, fmt.Errorf("cannot publish(%d)", resp.StatusCode) } return resp, nil @@ -238,6 +277,20 @@ func (c *Client) Conn(topicID uint64) *Conn { return conn } +// pinger periodically pings the broker to check that it is alive. +func (c *Client) pinger(closing chan struct{}) { + defer c.wg.Done() + + for { + select { + case <-closing: + return + case <-time.After(c.PingInterval): + c.Ping() + } + } +} + // ClientConfig represents the configuration that must be persisted across restarts. type ClientConfig struct { Brokers []url.URL `json:"brokers"` @@ -490,7 +543,6 @@ func (c *Conn) stream(req *http.Request, closing <-chan struct{}) error { // Decode message from the stream. m := &Message{} if err := dec.Decode(m); err == io.EOF { - warn("EOF!!!") return nil } else if err != nil { return fmt.Errorf("decode: %s", err) diff --git a/messaging/client_test.go b/messaging/client_test.go index 09e21686ccd..613fc76db30 100644 --- a/messaging/client_test.go +++ b/messaging/client_test.go @@ -11,6 +11,32 @@ import ( "github.com/influxdb/influxdb/messaging" ) +// Ensure a client can check if the server is alive. +func TestClient_Ping(t *testing.T) { + var pinged bool + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + if req.URL.Path != "/messaging/ping" { + t.Fatalf("unexpected path: %s", req.URL.Path) + } + pinged = true + })) + defer s.Close() + + // Create client. + c := messaging.NewClient() + if err := c.Open("", []url.URL{*MustParseURL(s.URL)}); err != nil { + t.Fatal(err) + } + defer c.Close() + + // Ping server. + if err := c.Ping(); err != nil { + t.Fatal(err) + } else if !pinged { + t.Fatal("ping not received") + } +} + // Ensure a client can be opened and connections can be created. func TestClient_Conn(t *testing.T) { s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { diff --git a/messaging/handler.go b/messaging/handler.go index 9c7bef5186e..61e71136858 100644 --- a/messaging/handler.go +++ b/messaging/handler.go @@ -48,6 +48,8 @@ func (h *Handler) ServeHTTP(w http.ResponseWriter, r *http.Request) { } else { http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed) } + case "/messaging/ping": + h.servePing(w, r) default: http.NotFound(w, r) } @@ -161,6 +163,11 @@ func (h *Handler) postHeartbeat(w http.ResponseWriter, r *http.Request) { } } +// servePing returns a status 200. +func (h *Handler) servePing(w http.ResponseWriter, r *http.Request) { + w.WriteHeader(http.StatusOK) +} + // error writes an error to the client and sets the status code. func (h *Handler) error(w http.ResponseWriter, err error, code int) { s := err.Error() diff --git a/messaging/handler_test.go b/messaging/handler_test.go index 328d717eda1..01ee316d388 100644 --- a/messaging/handler_test.go +++ b/messaging/handler_test.go @@ -182,7 +182,7 @@ func TestHandler_postHeartbeat(t *testing.T) { } // Ensure an error is returned when heartbeating with the wrong HTTP method. -func heartbeat_ErrMethodNotAllowed(t *testing.T) { +func TestHandler_postHeartbeat_ErrMethodNotAllowed(t *testing.T) { s := httptest.NewServer(&messaging.Handler{}) defer s.Close() @@ -195,6 +195,21 @@ func heartbeat_ErrMethodNotAllowed(t *testing.T) { resp.Body.Close() } +// Ensure a handler can respond to a ping. +func TestHandler_servePing(t *testing.T) { + s := httptest.NewServer(&messaging.Handler{}) + defer s.Close() + + // Send request to the broker. + resp, err := http.Post(s.URL+`/messaging/ping`, "application/octet-stream", nil) + if err != nil { + t.Fatal(err) + } else if resp.StatusCode != http.StatusOK { + t.Fatalf("unexpected status: %d: %s", resp.StatusCode, resp.Header.Get("X-Broker-Error")) + } + resp.Body.Close() +} + // Ensure the handler routes raft requests to the raft handler. func TestHandler_raft(t *testing.T) { var h messaging.Handler diff --git a/raft/log.go b/raft/log.go index b6dd75f9dbb..1d7c14b46c1 100644 --- a/raft/log.go +++ b/raft/log.go @@ -1554,18 +1554,11 @@ func (l *Log) advanceWriter(writer *logWriter, snapshotIndex uint64) error { default: } - // Determine the highest snapshot index. The writer's snapshot index can - // be higher if non-command entries have been applied. - if writer.snapshotIndex > snapshotIndex { - snapshotIndex = writer.snapshotIndex - } - snapshotIndex++ - // Write pending entries. if len(l.entries) > 0 { startIndex := l.entries[0].Index enc := NewLogEntryEncoder(writer.Writer) - for _, e := range l.entries[snapshotIndex-startIndex:] { + for _, e := range l.entries[snapshotIndex-startIndex+1:] { if err := enc.Encode(e); err != nil { return err } diff --git a/server.go b/server.go index 1a9e29b2f80..9e74fb68524 100644 --- a/server.go +++ b/server.go @@ -163,7 +163,7 @@ func (s *Server) SetLogOutput(w io.Writer) { } // Open initializes the server from a given path. -func (s *Server) Open(path string) error { +func (s *Server) Open(path string, client MessagingClient) error { // Ensure the server isn't already open and there's a path provided. if s.opened() { return ErrServerOpen @@ -176,23 +176,41 @@ func (s *Server) Open(path string) error { // Create required directories. if err := os.MkdirAll(path, 0755); err != nil { + _ = s.close() return err } if err := os.MkdirAll(filepath.Join(path, "shards"), 0755); err != nil { + _ = s.close() return err } + // Set the messaging client. + s.client = client + // Open metadata store. if err := s.meta.open(s.metaPath()); err != nil { + _ = s.close() return fmt.Errorf("meta: %s", err) } // Load state from metastore. if err := s.load(); err != nil { + _ = s.close() return fmt.Errorf("load: %s", err) } - // TODO: Open shard data stores. + // Create connection for broadcast topic. + conn := client.Conn(BroadcastTopicID) + if err := conn.Open(s.index, true); err != nil { + _ = s.close() + return fmt.Errorf("open conn: %s", err) + } + + // Begin streaming messages from broadcast topic. + done := make(chan struct{}, 0) + s.done = done + go s.processor(conn, done) + // TODO: Associate series ids with shards. return nil @@ -205,25 +223,36 @@ func (s *Server) opened() bool { return s.path != "" } func (s *Server) Close() error { s.mu.Lock() defer s.mu.Unlock() + return s.close() +} +func (s *Server) close() error { if !s.opened() { return ErrServerClosed } if s.rpDone != nil { close(s.rpDone) + s.rpDone = nil } if s.sgpcDone != nil { close(s.sgpcDone) + s.sgpcDone = nil } // Remove path. s.path = "" s.index = 0 - // Close message processing. - s.setClient(nil) + // Stop broadcast topic processing. + if s.done != nil { + close(s.done) + s.done = nil + } + + // Remove client. + s.client = nil // Close metastore. _ = s.meta.close() @@ -425,45 +454,6 @@ func (s *Server) Client() MessagingClient { return s.client } -// SetClient sets the messaging client on the server. -func (s *Server) SetClient(client MessagingClient) error { - s.mu.Lock() - defer s.mu.Unlock() - return s.setClient(client) -} - -func (s *Server) setClient(client MessagingClient) error { - // Ensure the server is open. - if !s.opened() { - return ErrServerClosed - } - - // Stop previous processor, if running. - if s.done != nil { - close(s.done) - s.done = nil - } - - // Set the messaging client. - s.client = client - - // Start goroutine to read messages from the broadcast channel. - if client != nil { - // Create connection for broadcast channel. - conn := client.Conn(BroadcastTopicID) - if err := conn.Open(s.index, true); err != nil { - return fmt.Errorf("open conn: %s", err) - } - - // Stream messages - done := make(chan struct{}, 0) - s.done = done - go s.processor(conn, done) - } - - return nil -} - // broadcast encodes a message as JSON and send it to the broker's broadcast topic. // This function waits until the message has been processed by the server. // Returns the broker log index of the message or an error. diff --git a/server_test.go b/server_test.go index a356718dfbd..40de79501b4 100644 --- a/server_test.go +++ b/server_test.go @@ -21,9 +21,11 @@ import ( // Ensure the server can be successfully opened and closed. func TestServer_Open(t *testing.T) { + c := test.NewMessagingClient() + defer c.Close() s := NewServer() defer s.Close() - if err := s.Server.Open(tempfile()); err != nil { + if err := s.Server.Open(tempfile(), c); err != nil { t.Fatal(err) } if err := s.Server.Close(); err != nil { @@ -2170,10 +2172,7 @@ func OpenServer(client influxdb.MessagingClient) *Server { // OpenUninitializedServer returns a new, uninitialized, open test server instance. func OpenUninitializedServer(client influxdb.MessagingClient) *Server { s := NewServer() - if err := s.Open(tempfile()); err != nil { - panic(err.Error()) - } - if err := s.SetClient(client); err != nil { + if err := s.Open(tempfile(), client); err != nil { panic(err.Error()) } return s @@ -2202,12 +2201,9 @@ func (s *Server) Restart() { } // Open and reset the client. - if err := s.Server.Open(path); err != nil { + if err := s.Server.Open(path, client); err != nil { panic("open: " + err.Error()) } - if err := s.Server.SetClient(client); err != nil { - panic("client: " + err.Error()) - } } // Close shuts down the server and removes all temporary files. diff --git a/tests/create_write_single_query.sh b/tests/create_write_single_query.sh index cc719f9f462..bf8aae60274 100755 --- a/tests/create_write_single_query.sh +++ b/tests/create_write_single_query.sh @@ -9,3 +9,4 @@ curl -d '{"database" : "foo", "retentionPolicy" : "bar", "points": [{"name": "cp echo "querying data" curl -G http://localhost:8086/query --data-urlencode "db=foo" --data-urlencode "q=SELECT sum(value) FROM \"foo\".\"bar\".cpu GROUP BY time(1h)" + From 7880bc24522cfb090bfafb1bac0e12afd179a683 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Thu, 12 Mar 2015 12:12:26 -0600 Subject: [PATCH 11/20] Add zero length data checks. --- cmd/influxd/server_integration_test.go | 1 - messaging/client.go | 5 +++++ messaging/errors.go | 3 +++ messaging/handler.go | 4 ++++ server.go | 2 ++ server_test.go | 8 ++++++-- 6 files changed, 20 insertions(+), 3 deletions(-) diff --git a/cmd/influxd/server_integration_test.go b/cmd/influxd/server_integration_test.go index 632b7a749be..f3ba0edfce3 100644 --- a/cmd/influxd/server_integration_test.go +++ b/cmd/influxd/server_integration_test.go @@ -841,7 +841,6 @@ func TestSingleServer(t *testing.T) { } func Test3NodeServer(t *testing.T) { - t.Skip() testName := "3-node server integration" if testing.Short() { t.Skip(fmt.Sprintf("skipping '%s'", testName)) diff --git a/messaging/client.go b/messaging/client.go index 2cf9423c4e9..f03002cc857 100644 --- a/messaging/client.go +++ b/messaging/client.go @@ -548,6 +548,11 @@ func (c *Conn) stream(req *http.Request, closing <-chan struct{}) error { return fmt.Errorf("decode: %s", err) } + // Panic if we received no data. + if len(m.Data) == 0 { + panic("messaging conn no data recv") + } + // TODO: Write broker set updates, do not passthrough to channel. // Write message to streaming channel. diff --git a/messaging/errors.go b/messaging/errors.go index 4dc1927679a..35462b99e13 100644 --- a/messaging/errors.go +++ b/messaging/errors.go @@ -72,4 +72,7 @@ var ( // ErrReaderClosed is returned when reading from a closed topic reader. ErrReaderClosed = errors.New("reader closed") + + // ErrMessageDataRequired is returned when publishing a message without data. + ErrMessageDataRequired = errors.New("message data required") ) diff --git a/messaging/handler.go b/messaging/handler.go index 61e71136858..51a1492356a 100644 --- a/messaging/handler.go +++ b/messaging/handler.go @@ -116,10 +116,14 @@ func (h *Handler) postMessages(w http.ResponseWriter, r *http.Request) { } // Read the request body. + // Exit if there is no message data provided. data, err := ioutil.ReadAll(r.Body) if err != nil { h.error(w, err, http.StatusInternalServerError) return + } else if len(data) == 0 { + h.error(w, ErrMessageDataRequired, http.StatusBadRequest) + return } // Publish message to the broker. diff --git a/server.go b/server.go index 70826e5a56e..51bd26d6829 100644 --- a/server.go +++ b/server.go @@ -1602,6 +1602,8 @@ func (s *Server) WriteSeries(database, retentionPolicy string, points []Point) ( var err error var maxIndex uint64 for i, d := range shardData { + assert(len(d) > 0, "raw series data required: topic=%d", i) + index, err := s.client.Publish(&messaging.Message{ Type: writeRawSeriesMessageType, TopicID: i, diff --git a/server_test.go b/server_test.go index f97857a0352..bb62961f9c6 100644 --- a/server_test.go +++ b/server_test.go @@ -596,7 +596,9 @@ func TestServer_CreateRetentionPolicy(t *testing.T) { // Ensure the database can create a new retention policy with infinite duration. func TestServer_CreateRetentionPolicyInfinite(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() // Create a database. @@ -629,7 +631,9 @@ func TestServer_CreateRetentionPolicyInfinite(t *testing.T) { // Ensure the database can creates a default retention policy. func TestServer_CreateRetentionPolicyDefault(t *testing.T) { - s := OpenServer(NewMessagingClient()) + c := test.NewMessagingClient() + defer c.Close() + s := OpenServer(c) defer s.Close() s.RetentionAutoCreate = true From c7d49209e10705d0bfffd963c36c9d1d3afc77eb Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Thu, 12 Mar 2015 15:04:36 -0600 Subject: [PATCH 12/20] Update urlgen to end at current time. --- tests/siege/README.md | 2 +- tests/urlgen/urlgen.go | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/siege/README.md b/tests/siege/README.md index ce6c91ca5b3..3ef158d685b 100644 --- a/tests/siege/README.md +++ b/tests/siege/README.md @@ -17,7 +17,7 @@ You can do this with the following commands: ```sh $ curl -G http://localhost:8086/query --data-urlencode "q=CREATE DATABASE db" -$ curl -G http://localhost:8086/query --data-urlencode "q=CREATE RETENTION POLICY raw ON db DURATION 1h REPLICATION 1 DEFAULT" +$ curl -G http://localhost:8086/query --data-urlencode "q=CREATE RETENTION POLICY raw ON db DURATION 30d REPLICATION 3 DEFAULT" ``` diff --git a/tests/urlgen/urlgen.go b/tests/urlgen/urlgen.go index a166747e577..1baf02c221e 100644 --- a/tests/urlgen/urlgen.go +++ b/tests/urlgen/urlgen.go @@ -11,17 +11,17 @@ func main() { intervalN := flag.Int("interval", 10, "interval") seriesN := flag.Int("series", 1, "Number of unique series to generate.") clientN := flag.Int("clients", 10, "Number of clients to simulate.") - startDate := flag.String("start", time.Now().Format(time.RFC3339), "Date to start with.") flag.Parse() - t, _ := time.Parse(time.RFC3339, *startDate) - oneSecond := 1 * time.Second + // Calculate time so that the last point ends now. + n := (*clientN) * (*seriesN) * (*intervalN) + t := time.Now().UTC().Add(-time.Duration(n) * time.Second) for i := 0; i < *clientN; i++ { for j := 0; j < *seriesN; j++ { points := make([]*Point, 0) for k := 0; k < *intervalN; k++ { - t = t.Add(oneSecond) + t = t.Add(1 * time.Second) points = append(points, &Point{ Name: "cpu", Timestamp: t, From 12e8939647b5f91f4115fb73b38160149f239b9f Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Thu, 12 Mar 2015 15:34:20 -0600 Subject: [PATCH 13/20] Fix messaging client redirection. --- messaging/client.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/messaging/client.go b/messaging/client.go index f03002cc857..e04a5b4f3b2 100644 --- a/messaging/client.go +++ b/messaging/client.go @@ -187,7 +187,7 @@ func (c *Client) Publish(m *Message) (uint64, error) { "type": {strconv.FormatUint(uint64(m.Type), 10)}, "topicID": {strconv.FormatUint(m.TopicID, 10)}, } - resp, err := c.do("POST", "/messaging/messages", values, "application/octet-stream", bytes.NewReader(m.Data)) + resp, err := c.do("POST", "/messaging/messages", values, "application/octet-stream", m.Data) if err != nil { return 0, fmt.Errorf("do: %s", err) } @@ -224,7 +224,7 @@ func (c *Client) Ping() error { // do sends an HTTP request to the given path with the current leader URL. // This will automatically retry the request if it is redirected. -func (c *Client) do(method, path string, values url.Values, contentType string, body io.Reader) (*http.Response, error) { +func (c *Client) do(method, path string, values url.Values, contentType string, body []byte) (*http.Response, error) { for { // Generate URL. u := c.URL() @@ -232,7 +232,7 @@ func (c *Client) do(method, path string, values url.Values, contentType string, u.RawQuery = values.Encode() // Create request. - req, err := http.NewRequest(method, u.String(), body) + req, err := http.NewRequest(method, u.String(), bytes.NewReader(body)) if err != nil { return nil, fmt.Errorf("new request: %s", err) } From fc189cd2ae8ce2e49dc7f00584434425574e19b1 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Thu, 12 Mar 2015 17:15:41 -0600 Subject: [PATCH 14/20] Remove /test from .gitignore --- .gitignore | 1 - test/messaging.go | 152 ++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 152 insertions(+), 1 deletion(-) create mode 100644 test/messaging.go diff --git a/.gitignore b/.gitignore index e3b88aa02a1..6500c1cd0ff 100644 --- a/.gitignore +++ b/.gitignore @@ -2,7 +2,6 @@ src/ config.json -/test /bin/ /pkg/ diff --git a/test/messaging.go b/test/messaging.go new file mode 100644 index 00000000000..ae0e9d852a0 --- /dev/null +++ b/test/messaging.go @@ -0,0 +1,152 @@ +package test + +import ( + "io" + "net/url" + "sync" + "time" + + "github.com/influxdb/influxdb" + "github.com/influxdb/influxdb/messaging" +) + +// MessagingClient represents a test client for the messaging broker. +type MessagingClient struct { + mu sync.Mutex + index uint64 // highest index + conns []*MessagingConn // list of all connections + + messagesByTopicID map[uint64][]*messaging.Message // message by topic + + PublishFunc func(*messaging.Message) (uint64, error) + ConnFunc func(topicID uint64) influxdb.MessagingConn +} + +// NewMessagingClient returns a new instance of MessagingClient. +func NewMessagingClient() *MessagingClient { + c := &MessagingClient{ + messagesByTopicID: make(map[uint64][]*messaging.Message), + } + c.PublishFunc = c.DefaultPublishFunc + c.ConnFunc = c.DefaultConnFunc + return c +} + +func (c *MessagingClient) Open(path string, urls []url.URL) error { return nil } + +// Close closes all open connections. +func (c *MessagingClient) Close() error { + c.mu.Lock() + defer c.mu.Unlock() + + for _, conn := range c.conns { + conn.Close() + } + + return nil +} + +func (c *MessagingClient) Publish(m *messaging.Message) (uint64, error) { return c.PublishFunc(m) } + +// DefaultPublishFunc sets an autoincrementing index on the message and sends it to each topic connection. +func (c *MessagingClient) DefaultPublishFunc(m *messaging.Message) (uint64, error) { + c.mu.Lock() + defer c.mu.Unlock() + + // Increment index and assign it to message. + c.index++ + m.Index = c.index + + // Append message to the topic. + c.messagesByTopicID[m.TopicID] = append(c.messagesByTopicID[m.TopicID], m) + + // Send to each connection for the topic. + for _, conn := range c.conns { + if conn.topicID == m.TopicID { + conn.Send(m) + } + } + + return m.Index, nil +} + +func (c *MessagingClient) Conn(topicID uint64) influxdb.MessagingConn { + return c.ConnFunc(topicID) +} + +// DefaultConnFunc returns a connection for a specific topic. +func (c *MessagingClient) DefaultConnFunc(topicID uint64) influxdb.MessagingConn { + c.mu.Lock() + defer c.mu.Unlock() + + // Create new connection. + conn := NewMessagingConn(topicID) + + // Track connections. + c.conns = append(c.conns, conn) + + return conn +} + +// Sync blocks until a given index has been sent through the client. +func (c *MessagingClient) Sync(index uint64) { + for { + c.mu.Lock() + if c.index >= index { + c.mu.Unlock() + time.Sleep(10 * time.Millisecond) + return + } + c.mu.Unlock() + + // Otherwise wait momentarily and check again. + time.Sleep(1 * time.Millisecond) + } +} + +func (c *MessagingClient) SetLogOutput(_ io.Writer) {} + +// MessagingConn represents a mockable connection implementing influxdb.MessagingConn. +type MessagingConn struct { + mu sync.Mutex + topicID uint64 + index uint64 + c chan *messaging.Message +} + +// NewMessagingConn returns a new instance of MessagingConn. +func NewMessagingConn(topicID uint64) *MessagingConn { + return &MessagingConn{ + topicID: topicID, + } +} + +// Open starts the stream from a given index. +func (c *MessagingConn) Open(index uint64, streaming bool) error { + // TODO: Fill connection stream with existing messages. + c.c = make(chan *messaging.Message, 1024) + return nil +} + +// Close closes the streaming channel. +func (c *MessagingConn) Close() error { + close(c.c) + return nil +} + +// C returns a channel for streaming message. +func (c *MessagingConn) C() <-chan *messaging.Message { return c.c } + +func (c *MessagingConn) Send(m *messaging.Message) { + // Ignore any old messages. + c.mu.Lock() + if m.Index <= c.index { + c.mu.Unlock() + return + } + c.index = m.Index + c.mu.Unlock() + + // Send message to channel. + c.c <- m +} From 8e813ec351b6874ec18408a06180ddcfc1942c2c Mon Sep 17 00:00:00 2001 From: Todd Persen Date: Fri, 13 Mar 2015 10:09:37 -0700 Subject: [PATCH 15/20] Update CHANGELOG.md for v0.9.0-rc11 --- CHANGELOG.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e4e9b50956d..1bc951197e3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,6 @@ -## v0.9.0-rc11 [unreleased] +## v0.9.0-rc12 [unreleased] + +## v0.9.0-rc11 [2015-03-12] ### Bugfixes - [#1917](https://github.com/influxdb/influxdb/pull/1902): Creating Infinite Retention Policy Failed. From 53dbec82320318458ee91b2f474bdc11e4313814 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 14 Mar 2015 13:36:06 -0600 Subject: [PATCH 16/20] Add config notifications and increased test coverage. --- cmd/influxd/run.go | 12 +- messaging/broker.go | 8 + messaging/broker_test.go | 4 + messaging/client.go | 189 +++++++++++--- messaging/client_test.go | 536 ++++++++++++++++++++++++++++++++++++-- messaging/errors.go | 3 - messaging/handler.go | 17 ++ messaging/handler_test.go | 42 ++- raft/log.go | 46 +++- server.go | 6 +- test/messaging.go | 5 +- 11 files changed, 789 insertions(+), 79 deletions(-) diff --git a/cmd/influxd/run.go b/cmd/influxd/run.go index 8b87d5416bf..bb01e416a31 100644 --- a/cmd/influxd/run.go +++ b/cmd/influxd/run.go @@ -300,10 +300,20 @@ func openServer(config *Config, b *influxdb.Broker, initServer, initBroker, conf // Create messaging client to the brokers. c := influxdb.NewMessagingClient() c.SetLogOutput(w) - if err := c.Open(filepath.Join(config.Data.Dir, messagingClientFile), clientJoinURLs); err != nil { + if err := c.Open(filepath.Join(config.Data.Dir, messagingClientFile)); err != nil { log.Fatalf("messaging client error: %s", err) } + // If join URLs were passed in then use them to override the client's URLs. + if len(clientJoinURLs) > 0 { + c.SetURLs(clientJoinURLs) + } + + // If no URLs exist on the client the return an error since we cannot reach a broker. + if len(c.URLs()) == 0 { + log.Fatal("messaging client has no broker URLs") + } + // Create and open the server. s := influxdb.NewServer() s.SetLogOutput(w) diff --git a/messaging/broker.go b/messaging/broker.go index 14f2abf40b4..e584d53a1df 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -36,7 +36,9 @@ type Broker struct { // Log is the distributed raft log that commands are applied to. Log interface { URL() url.URL + URLs() []url.URL Leader() (uint64, url.URL) + IsLeader() bool ClusterID() uint64 Apply(data []byte) (index uint64, err error) } @@ -68,6 +70,12 @@ func (b *Broker) metaPath() string { // URL returns the URL of the broker. func (b *Broker) URL() url.URL { return b.Log.URL() } +// URLs returns a list of all broker URLs in the cluster. +func (b *Broker) URLs() []url.URL { return b.Log.URLs() } + +// IsLeader returns true if the broker is the current cluster leader. +func (b *Broker) IsLeader() bool { return b.Log.IsLeader() } + // LeaderURL returns the URL to the leader broker. func (b *Broker) LeaderURL() url.URL { _, u := b.Log.Leader() diff --git a/messaging/broker_test.go b/messaging/broker_test.go index edb03eec89e..80a15f97ff3 100644 --- a/messaging/broker_test.go +++ b/messaging/broker_test.go @@ -707,14 +707,18 @@ func (b *Broker) MustReadAllTopic(topicID uint64) (a []*messaging.Message) { type BrokerLog struct { ApplyFunc func(data []byte) (uint64, error) ClusterIDFunc func() uint64 + IsLeaderFunc func() bool LeaderFunc func() (uint64, url.URL) URLFunc func() url.URL + URLsFunc func() []url.URL } func (l *BrokerLog) Apply(data []byte) (uint64, error) { return l.ApplyFunc(data) } func (l *BrokerLog) ClusterID() uint64 { return l.ClusterIDFunc() } +func (l *BrokerLog) IsLeader() bool { return l.IsLeaderFunc() } func (l *BrokerLog) Leader() (uint64, url.URL) { return l.LeaderFunc() } func (l *BrokerLog) URL() url.URL { return l.URLFunc() } +func (l *BrokerLog) URLs() []url.URL { return l.URLsFunc() } // Messages represents a collection of messages. // This type provides helper functions. diff --git a/messaging/client.go b/messaging/client.go index e04a5b4f3b2..b9f64a2ff61 100644 --- a/messaging/client.go +++ b/messaging/client.go @@ -17,17 +17,20 @@ import ( "time" ) -// DefaultReconnectTimeout is the default time to wait between when a broker -// stream disconnects and another connection is retried. -const DefaultReconnectTimeout = 1000 * time.Millisecond +const ( + // DefaultReconnectTimeout is the default time to wait between when a broker + // stream disconnects and another connection is retried. + DefaultReconnectTimeout = 1000 * time.Millisecond -// DefaultPingInterval is the default time to wait between checks to the broker. -const DefaultPingInterval = 1000 * time.Millisecond + // DefaultPingInterval is the default time to wait between checks to the broker. + DefaultPingInterval = 1000 * time.Millisecond +) // Client represents a client for the broker's HTTP API. type Client struct { mu sync.Mutex - conns []*Conn + path string // config file path + conns []*Conn // all connections opened by client url url.URL // current known leader URL urls []url.URL // list of available broker URLs @@ -80,10 +83,27 @@ func (c *Client) setURL(u url.URL) { } } -// RandomizeURL sets a random URL from the configuration. -func (c *Client) RandomizeURL() { +// URLs returns a list of possible broker URLs to connect to. +func (c *Client) URLs() []url.URL { + c.mu.Lock() + defer c.mu.Unlock() + return c.urls +} + +// SetURLs sets a list of possible URLs to connect to for the client and its connections. +func (c *Client) SetURLs(a []url.URL) { c.mu.Lock() defer c.mu.Unlock() + c.setURLs(a) +} + +func (c *Client) setURLs(a []url.URL) { + // Ignore if the URL list is the same. + if urlsEqual(c.urls, a) { + return + } + + c.urls = a c.randomizeURL() } @@ -102,8 +122,8 @@ func (c *Client) SetLogOutput(w io.Writer) { c.Logger = log.New(w, "[messaging] ", log.LstdFlags) } -// Open reads the configuration from the specified path or uses the URLs provided. -func (c *Client) Open(path string, urls []url.URL) error { +// Open opens the client and reads the configuration from the specified path. +func (c *Client) Open(path string) error { c.mu.Lock() defer c.mu.Unlock() @@ -113,29 +133,12 @@ func (c *Client) Open(path string, urls []url.URL) error { } // Read URLs from file if no URLs are provided. - if len(urls) == 0 { - if b, err := ioutil.ReadFile(path); os.IsNotExist(err) { - // nop - } else if err != nil { - return err - } else { - var config ClientConfig - if err := json.Unmarshal(b, &config); err != nil { - return err - } - c.urls = config.Brokers - } - } - - // Ensure we have at least one URL. - if len(urls) < 1 { - return ErrBrokerURLRequired + c.path = path + if err := c.loadConfig(); err != nil { + _ = c.close() + return fmt.Errorf("load config: %s", err) } - // Set the URLs whether they're from the config or passed in. - c.urls = urls - c.randomizeURL() - // Set open flag. c.opened = true @@ -151,7 +154,10 @@ func (c *Client) Open(path string, urls []url.URL) error { func (c *Client) Close() error { c.mu.Lock() defer c.mu.Unlock() + return c.close() +} +func (c *Client) close() error { // Return error if the client is already closed. if !c.opened { return ErrClientClosed @@ -180,6 +186,53 @@ func (c *Client) Close() error { return nil } +// loadConfig reads the configuration from disk and sets the options on the client. +func (c *Client) loadConfig() error { + // Open config file for reading. + f, err := os.Open(c.path) + if os.IsNotExist(err) { + c.urls = nil + return nil + } else if err != nil { + return fmt.Errorf("open config: %s", err) + } + defer f.Close() + + // Decode config from file. + var config ClientConfig + if err := json.NewDecoder(f).Decode(&config); err != nil { + return fmt.Errorf("decode config: %s", err) + } + + // Set options. + c.urls = config.URLs + + return nil +} + +// setConfig writes a new config to disk and updates urls on the client. +func (c *Client) setConfig(config ClientConfig) error { + // Only write to disk if we have a path. + if c.path != "" { + // Open config file for writing. + f, err := os.Create(c.path) + if err != nil { + return fmt.Errorf("create: %s", err) + } + defer f.Close() + + // Encode config to file. + if err := json.NewEncoder(f).Encode(&config); err != nil { + return fmt.Errorf("encode config: %s", err) + } + } + + // Set options. + c.urls = config.URLs + + return nil +} + // Publish sends a message to the broker and returns an index or error. func (c *Client) Publish(m *Message) (uint64, error) { // Post message to broker. @@ -198,7 +251,7 @@ func (c *Client) Publish(m *Message) (uint64, error) { if errstr := resp.Header.Get("X-Broker-Error"); errstr != "" { return 0, errors.New(errstr) } - return 0, fmt.Errorf("cannot publish(%d)", resp.StatusCode) + return 0, fmt.Errorf("cannot publish: status=%d", resp.StatusCode) } // Parse broker index. @@ -218,7 +271,26 @@ func (c *Client) Ping() error { if err != nil { return err } - resp.Body.Close() + defer resp.Body.Close() + + // Read entire body. + b, err := ioutil.ReadAll(resp.Body) + if err != nil { + return fmt.Errorf("read ping body: %s", err) + } + + // Update config if body is passed back. + if len(b) != 0 { + var config ClientConfig + if err := json.Unmarshal(b, &config); err != nil { + return fmt.Errorf("unmarshal config: %s", err) + } + + if err := c.setConfig(config); err != nil { + return fmt.Errorf("update config: %s", err) + } + } + return nil } @@ -293,15 +365,39 @@ func (c *Client) pinger(closing chan struct{}) { // ClientConfig represents the configuration that must be persisted across restarts. type ClientConfig struct { - Brokers []url.URL `json:"brokers"` - Leader url.URL `json:"leader"` + URLs []url.URL } -// NewClientConfig returns a new instance of ClientConfig. -func NewClientConfig(u []url.URL) *ClientConfig { - return &ClientConfig{ - Brokers: u, +func (c ClientConfig) MarshalJSON() ([]byte, error) { + var other clientConfigJSON + other.URLs = make([]string, len(c.URLs)) + for i, u := range c.URLs { + other.URLs[i] = u.String() } + return json.Marshal(&other) +} + +func (c *ClientConfig) UnmarshalJSON(b []byte) error { + var other clientConfigJSON + if err := json.Unmarshal(b, &other); err != nil { + return err + } + + c.URLs = make([]url.URL, len(other.URLs)) + for i := range other.URLs { + u, err := url.Parse(other.URLs[i]) + if err != nil { + return err + } + c.URLs[i] = *u + } + + return nil +} + +// clientConfigJSON represents the JSON +type clientConfigJSON struct { + URLs []string `json:"urls"` } // Conn represents a stream over the client for a single topic. @@ -465,7 +561,7 @@ func (c *Conn) Heartbeat() error { if errstr := resp.Header.Get("X-Broker-Error"); errstr != "" { return errors.New(errstr) } - return fmt.Errorf("heartbeat error: %d", resp.StatusCode) + return fmt.Errorf("heartbeat error: status=%d", resp.StatusCode) } return nil } @@ -563,3 +659,16 @@ func (c *Conn) stream(req *http.Request, closing <-chan struct{}) error { } } } + +// urlsEqual returns true if a and b contain the same URLs in the same order. +func urlsEqual(a, b []url.URL) bool { + if len(a) != len(b) { + return false + } + for i := range a { + if a[i] != b[i] { + return false + } + } + return true +} diff --git a/messaging/client_test.go b/messaging/client_test.go index 613fc76db30..8feadacc215 100644 --- a/messaging/client_test.go +++ b/messaging/client_test.go @@ -1,16 +1,323 @@ package messaging_test import ( + "encoding/json" "io/ioutil" "net/http" "net/http/httptest" "net/url" + "os" "reflect" + "strings" "testing" "github.com/influxdb/influxdb/messaging" ) +// Ensure a client can open the configuration file, if it exists. +func TestClient_Open_WithConfig(t *testing.T) { + // Write configuration file. + path := NewTempFile() + defer os.Remove(path) + MustWriteFile(path, []byte(`{"urls":["//hostA"]}`)) + + // Open new client against path. + c := NewClient() + if err := c.Open(path); err != nil { + t.Fatalf("unexpected error: %s", err) + } + defer c.Close() + + // Verify that urls were populated. + if a := c.URLs(); !reflect.DeepEqual(a, []url.URL{{Host: "hostA"}}) { + t.Fatalf("unexpected urls: %#v", a) + } +} + +// Ensure a client will ignore non-existent a config file. +func TestClient_Open_WithMissingConfig(t *testing.T) { + path := NewTempFile() + c := NewClient() + c.SetURLs([]url.URL{{Host: "//hostA"}}) + if err := c.Open(path); err != nil { + t.Fatalf("unexpected error: %s", err) + } + defer c.Close() + + // Verify that urls were cleared. + if a := c.URLs(); len(a) != 0 { + t.Fatalf("unexpected urls: %#v", a) + } +} + +// Ensure a client can return an error if the configuration file is corrupt. +func TestClient_Open_WithInvalidConfig(t *testing.T) { + // Write bad configuration file. + path := NewTempFile() + defer os.Remove(path) + MustWriteFile(path, []byte(`{"urls":`)) + + // Open new client against path. + c := NewClient() + if err := c.Open(path); err == nil || err.Error() != `load config: decode config: unexpected EOF` { + t.Fatalf("unexpected error: %s", err) + } + defer c.Close() +} + +// Ensure a client can return an error if the configuration file has non-readable permissions. +func TestClient_Open_WithBadPermConfig(t *testing.T) { + // Write inaccessible configuration file. + path := NewTempFile() + defer os.Remove(path) + MustWriteFile(path, []byte(`{"urls":["//hostA"]}`)) + os.Chmod(path, 0000) + + // Open new client against path. + c := NewClient() + if err := c.Open(path); err == nil || !strings.Contains(err.Error(), `permission denied`) { + t.Fatalf("unexpected error: %s", err) + } + defer c.Close() +} + +// Ensure a client returns an error when reopening. +func TestClient_Open_ErrClientOpen(t *testing.T) { + c := NewClient() + c.Open("") + defer c.Close() + if err := c.Open(""); err != messaging.ErrClientOpen { + t.Fatalf("unexpected error: %s", err) + } +} + +// Ensure the URL on a client can be set and retrieved. +func TestClient_SetURL(t *testing.T) { + c := NewClient() + defer c.Close() + + c.SetURL(url.URL{Host: "localhost"}) + if u := c.URL(); u != (url.URL{Host: "localhost"}) { + t.Fatalf("unexpected url: %s", u) + } +} + +// Ensure a client will update its connection urls. +func TestClient_SetURL_UpdateConn(t *testing.T) { + c := NewClient() + c.MustOpen("") + c.SetURLs([]url.URL{{Host: "hostA"}}) + defer c.Close() + + // Create connection & check URL. + conn := c.Conn(0) + if u := conn.URL(); u != (url.URL{Host: "hostA"}) { + t.Fatalf("unexpected initial connection url: %s", u) + } + + // Update client url. + c.SetURL(url.URL{Host: "hostB"}) + + // Check that connection url was updated. + if u := conn.URL(); u != (url.URL{Host: "hostB"}) { + t.Fatalf("unexpected new connection url: %s", u) + } +} + +// Ensure a set of URLs can be set on the client and retrieved. +// One of those URLs should be randomly set as the current URL. +func TestClient_SetURLs(t *testing.T) { + c := NewClient() + defer c.Close() + + // Set and retrieve URLs. + c.SetURLs([]url.URL{{Host: "hostA"}, {Host: "hostB"}}) + if a := c.URLs(); a[0] != (url.URL{Host: "hostA"}) { + t.Fatalf("unexpected urls length: %d", len(a)) + } else if a := c.URLs(); a[0] != (url.URL{Host: "hostA"}) { + t.Fatalf("unexpected url(0): %s", a[0]) + } else if a := c.URLs(); a[1] != (url.URL{Host: "hostB"}) { + t.Fatalf("unexpected url(1): %s", a[1]) + } + + // Current URL should be one of the URLs set. + if u := c.URL(); u != (url.URL{Host: "hostA"}) && u != (url.URL{Host: "hostB"}) { + t.Fatalf("unexpected url: %s", u) + } +} + +// Ensure that an empty set of URLs can be set to the client. +func TestClient_SetURLs_NoURLs(t *testing.T) { + c := NewClient() + defer c.Close() + c.SetURLs([]url.URL{}) +} + +// Ensure a client can publish a message to the broker. +func TestClient_Publish(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + if req.URL.Path != "/messaging/messages" { + t.Fatalf("unexpected path: %s", req.URL.Path) + } else if req.Method != "POST" { + t.Fatalf("unexpected method: %s", req.Method) + } else if typ := req.URL.Query().Get("type"); typ != "1" { + t.Fatalf("unexpected type: %s", typ) + } else if topicID := req.URL.Query().Get("topicID"); topicID != "2" { + t.Fatalf("unexpected topicID: %s", topicID) + } + + w.Header().Set("X-Broker-Index", "200") + })) + defer s.Close() + + // Create client. + c := NewClient() + c.MustOpen("") + c.SetURL(*MustParseURL(s.URL)) + defer c.Close() + + // Publish message to server. + if index, err := c.Publish(&messaging.Message{Type: 1, TopicID: 2, Data: []byte{0, 0, 0, 0}}); err != nil { + t.Fatal(err) + } else if index != 200 { + t.Fatalf("unexpected index: %d", index) + } +} + +// Ensure a client can redirect a published a message to another broker. +func TestClient_Publish_Redirect(t *testing.T) { + // Create a server to receive redirection. + s0 := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + if req.URL.Path != "/messaging/messages" { + t.Fatalf("unexpected path: %s", req.URL.Path) + } else if req.Method != "POST" { + t.Fatalf("unexpected method: %s", req.Method) + } else if typ := req.URL.Query().Get("type"); typ != "1" { + t.Fatalf("unexpected type: %s", typ) + } else if topicID := req.URL.Query().Get("topicID"); topicID != "2" { + t.Fatalf("unexpected topicID: %s", topicID) + } + + w.Header().Set("X-Broker-Index", "200") + })) + defer s0.Close() + + // Create another server to redirect to the first one. + s1 := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + http.Redirect(w, req, s0.URL+req.URL.Path, http.StatusTemporaryRedirect) + })) + defer s1.Close() + + // Create client. + c := NewClient() + c.MustOpen("") + c.SetURL(*MustParseURL(s1.URL)) + defer c.Close() + + // Publish message to server. + if index, err := c.Publish(&messaging.Message{Type: 1, TopicID: 2, Data: []byte{0, 0, 0, 0}}); err != nil { + t.Fatal(err) + } else if index != 200 { + t.Fatalf("unexpected index: %d", index) + } +} + +// Ensure a client returns an error if the responses Location header is invalid. +func TestClient_Publish_Redirect_ErrInvalidLocation(t *testing.T) { + // Create another server to redirect to the first one. + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + http.Redirect(w, req, "http://%f", http.StatusTemporaryRedirect) + })) + defer s.Close() + + // Create client. + c := NewClient() + c.MustOpen("") + c.SetURL(*MustParseURL(s.URL)) + defer c.Close() + + // Publish message to server. + if _, err := c.Publish(&messaging.Message{}); err == nil || err.Error() != `do: invalid redirect location: http://%f` { + t.Fatal(err) + } +} + +// Ensure a client returns an error publishing to a down broker. +func TestClient_Publish_ErrConnectionRefused(t *testing.T) { + s := httptest.NewServer(nil) + s.Close() + + // Create client. + c := NewClient() + c.MustOpen("") + c.SetURL(*MustParseURL(s.URL)) + defer c.Close() + + // Publish message to server. + if _, err := c.Publish(&messaging.Message{}); err == nil || !strings.Contains(err.Error(), `connection refused`) { + t.Fatal(err) + } +} + +// Ensure a client returns an error if returned by the server. +func TestClient_Publish_ErrBrokerError(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + w.Header().Set("X-Broker-Error", "oh no") + w.WriteHeader(http.StatusInternalServerError) + })) + defer s.Close() + + // Create client. + c := NewClient() + c.MustOpen("") + c.SetURL(*MustParseURL(s.URL)) + defer c.Close() + + // Publish message to server. + if _, err := c.Publish(&messaging.Message{}); err == nil || err.Error() != `oh no` { + t.Fatal(err) + } +} + +// Ensure a client returns an error if a non-broker error occurs. +func TestClient_Publish_ErrHTTPError(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + w.WriteHeader(http.StatusInternalServerError) + })) + defer s.Close() + + // Create client. + c := NewClient() + c.MustOpen("") + c.SetURL(*MustParseURL(s.URL)) + defer c.Close() + + // Publish message to server. + if _, err := c.Publish(&messaging.Message{}); err == nil || err.Error() != `cannot publish: status=500` { + t.Fatal(err) + } +} + +// Ensure a client returns an error if the returned index is invalid. +func TestClient_Publish_ErrInvalidIndex(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + w.Header().Set("X-Broker-Index", "xxx") + + })) + defer s.Close() + + // Create client. + c := NewClient() + c.MustOpen("") + c.SetURL(*MustParseURL(s.URL)) + defer c.Close() + + // Publish message to server. + if _, err := c.Publish(&messaging.Message{}); err == nil || err.Error() != `invalid index: strconv.ParseUint: parsing "xxx": invalid syntax` { + t.Fatal(err) + } +} + // Ensure a client can check if the server is alive. func TestClient_Ping(t *testing.T) { var pinged bool @@ -23,10 +330,9 @@ func TestClient_Ping(t *testing.T) { defer s.Close() // Create client. - c := messaging.NewClient() - if err := c.Open("", []url.URL{*MustParseURL(s.URL)}); err != nil { - t.Fatal(err) - } + c := NewClient() + c.MustOpen("") + c.SetURLs([]url.URL{*MustParseURL(s.URL)}) defer c.Close() // Ping server. @@ -37,6 +343,97 @@ func TestClient_Ping(t *testing.T) { } } +// Ensure a client returns an error if the ping cannot connect to the server. +func TestClient_Ping_ErrConnectionRefused(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) {})) + s.Close() + + // Create client. + c := NewClient() + c.MustOpen("") + c.SetURLs([]url.URL{*MustParseURL(s.URL)}) + defer c.Close() + + // Ping server. + if err := c.Ping(); err == nil || !strings.Contains(err.Error(), `connection refused`) { + t.Fatal(err) + } +} + +// Ensure a client returns an error if the body of the response cannot be read. +func TestClient_Ping_ErrRead(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + w.Header().Set("Content-Length", "10") + w.Write(make([]byte, 9)) + })) + defer s.Close() + + // Create client. + c := NewClient() + c.MustOpen("") + c.SetURLs([]url.URL{*MustParseURL(s.URL)}) + defer c.Close() + + // Ping server. + if err := c.Ping(); err == nil || err.Error() != `read ping body: unexpected EOF` { + t.Fatal(err) + } +} + +// Ensure a client can receive config data from the broker on ping. +func TestClient_Ping_ReceiveConfig(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + w.Write([]byte(`{"urls":["//local.dev"]}`)) + })) + defer s.Close() + + // Create a temp file for configuration. + path := NewTempFile() + defer os.Remove(path) + + // Create client. + c := NewClient() + c.MustOpen(path) + c.SetURLs([]url.URL{*MustParseURL(s.URL)}) + defer c.Close() + + // Ping server. + if err := c.Ping(); err != nil { + t.Fatal(err) + } + + // Confirm config change. + if a := c.URLs(); len(a) != 1 { + t.Fatalf("unexpected urls length: %d", len(a)) + } else if a[0] != (url.URL{Host: "local.dev"}) { + t.Fatalf("unexpected url(0): %s", a[0]) + } + + // Confirm config was rewritten. + if b, _ := ioutil.ReadFile(path); string(b) != `{"urls":["//local.dev"]}`+"\n" { + t.Fatalf("unexpected config file: %s", b) + } +} + +// Ensure a client returns an error when ping response is invalid. +func TestClient_Ping_ErrInvalidResponse(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + w.Write([]byte(`{"urls":`)) + })) + defer s.Close() + + // Create client. + c := NewClient() + c.MustOpen("") + c.SetURLs([]url.URL{*MustParseURL(s.URL)}) + defer c.Close() + + // Ping server. + if err := c.Ping(); err == nil || err.Error() != `unmarshal config: unexpected end of JSON input` { + t.Fatal(err) + } +} + // Ensure a client can be opened and connections can be created. func TestClient_Conn(t *testing.T) { s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { @@ -50,15 +447,16 @@ func TestClient_Conn(t *testing.T) { defer s.Close() // Create and open connection to server. - c := messaging.NewClient() - if err := c.Open("", []url.URL{*MustParseURL(s.URL)}); err != nil { - t.Fatal(err) - } + c := NewClient() + c.MustOpen("") + c.SetURLs([]url.URL{*MustParseURL(s.URL)}) // Connect on topic #1. conn1 := c.Conn(1) if err := conn1.Open(0, false); err != nil { t.Fatal(err) + } else if conn1.TopicID() != 1 { + t.Fatalf("unexpected topic id(1): %d", conn1.TopicID()) } else if m := <-conn1.C(); !reflect.DeepEqual(m, &messaging.Message{Index: 1, Data: []byte{100}}) { t.Fatalf("unexpected message(1): %#v", m) } @@ -205,21 +603,122 @@ func TestConn_Heartbeat(t *testing.T) { } } -// Client represents a test wrapper for the broker client. +// Ensure that a connection returns an error if it cannot connect to the broker. +func TestConn_Heartbeat_ErrConnectionRefused(t *testing.T) { + s := httptest.NewServer(nil) + s.Close() + + // Create connection and heartbeat. + c := messaging.NewConn(0) + c.SetURL(*MustParseURL(s.URL)) + if err := c.Heartbeat(); err == nil || !strings.Contains(err.Error(), `connection refused`) { + t.Fatalf("unexpected error: %s", err) + } +} + +// Ensure that a connection returns an error if the heartbeat is redirected. +// This occurs when the broker is not the leader. The client will update the URL later. +func TestConn_Heartbeat_ErrNoLeader(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + w.WriteHeader(http.StatusTemporaryRedirect) + })) + defer s.Close() + + // Create connection and heartbeat. + c := messaging.NewConn(0) + c.SetURL(*MustParseURL(s.URL)) + if err := c.Heartbeat(); err != messaging.ErrNoLeader { + t.Fatalf("unexpected error: %s", err) + } +} + +// Ensure that a connection returns a broker error while heartbeating. +func TestConn_Heartbeat_ErrBrokerError(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + w.Header().Set("X-Broker-Error", "oh no") + w.WriteHeader(http.StatusInternalServerError) + })) + defer s.Close() + + // Create connection and heartbeat. + c := messaging.NewConn(0) + c.SetURL(*MustParseURL(s.URL)) + if err := c.Heartbeat(); err == nil || err.Error() != `oh no` { + t.Fatalf("unexpected error: %s", err) + } +} + +// Ensure that a connection returns an http error while heartbeating. +func TestConn_Heartbeat_ErrHTTPError(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + w.WriteHeader(http.StatusInternalServerError) + })) + defer s.Close() + + // Create connection and heartbeat. + c := messaging.NewConn(0) + c.SetURL(*MustParseURL(s.URL)) + if err := c.Heartbeat(); err == nil || err.Error() != `heartbeat error: status=500` { + t.Fatalf("unexpected error: %s", err) + } +} + +// Ensure that the client config can be serialized to JSON. +func TestClientConfig_MarshalJSON(t *testing.T) { + c := messaging.ClientConfig{URLs: []url.URL{{Host: "hostA"}, {Host: "hostB"}}} + if b, err := json.Marshal(&c); err != nil { + t.Fatal(err) + } else if string(b) != `{"urls":["//hostA","//hostB"]}` { + t.Fatalf("unexpected json: %s", b) + } +} + +// Ensure that the client config can be deserialized from JSON. +func TestClientConfig_UnmarshalJSON(t *testing.T) { + var c messaging.ClientConfig + if err := json.Unmarshal([]byte(`{"urls":["//hostA","//hostB"]}`), &c); err != nil { + t.Fatal(err) + } + if len(c.URLs) != 2 { + t.Fatalf("unexpected url count: %d", len(c.URLs)) + } else if c.URLs[0] != (url.URL{Host: "hostA"}) { + t.Fatalf("unexpected url(0): %s", c.URLs[0]) + } else if c.URLs[1] != (url.URL{Host: "hostB"}) { + t.Fatalf("unexpected url(1): %s", c.URLs[1]) + } +} + +// Ensure that the client config returns an error when handling an invalid field type. +func TestClientConfig_UnmarshalJSON_ErrInvalidType(t *testing.T) { + var c messaging.ClientConfig + if err := json.Unmarshal([]byte(`{"urls":0}`), &c); err == nil || err.Error() != `json: cannot unmarshal number into Go value of type []string` { + t.Fatalf("unexpected error: %s", err) + } +} + +// Ensure that the client config returns an error when handling an invalid url. +func TestClientConfig_UnmarshalJSON_ErrInvalidURL(t *testing.T) { + var c messaging.ClientConfig + if err := json.Unmarshal([]byte(`{"urls":["http://%foo"]}`), &c); err == nil || err.Error() != `parse http://%foo: hexadecimal escape in host` { + t.Fatalf("unexpected error: %s", err) + } +} + +// Client represents a test wrapper for messaging.Client. type Client struct { *messaging.Client } -// NewClient returns a new instance of Client. -func NewClient(replicaID uint64) *Client { - return &Client{ - Client: messaging.NewClient(), - } +// NewClient returns an new instance of Client. +func NewClient() *Client { + return &Client{messaging.NewClient()} } -// Close shuts down the client and server. -func (c *Client) Close() { - c.Client.Close() +// MustOpen opens the client. Panic on error. +func (c *Client) MustOpen(path string) { + if err := c.Open(path); err != nil { + panic(err.Error()) + } } // NewTempFile returns the path of a new temporary file. @@ -229,6 +728,7 @@ func NewTempFile() string { if err != nil { panic(err) } - defer f.Close() + f.Close() + os.Remove(f.Name()) return f.Name() } diff --git a/messaging/errors.go b/messaging/errors.go index 35462b99e13..ce870317194 100644 --- a/messaging/errors.go +++ b/messaging/errors.go @@ -46,9 +46,6 @@ var ( // ErrConnCannotReuse is returned when opening a previously closed connection. ErrConnCannotReuse = errors.New("cannot reuse connection") - // ErrBrokerURLRequired is returned when opening a broker without URLs. - ErrBrokerURLRequired = errors.New("broker url required") - // ErrMessageTypeRequired is returned publishing a message without a type. ErrMessageTypeRequired = errors.New("message type required") diff --git a/messaging/handler.go b/messaging/handler.go index 51a1492356a..f7318c71cfa 100644 --- a/messaging/handler.go +++ b/messaging/handler.go @@ -1,6 +1,7 @@ package messaging import ( + "encoding/json" "io" "io/ioutil" "log" @@ -15,6 +16,8 @@ import ( // Handler represents an HTTP handler by the broker. type Handler struct { Broker interface { + URLs() []url.URL + IsLeader() bool LeaderURL() url.URL TopicReader(topicID, index uint64, streaming bool) io.ReadCloser Publish(m *Message) (uint64, error) @@ -169,6 +172,20 @@ func (h *Handler) postHeartbeat(w http.ResponseWriter, r *http.Request) { // servePing returns a status 200. func (h *Handler) servePing(w http.ResponseWriter, r *http.Request) { + // Redirect if not leader. + if !h.Broker.IsLeader() { + h.redirectToLeader(w, r) + return + } + + // Write out client configuration. + var config ClientConfig + config.URLs = h.Broker.URLs() + if err := json.NewEncoder(w).Encode(&config); err != nil { + log.Printf("unable to write client config: %s", err) + return + } + w.WriteHeader(http.StatusOK) } diff --git a/messaging/handler_test.go b/messaging/handler_test.go index 01ee316d388..198be0dc9cf 100644 --- a/messaging/handler_test.go +++ b/messaging/handler_test.go @@ -3,6 +3,7 @@ package messaging_test import ( "bytes" "io" + "io/ioutil" "net/http" "net/http/httptest" "net/url" @@ -195,19 +196,48 @@ func TestHandler_postHeartbeat_ErrMethodNotAllowed(t *testing.T) { resp.Body.Close() } -// Ensure a handler can respond to a ping. +// Ensure a handler can respond to a ping with the current cluster configuration. func TestHandler_servePing(t *testing.T) { - s := httptest.NewServer(&messaging.Handler{}) + var hb HandlerBroker + hb.IsLeaderFunc = func() bool { return true } + hb.URLsFunc = func() []url.URL { return []url.URL{{Host: "hostA"}, {Host: "hostB"}} } + s := httptest.NewServer(&messaging.Handler{Broker: &hb}) defer s.Close() // Send request to the broker. resp, err := http.Post(s.URL+`/messaging/ping`, "application/octet-stream", nil) if err != nil { t.Fatal(err) - } else if resp.StatusCode != http.StatusOK { + } + defer resp.Body.Close() + + if resp.StatusCode != http.StatusOK { t.Fatalf("unexpected status: %d: %s", resp.StatusCode, resp.Header.Get("X-Broker-Error")) + } else if b, _ := ioutil.ReadAll(resp.Body); string(b) != `{"urls":["//hostA","//hostB"]}`+"\n" { + t.Fatalf("unexpected body: %s", b) + } +} + +// Ensure a handler can respond to a ping with the current cluster configuration. +func TestHandler_servePing_NotLeader(t *testing.T) { + var hb HandlerBroker + hb.IsLeaderFunc = func() bool { return false } + hb.LeaderURLFunc = func() url.URL { return url.URL{Scheme: "http", Host: "other"} } + s := httptest.NewServer(&messaging.Handler{Broker: &hb}) + defer s.Close() + + // Send request to the broker. + resp, err := http.Post(s.URL+`/messaging/ping`, "application/octet-stream", nil) + if err != nil { + t.Fatal(err) + } + defer resp.Body.Close() + + if resp.StatusCode != http.StatusTemporaryRedirect { + t.Fatalf("unexpected status: %d: %s", resp.StatusCode, resp.Header.Get("X-Broker-Error")) + } else if loc := resp.Header.Get("Location"); loc != "http://other/messaging/ping" { + t.Fatalf("unexpected redirect location: %s", loc) } - resp.Body.Close() } // Ensure the handler routes raft requests to the raft handler. @@ -239,12 +269,16 @@ func TestHandler_ErrNotFound(t *testing.T) { // HandlerBroker is a mockable type that implements Handler.Broker. type HandlerBroker struct { + URLsFunc func() []url.URL + IsLeaderFunc func() bool LeaderURLFunc func() url.URL PublishFunc func(m *messaging.Message) (uint64, error) TopicReaderFunc func(topicID, index uint64, streaming bool) io.ReadCloser SetTopicMaxIndexFunc func(topicID, index uint64) error } +func (b *HandlerBroker) URLs() []url.URL { return b.URLsFunc() } +func (b *HandlerBroker) IsLeader() bool { return b.IsLeaderFunc() } func (b *HandlerBroker) LeaderURL() url.URL { return b.LeaderURLFunc() } func (b *HandlerBroker) Publish(m *messaging.Message) (uint64, error) { return b.PublishFunc(m) } func (b *HandlerBroker) TopicReader(topicID, index uint64, streaming bool) io.ReadCloser { diff --git a/raft/log.go b/raft/log.go index 1d7c14b46c1..e47696c8421 100644 --- a/raft/log.go +++ b/raft/log.go @@ -172,6 +172,23 @@ func (l *Log) SetURL(u url.URL) { l.url = u } +// URLs returns a list of all URLs in the cluster. +func (l *Log) URLs() []url.URL { + l.mu.Lock() + defer l.mu.Unlock() + + if l.config == nil { + return nil + } + + var a []url.URL + for _, n := range l.config.Nodes { + a = append(a, n.URL) + } + + return a +} + func (l *Log) idPath() string { return filepath.Join(l.path, "id") } func (l *Log) termPath() string { return filepath.Join(l.path, "term") } func (l *Log) configPath() string { return filepath.Join(l.path, "config") } @@ -549,23 +566,19 @@ func (l *Log) tracef(msg string, v ...interface{}) { } } -// Leader returns the id and URL associated with the current leader. -// Returns zero if there is no current leader. -func (l *Log) Leader() (id uint64, u url.URL) { +// IsLeader returns true if the log is the current leader. +func (l *Log) IsLeader() bool { l.mu.Lock() defer l.mu.Unlock() - return l.leader() + return l.id != 0 && l.id == l.leaderID } -// ClusterID returns the identifier for the cluster. -// Returns zero if the cluster has not been initialized yet. -func (l *Log) ClusterID() uint64 { +// Leader returns the id and URL associated with the current leader. +// Returns zero if there is no current leader. +func (l *Log) Leader() (id uint64, u url.URL) { l.mu.Lock() defer l.mu.Unlock() - if l.config == nil { - return 0 - } - return l.config.ClusterID + return l.leader() } func (l *Log) leader() (id uint64, u url.URL) { @@ -583,6 +596,17 @@ func (l *Log) leader() (id uint64, u url.URL) { return n.ID, n.URL } +// ClusterID returns the identifier for the cluster. +// Returns zero if the cluster has not been initialized yet. +func (l *Log) ClusterID() uint64 { + l.mu.Lock() + defer l.mu.Unlock() + if l.config == nil { + return 0 + } + return l.config.ClusterID +} + // Join contacts a node in the cluster to request membership. // A log cannot join a cluster if it has already been initialized. func (l *Log) Join(u url.URL) error { diff --git a/server.go b/server.go index 20415ef7e6d..d492df0ea77 100644 --- a/server.go +++ b/server.go @@ -2964,9 +2964,13 @@ func (r *Results) Error() error { // MessagingClient represents the client used to connect to brokers. type MessagingClient interface { - Open(path string, urls []url.URL) error + Open(path string) error Close() error + // Retrieves or sets the current list of broker URLs. + URLs() []url.URL + SetURLs([]url.URL) + // Publishes a message to the broker. Publish(m *messaging.Message) (index uint64, err error) diff --git a/test/messaging.go b/test/messaging.go index ae0e9d852a0..10feb942ea4 100644 --- a/test/messaging.go +++ b/test/messaging.go @@ -32,7 +32,7 @@ func NewMessagingClient() *MessagingClient { return c } -func (c *MessagingClient) Open(path string, urls []url.URL) error { return nil } +func (c *MessagingClient) Open(path string) error { return nil } // Close closes all open connections. func (c *MessagingClient) Close() error { @@ -46,6 +46,9 @@ func (c *MessagingClient) Close() error { return nil } +func (c *MessagingClient) URLs() []url.URL { return []url.URL{{Host: "local"}} } +func (c *MessagingClient) SetURLs([]url.URL) {} + func (c *MessagingClient) Publish(m *messaging.Message) (uint64, error) { return c.PublishFunc(m) } // DefaultPublishFunc sets an autoincrementing index on the message and sends it to each topic connection. From 96748cb2179861f172c6ea2424b056912b33e57d Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 14 Mar 2015 13:49:25 -0600 Subject: [PATCH 17/20] Update file permissions. --- messaging/broker.go | 14 +++++++------- messaging/broker_test.go | 4 ++-- metastore.go | 2 +- raft/log.go | 4 ++-- shard.go | 2 +- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/messaging/broker.go b/messaging/broker.go index e584d53a1df..59d47c28a01 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -135,13 +135,13 @@ func (b *Broker) Open(path string) error { b.path = path // Ensure root directory exists. - if err := os.MkdirAll(path, 0700); err != nil { + if err := os.MkdirAll(path, 0777); err != nil { b.close() return fmt.Errorf("mkdir: %s", err) } // Open meta file. - meta, err := bolt.Open(b.metaPath(), 0600, &bolt.Options{Timeout: 1 * time.Second}) + meta, err := bolt.Open(b.metaPath(), 0666, &bolt.Options{Timeout: 1 * time.Second}) if err != nil { b.close() return fmt.Errorf("open meta: %s", err) @@ -359,7 +359,7 @@ func (b *Broker) Restore(r io.Reader) error { // Remove and recreate broker path. if err := b.reset(); err != nil && !os.IsNotExist(err) { return fmt.Errorf("reset: %s", err) - } else if err = os.MkdirAll(b.path, 0700); err != nil { + } else if err = os.MkdirAll(b.path, 0777); err != nil { return fmt.Errorf("mkdir: %s", err) } @@ -387,7 +387,7 @@ func (b *Broker) Restore(r io.Reader) error { t := NewTopic(st.ID, b.topicPath(st.ID)) // Create topic directory. - if err := os.MkdirAll(t.Path(), 0700); err != nil { + if err := os.MkdirAll(t.Path(), 0777); err != nil { return fmt.Errorf("make topic dir: %s", err) } @@ -677,7 +677,7 @@ func (t *Topic) Open() error { t.opened = true // Ensure the parent directory exists. - if err := os.MkdirAll(t.path, 0700); err != nil { + if err := os.MkdirAll(t.path, 0777); err != nil { t.close() return err } @@ -702,7 +702,7 @@ func (t *Topic) Open() error { t.index = index // Open file handle on the segment. - f, err := os.OpenFile(s.Path, os.O_RDWR|os.O_APPEND, 0600) + f, err := os.OpenFile(s.Path, os.O_RDWR|os.O_APPEND, 0666) if err != nil { t.close() return fmt.Errorf("open segment: %s", err) @@ -776,7 +776,7 @@ func (t *Topic) WriteMessage(m *Message) error { // Create a new segment if we have no handle. if t.file == nil { - f, err := os.OpenFile(t.segmentPath(m.Index), os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0600) + f, err := os.OpenFile(t.segmentPath(m.Index), os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0666) if err != nil { return fmt.Errorf("create segment file: %s", err) } diff --git a/messaging/broker_test.go b/messaging/broker_test.go index 80a15f97ff3..f524af1dbf5 100644 --- a/messaging/broker_test.go +++ b/messaging/broker_test.go @@ -751,10 +751,10 @@ func tempfile() string { // MustWriteFile writes data to a file. Panic on error. func MustWriteFile(filename string, data []byte) { - if err := os.MkdirAll(filepath.Dir(filename), 0700); err != nil { + if err := os.MkdirAll(filepath.Dir(filename), 0777); err != nil { panic(err.Error()) } - if err := ioutil.WriteFile(filename, data, 0600); err != nil { + if err := ioutil.WriteFile(filename, data, 0666); err != nil { panic(err.Error()) } } diff --git a/metastore.go b/metastore.go index 76db085dbbf..8f5bae03811 100644 --- a/metastore.go +++ b/metastore.go @@ -16,7 +16,7 @@ type metastore struct { // open initializes the metastore. func (m *metastore) open(path string) error { // Open the bolt-backed database. - db, err := bolt.Open(path, 0600, &bolt.Options{Timeout: 1 * time.Second}) + db, err := bolt.Open(path, 0666, &bolt.Options{Timeout: 1 * time.Second}) if err != nil { return err } diff --git a/raft/log.go b/raft/log.go index e47696c8421..434efe10bd3 100644 --- a/raft/log.go +++ b/raft/log.go @@ -407,7 +407,7 @@ func (l *Log) readID() (uint64, error) { // writeID writes the log identifier to file. func (l *Log) writeID(id uint64) error { b := []byte(strconv.FormatUint(id, 10)) - return ioutil.WriteFile(l.idPath(), b, 0600) + return ioutil.WriteFile(l.idPath(), b, 0666) } // readTerm reads the log term from file. @@ -432,7 +432,7 @@ func (l *Log) readTerm() (uint64, error) { // writeTerm writes the current log term to file. func (l *Log) writeTerm(term uint64) error { b := []byte(strconv.FormatUint(term, 10)) - return ioutil.WriteFile(l.termPath(), b, 0600) + return ioutil.WriteFile(l.termPath(), b, 0666) } // readConfig reads the configuration from disk. diff --git a/shard.go b/shard.go index fa6635e2faa..e3e24341752 100644 --- a/shard.go +++ b/shard.go @@ -76,7 +76,7 @@ func (s *Shard) open(path string, conn MessagingConn) error { } // Open store on shard. - store, err := bolt.Open(path, 0600, &bolt.Options{Timeout: 1 * time.Second}) + store, err := bolt.Open(path, 0666, &bolt.Options{Timeout: 1 * time.Second}) if err != nil { return err } From b045ad5d924cccd48fb6ba30ee7022ab6afdf335 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 14 Mar 2015 13:55:19 -0600 Subject: [PATCH 18/20] Wrap open logic in anonymous functions. --- messaging/broker.go | 116 +++++++++++++++++++++++--------------------- messaging/client.go | 20 +++++--- 2 files changed, 75 insertions(+), 61 deletions(-) diff --git a/messaging/broker.go b/messaging/broker.go index 59d47c28a01..98352a2917c 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -132,43 +132,47 @@ func (b *Broker) Open(path string) error { if path == "" { return ErrPathRequired } - b.path = path - // Ensure root directory exists. - if err := os.MkdirAll(path, 0777); err != nil { - b.close() - return fmt.Errorf("mkdir: %s", err) - } + if err := func() error { + b.path = path - // Open meta file. - meta, err := bolt.Open(b.metaPath(), 0666, &bolt.Options{Timeout: 1 * time.Second}) - if err != nil { - b.close() - return fmt.Errorf("open meta: %s", err) - } - b.meta = meta + // Ensure root directory exists. + if err := os.MkdirAll(path, 0777); err != nil { + return fmt.Errorf("mkdir: %s", err) + } - // Initialize data from meta store. - if err := b.meta.Update(func(tx *bolt.Tx) error { - tx.CreateBucketIfNotExists([]byte("meta")) + // Open meta file. + meta, err := bolt.Open(b.metaPath(), 0666, &bolt.Options{Timeout: 1 * time.Second}) + if err != nil { + return fmt.Errorf("open meta: %s", err) + } + b.meta = meta + + // Initialize data from meta store. + if err := b.meta.Update(func(tx *bolt.Tx) error { + tx.CreateBucketIfNotExists([]byte("meta")) + + // Read in index from meta store, if set. + if v := tx.Bucket([]byte("meta")).Get([]byte("index")); v != nil { + b.index = btou64(v) + } - // Read in index from meta store, if set. - if v := tx.Bucket([]byte("meta")).Get([]byte("index")); v != nil { - b.index = btou64(v) + return nil + }); err != nil { + return err + } + + // Read all topic metadata into memory. + if err := b.openTopics(); err != nil { + return fmt.Errorf("open topics: %s", err) } return nil - }); err != nil { + }(); err != nil { _ = b.close() return err } - // Read all topic metadata into memory. - if err := b.openTopics(); err != nil { - b.close() - return fmt.Errorf("open topics: %s", err) - } - return nil } @@ -674,40 +678,44 @@ func (t *Topic) Open() error { } else if t.path == "" { return ErrPathRequired } - t.opened = true - - // Ensure the parent directory exists. - if err := os.MkdirAll(t.path, 0777); err != nil { - t.close() - return err - } - // Read available segments. - segments, err := ReadSegments(t.path) - if err != nil && !os.IsNotExist(err) { - t.close() - return fmt.Errorf("read segments: %s", err) - } + if err := func() error { + t.opened = true - // Read max index and open file handle if we have segments. - if len(segments) > 0 { - s := segments.Last() + // Ensure the parent directory exists. + if err := os.MkdirAll(t.path, 0777); err != nil { + return err + } - // Read the last segment and extract the last message index. - index, err := ReadSegmentMaxIndex(s.Path) - if err != nil { - t.close() - return fmt.Errorf("read segment max index: %s", err) + // Read available segments. + segments, err := ReadSegments(t.path) + if err != nil && !os.IsNotExist(err) { + return fmt.Errorf("read segments: %s", err) } - t.index = index - // Open file handle on the segment. - f, err := os.OpenFile(s.Path, os.O_RDWR|os.O_APPEND, 0666) - if err != nil { - t.close() - return fmt.Errorf("open segment: %s", err) + // Read max index and open file handle if we have segments. + if len(segments) > 0 { + s := segments.Last() + + // Read the last segment and extract the last message index. + index, err := ReadSegmentMaxIndex(s.Path) + if err != nil { + return fmt.Errorf("read segment max index: %s", err) + } + t.index = index + + // Open file handle on the segment. + f, err := os.OpenFile(s.Path, os.O_RDWR|os.O_APPEND, 0666) + if err != nil { + return fmt.Errorf("open segment: %s", err) + } + t.file = f } - t.file = f + + return nil + }(); err != nil { + _ = t.close() + return err } return nil diff --git a/messaging/client.go b/messaging/client.go index b9f64a2ff61..32511e42111 100644 --- a/messaging/client.go +++ b/messaging/client.go @@ -132,16 +132,22 @@ func (c *Client) Open(path string) error { return ErrClientOpen } - // Read URLs from file if no URLs are provided. - c.path = path - if err := c.loadConfig(); err != nil { + if err := func() error { + // Read URLs from file if no URLs are provided. + c.path = path + if err := c.loadConfig(); err != nil { + return fmt.Errorf("load config: %s", err) + } + + // Set open flag. + c.opened = true + + return nil + }(); err != nil { _ = c.close() - return fmt.Errorf("load config: %s", err) + return err } - // Set open flag. - c.opened = true - // Start background ping. c.closing = make(chan struct{}, 0) c.wg.Add(1) From 41d357ac04c5ea7893b35d0eae76b475846748bf Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 14 Mar 2015 14:07:09 -0600 Subject: [PATCH 19/20] Fixes based on code review comments. --- messaging/broker.go | 10 +++++----- messaging/handler.go | 6 +++--- server.go | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/messaging/broker.go b/messaging/broker.go index 98352a2917c..e26745c1a84 100644 --- a/messaging/broker.go +++ b/messaging/broker.go @@ -176,7 +176,7 @@ func (b *Broker) Open(path string) error { return nil } -// loadTopics reads all topic metadata into memory. +// openTopics reads all topic metadata into memory. func (b *Broker) openTopics() error { // Read all topics from the broker directory. topics, err := ReadTopics(b.path) @@ -237,8 +237,8 @@ func (b *Broker) closeTopics() { b.topics = make(map[uint64]*Topic) } -// SetMaxIndex sets the highest index seen by the broker. -// This is only used for internal log messages and topics may have a higher index. +// SetMaxIndex sets the highest index applied by the broker. +// This is only used for internal log messages. Topics may have a higher index. func (b *Broker) SetMaxIndex(index uint64) error { b.mu.Lock() defer b.mu.Unlock() @@ -308,7 +308,7 @@ func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) { // Create snapshot topic. st := &snapshotTopic{ID: t.id} - // Read segments from disk, not topic. + // Read segments from disk. segments, err := ReadSegments(t.path) if err != nil && !os.IsNotExist(err) { return nil, fmt.Errorf("read segments: %s", err) @@ -418,7 +418,7 @@ func (b *Broker) Restore(r io.Reader) error { } } - // Open new empty topic file. + // Open topic. if err := t.Open(); err != nil { return fmt.Errorf("open topic: %s", err) } diff --git a/messaging/handler.go b/messaging/handler.go index f7318c71cfa..4329912766b 100644 --- a/messaging/handler.go +++ b/messaging/handler.go @@ -105,7 +105,7 @@ func (h *Handler) getMessages(w http.ResponseWriter, req *http.Request) { // postMessages publishes a message to the broker. func (h *Handler) postMessages(w http.ResponseWriter, r *http.Request) { // Read the message type. - typ, err := strconv.ParseUint(r.URL.Query().Get("type"), 10, 16) + typ, err := strconv.ParseUint(r.URL.Query().Get("type"), 10, 64) if err != nil { h.error(w, ErrMessageTypeRequired, http.StatusBadRequest) return @@ -147,14 +147,14 @@ func (h *Handler) postMessages(w http.ResponseWriter, r *http.Request) { // replicated index for a given topic. func (h *Handler) postHeartbeat(w http.ResponseWriter, r *http.Request) { // Read the topic id. - topicID, err := strconv.ParseUint(r.URL.Query().Get("topicID"), 10, 16) + topicID, err := strconv.ParseUint(r.URL.Query().Get("topicID"), 10, 64) if err != nil { h.error(w, ErrTopicRequired, http.StatusBadRequest) return } // Read the index. - index, err := strconv.ParseUint(r.URL.Query().Get("index"), 10, 16) + index, err := strconv.ParseUint(r.URL.Query().Get("index"), 10, 64) if err != nil { h.error(w, ErrIndexRequired, http.StatusBadRequest) return diff --git a/server.go b/server.go index d492df0ea77..6a98e775b8b 100644 --- a/server.go +++ b/server.go @@ -2802,7 +2802,7 @@ func (s *Server) processor(conn MessagingConn, done chan struct{}) { } } - // All other messages must be processed under lock. + // All messages must be processed under lock. func() { s.mu.Lock() defer s.mu.Unlock() From 7dc465b9dbbfd217477caace0a617fc8a3e0077f Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 14 Mar 2015 14:47:20 -0600 Subject: [PATCH 20/20] Fix shard close race condition. --- shard.go | 44 +++++++++++++++++++++++++++++++++----------- 1 file changed, 33 insertions(+), 11 deletions(-) diff --git a/shard.go b/shard.go index e3e24341752..60629b4ba52 100644 --- a/shard.go +++ b/shard.go @@ -4,9 +4,11 @@ import ( "encoding/binary" "errors" "fmt" + "sync" "time" "github.com/boltdb/bolt" + "github.com/influxdb/influxdb/messaging" ) // ShardGroup represents a group of shards created for a single time range. @@ -63,6 +65,9 @@ type Shard struct { index uint64 // highest replicated index store *bolt.DB // underlying data store conn MessagingConn // streaming connection to broker + + wg sync.WaitGroup // pending goroutines + closing chan struct{} // close notification } // newShard returns a new initialized Shard instance. @@ -93,26 +98,35 @@ func (s *Shard) open(path string, conn MessagingConn) error { s.index = btou64(buf) } + // Open connection. + if err := conn.Open(s.index, true); err != nil { + return fmt.Errorf("open shard conn: id=%d, idx=%d, err=%s", s.ID, s.index, err) + } + return nil }); err != nil { _ = s.close() return fmt.Errorf("init: %s", err) } - // Open connection. - if err := conn.Open(s.index, true); err != nil { - _ = s.close() - return fmt.Errorf("open shard conn: id=%d, idx=%d, err=%s", s.ID, s.index, err) - } - // Start importing from connection. - go s.processor(conn) + s.closing = make(chan struct{}) + s.wg.Add(1) + go s.processor(conn, s.closing) return nil } // close shuts down the shard's store. func (s *Shard) close() error { + // Wait for goroutines to stop. + if s.closing != nil { + close(s.closing) + s.closing = nil + } + + s.wg.Wait() + if s.store != nil { _ = s.store.Close() } @@ -201,12 +215,20 @@ func (s *Shard) dropSeries(seriesID uint32) error { } // processor runs in a separate goroutine and processes all incoming broker messages. -func (s *Shard) processor(conn MessagingConn) { +func (s *Shard) processor(conn MessagingConn, closing <-chan struct{}) { + defer s.wg.Done() + for { // Read incoming message. - // Exit if the connection has been closed. - m, ok := <-conn.C() - if !ok { + // Exit if the connection has been closed or if shard is closing. + var ok bool + var m *messaging.Message + select { + case m, ok = <-conn.C(): + if !ok { + return + } + case <-closing: return }