influxdb/messaging/broker.go

1156 lines
28 KiB
Go
Raw Normal View History

2014-10-21 02:42:03 +00:00
package messaging
2014-10-03 03:13:42 +00:00
import (
"bufio"
2014-10-03 03:13:42 +00:00
"encoding/binary"
2014-10-04 17:27:12 +00:00
"encoding/json"
2014-10-03 03:13:42 +00:00
"fmt"
"io"
"log"
"net/url"
2014-10-03 03:13:42 +00:00
"os"
"path/filepath"
2014-10-12 18:05:03 +00:00
"sort"
2014-10-24 04:22:52 +00:00
"strconv"
2014-10-03 03:13:42 +00:00
"sync"
"time"
2014-10-03 03:13:42 +00:00
"github.com/boltdb/bolt"
2014-10-03 03:13:42 +00:00
"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
2014-10-12 18:05:03 +00:00
2014-10-03 03:13:42 +00:00
// Broker represents distributed messaging system segmented into topics.
// 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
2014-10-03 03:13:42 +00:00
meta *bolt.DB // metadata
topics map[uint64]*topic // topics by id
Logger *log.Logger
2014-10-03 03:13:42 +00:00
}
2014-10-21 02:42:03 +00:00
// NewBroker returns a new instance of a Broker with default values.
func NewBroker() *Broker {
2014-10-03 03:13:42 +00:00
b := &Broker{
log: raft.NewLog(),
topics: make(map[uint64]*topic),
Logger: log.New(os.Stderr, "[broker] ", log.LstdFlags),
2014-10-03 03:13:42 +00:00
}
b.log.FSM = (*brokerFSM)(b)
return b
}
// Path returns the path used when opening the broker.
// Returns empty string if the broker is not open.
func (b *Broker) Path() string { return b.path }
2015-02-22 09:13:34 +00:00
2015-02-25 03:32:20 +00:00
// Log returns the underlying raft log.
2015-02-22 09:13:34 +00:00
func (b *Broker) Log() *raft.Log { return b.log }
2014-10-03 03:13:42 +00:00
2015-02-25 03:32:20 +00:00
// metaPath returns the file path to the broker's metadata file.
2015-01-28 06:09:50 +00:00
func (b *Broker) metaPath() string {
if b.path == "" {
return ""
}
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))
}
2015-02-21 22:25:11 +00:00
// 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.RLock()
defer b.mu.RUnlock()
return b.index
}
2015-02-25 03:32:20 +00:00
// opened returns true if the broker is in an open and running state.
2014-10-03 03:13:42 +00:00
func (b *Broker) opened() bool { return b.path != "" }
2015-01-29 23:07:58 +00:00
// 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)
}
2014-10-03 03:13:42 +00:00
// 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 {
2014-10-03 03:13:42 +00:00
b.mu.Lock()
defer b.mu.Unlock()
// Require a non-blank path.
if path == "" {
2014-10-12 18:05:03 +00:00
return ErrPathRequired
2014-10-03 03:13:42 +00:00
}
b.path = path
// Require a non-blank connection address.
if u == nil {
return ErrConnectionAddressRequired
}
// 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 {
2015-01-28 06:09:50 +00:00
_ = 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.
2014-10-03 03:13:42 +00:00
if err := b.log.Open(filepath.Join(path, "raft")); err != nil {
2015-02-25 03:32:20 +00:00
_ = b.close()
2014-10-03 03:13:42 +00:00
return fmt.Errorf("raft: %s", err)
}
// Copy connection URL.
b.log.URL = &url.URL{}
*b.log.URL = *u
2014-10-03 03:13:42 +00:00
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
}
2014-10-03 03:13:42 +00:00
// Close closes the broker and all topics.
func (b *Broker) Close() error {
b.mu.Lock()
defer b.mu.Unlock()
2015-01-28 06:09:50 +00:00
return b.close()
}
2014-10-03 03:13:42 +00:00
2015-01-28 06:09:50 +00:00
func (b *Broker) close() error {
2014-10-03 03:13:42 +00:00
// Return error if the broker is already closed.
if !b.opened() {
2014-10-12 18:05:03 +00:00
return ErrClosed
2014-10-03 03:13:42 +00:00
}
b.path = ""
// Close all topics.
b.closeTopics()
2014-10-17 04:11:28 +00:00
2014-10-03 03:13:42 +00:00
// Close raft log.
_ = b.log.Close()
return nil
}
// closeTopics closes all topic files and clears the topics map.
func (b *Broker) closeTopics() {
for _, t := range b.topics {
_ = t.Close()
}
b.topics = make(map[uint64]*topic)
}
2015-01-28 06:09:50 +00:00
// createSnapshotHeader creates a snapshot header.
func (b *Broker) createSnapshotHeader() (*snapshotHeader, error) {
// Create parent header.
2015-03-01 14:06:25 +00:00
sh := &snapshotHeader{}
2015-01-28 06:09:50 +00:00
// Append topics.
for _, t := range b.topics {
2015-03-01 14:06:25 +00:00
// 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
2015-01-28 06:09:50 +00:00
}
}
// Append topic to the snapshot.
2015-03-01 14:06:25 +00:00
sh.Topics = append(sh.Topics, st)
2015-01-28 06:09:50 +00:00
}
2015-03-01 14:06:25 +00:00
return sh, nil
2015-01-28 06:09:50 +00:00
}
// U/RL returns the connection url for the broker.
2014-12-31 19:42:53 +00:00
func (b *Broker) URL() *url.URL {
return b.log.URL
}
2015-01-28 06:09:50 +00:00
// LeaderURL returns the connection url for the leader broker.
func (b *Broker) LeaderURL() *url.URL {
_, u := b.log.Leader()
return u
}
// IsLeader returns true if the broker is the current leader.
func (b *Broker) IsLeader() bool { return b.log.State() == raft.Leader }
2014-10-03 03:13:42 +00:00
// Initialize creates a new cluster.
func (b *Broker) Initialize() error {
if err := b.log.Initialize(); err != nil {
return fmt.Errorf("raft: %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)
}
return nil
}
2014-10-24 04:22:52 +00:00
// Publish writes a message.
2014-10-03 03:13:42 +00:00
// Returns the index of the message. Otherwise returns an error.
2014-10-24 04:22:52 +00:00
func (b *Broker) Publish(m *Message) (uint64, error) {
buf, err := m.MarshalBinary()
assert(err == nil, "marshal binary error: %s", err)
2014-10-17 15:53:10 +00:00
return b.log.Apply(buf)
2014-10-12 18:05:03 +00:00
}
2014-10-24 04:22:52 +00:00
// 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)
2014-10-12 18:05:03 +00:00
if err != nil {
return err
}
2014-10-24 04:22:52 +00:00
// Wait for message to apply.
if err := b.Sync(index); err != nil {
return err
}
return nil
2014-10-03 03:13:42 +00:00
}
2014-10-24 04:22:52 +00:00
// Sync pauses until the given index has been applied.
2015-02-25 03:32:20 +00:00
func (b *Broker) Sync(index uint64) error { return b.log.Wait(index) }
2014-10-03 03:13:42 +00:00
2015-03-01 14:06:25 +00:00
// initializes a new topic object. Requires lock.
2015-02-25 03:32:20 +00:00
func (b *Broker) newTopic(id uint64) *topic {
t := &topic{id: id, path: b.topicPath(id)}
2014-10-12 18:05:03 +00:00
b.topics[t.id] = t
2014-10-24 04:22:52 +00:00
return t
2014-10-12 18:05:03 +00:00
}
2015-03-01 14:06:25 +00:00
// creates and opens a topic if it doesn't already exist. Requires lock.
func (b *Broker) createTopicIfNotExists(id uint64) (*topic, error) {
2014-10-24 04:22:52 +00:00
if t := b.topics[id]; t != nil {
2015-03-01 14:06:25 +00:00
return t, nil
2014-10-12 18:05:03 +00:00
}
2015-01-28 08:18:35 +00:00
2015-03-01 14:06:25 +00:00
// Open topic.
t := b.newTopic(id)
2015-03-01 14:06:25 +00:00
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())
}
2015-01-28 08:18:35 +00:00
return t
2014-10-12 18:05:03 +00:00
}
// 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()
2014-10-24 04:22:52 +00:00
// Exit if the broker is closed.
if !b.opened() {
return nil, ErrClosed
2014-10-24 04:22:52 +00:00
}
2014-10-15 03:42:40 +00:00
// Return new reader.
r := &topicReader{
path: b.topicPath(topicID),
index: index,
streaming: streaming,
2014-10-15 03:42:40 +00:00
}
return r, nil
2014-10-12 18:05:03 +00:00
}
2014-10-03 03:13:42 +00:00
// 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),
2014-10-12 18:05:03 +00:00
})
return err
2014-10-12 18:05:03 +00:00
}
func (b *Broker) mustApplySetTopicMaxIndex(m *Message) {
topicID, index := unmarshalTopicIndex(m.Data)
2014-10-12 18:05:03 +00:00
b.mu.Lock()
defer b.mu.Unlock()
// Ignore if the topic doesn't exist or the index is already higher.
t := b.topics[topicID]
if t == nil || t.index >= index {
return
2014-10-12 18:05:03 +00:00
}
t.index = index
2014-10-12 18:05:03 +00:00
}
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
2014-10-03 03:13:42 +00:00
}
func unmarshalTopicIndex(b []byte) (topicID, index uint64) {
topicID = binary.BigEndian.Uint64(b[0:8])
index = binary.BigEndian.Uint64(b[8:16])
return
2015-03-01 14:06:25 +00:00
}
// Truncate removes log segments that have been replicated to all subscribed replicas.
func (b *Broker) Truncate() 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]
}
2015-03-01 14:06:25 +00:00
// 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
}
2015-03-01 14:06:25 +00:00
// 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)
}
2015-03-01 14:06:25 +00:00
}
}
*/
2015-03-01 14:06:25 +00:00
return nil
}
2014-10-03 03:13:42 +00:00
// 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) {
2014-10-03 03:13:42 +00:00
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
2014-10-03 03:13:42 +00:00
}
m.Index = e.Index
// 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())
}
2014-10-04 17:27:12 +00:00
}
2015-01-10 16:08:00 +00:00
// Save highest applied index in memory.
// Only internal messages need to have their indexes saved to disk.
2015-01-10 16:08:00 +00:00
b.index = e.Index
2014-10-04 17:27:12 +00:00
}
// 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)
}
2014-10-03 03:13:42 +00:00
// Index returns the highest index that the broker has seen.
func (fsm *brokerFSM) Index() (uint64, error) {
b := (*Broker)(fsm)
return b.index, nil
2014-10-03 03:13:42 +00:00
}
// Snapshot streams the current state of the broker and returns the index.
func (fsm *brokerFSM) Snapshot(w io.Writer) (uint64, error) {
b := (*Broker)(fsm)
2014-10-03 03:13:42 +00:00
// TODO: Prevent truncation during snapshot.
// Calculate header under lock.
b.mu.RLock()
2015-01-28 06:09:50 +00:00
hdr, err := b.createSnapshotHeader()
b.mu.RUnlock()
if err != nil {
return 0, fmt.Errorf("create snapshot: %s", err)
}
// Encode snapshot header.
2015-01-10 16:08:00 +00:00
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.
2015-01-10 16:08:00 +00:00
for _, t := range hdr.Topics {
2015-03-01 14:06:25 +00:00
for _, s := range t.Segments {
if _, err := copyFileN(w, s.path, s.Size); err != nil {
return 0, err
}
}
}
2015-01-10 16:08:00 +00:00
// Return the snapshot and its last applied index.
2015-03-01 14:06:25 +00:00
return hdr.Index, nil
}
2014-10-03 03:13:42 +00:00
// Restore reads the broker state.
func (fsm *brokerFSM) Restore(r io.Reader) error {
b := (*Broker)(fsm)
b.mu.Lock()
defer b.mu.Unlock()
// 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.
2015-03-01 14:06:25 +00:00
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.
2015-03-01 14:06:25 +00:00
for _, st := range sh.Topics {
2015-02-25 03:32:20 +00:00
t := b.newTopic(st.ID)
// Remove existing file if it exists.
2015-03-01 14:06:25 +00:00
if err := os.RemoveAll(t.path); err != nil && !os.IsNotExist(err) {
return err
}
2015-03-01 14:06:25 +00:00
// 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)
}
}
2014-10-03 03:13:42 +00:00
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)
}
2015-01-10 16:08:00 +00:00
// snapshotHeader represents the header of a snapshot.
type snapshotHeader struct {
Topics []*snapshotTopic `json:"topics"`
Index uint64 `json:"index"`
}
type snapshotTopic struct {
2015-03-01 14:06:25 +00:00
ID uint64 `json:"id"`
Segments []*snapshotTopicSegment `json:"segments"`
}
type snapshotTopicSegment struct {
Index uint64 `json:"index"`
Size int64 `json:"size"`
path string
}
// topic represents a single named queue of messages.
2014-10-03 03:13:42 +00:00
// Each topic is identified by a unique path.
2015-02-25 03:32:20 +00:00
//
// Topics write their entries to segmented log files which contain a
// contiguous range of entries.
2014-10-03 03:13:42 +00:00
type topic struct {
2015-02-25 03:32:20 +00:00
id uint64 // unique identifier
index uint64 // highest index replicated
2015-02-25 03:32:20 +00:00
path string // on-disk path
segments segments // list of available segments
2014-10-03 03:13:42 +00:00
}
2015-02-25 03:32:20 +00:00
// 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))
}
2014-10-12 18:05:03 +00:00
// open opens a topic for writing.
func (t *topic) open() error {
2015-03-01 14:06:25 +00:00
assert(len(t.segments) == 0, "topic already open: %d", t.id)
2014-10-12 18:05:03 +00:00
// Ensure the parent directory exists.
2015-02-25 03:32:20 +00:00
if err := os.MkdirAll(t.path, 0700); err != nil {
2014-10-12 18:05:03 +00:00
return err
}
2015-02-25 03:32:20 +00:00
// Read available segments.
if err := t.loadSegments(); err != nil {
return fmt.Errorf("read segments: %s", err)
}
2014-10-03 03:13:42 +00:00
return nil
}
2015-02-25 03:32:20 +00:00
// 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.
2014-10-12 18:05:03 +00:00
func (t *topic) Close() error {
2015-03-01 14:06:25 +00:00
for _, s := range t.segments {
_ = s.close()
2014-10-12 18:05:03 +00:00
}
return nil
2014-10-04 17:27:12 +00:00
}
// 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.
2015-02-25 03:32:20 +00:00
f, err := os.Open(t.segments.last().path)
if os.IsNotExist(err) {
return 0, nil
} else if err != nil {
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 index, nil
} else if err != nil {
return index, fmt.Errorf("decode: %s", err)
}
// Update the topic's highest index.
index = m.Index
}
}
2015-03-01 14:06:25 +00:00
// 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 {
2014-10-12 18:05:03 +00:00
// TODO: If index is too old then return an error.
2015-02-25 03:32:20 +00:00
// 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.
if err := t.writeSegmentTo(w, index, s); err != nil {
2015-02-25 03:32:20 +00:00
return fmt.Errorf("write segment(%d/%d): %s", t.id, s.index, err)
}
}
return nil
}
func (t *topic) writeSegmentTo(w io.Writer, index uint64, segment *segment) error {
2015-02-25 03:32:20 +00:00
// Open segment for reading.
2014-11-13 05:32:42 +00:00
// If it doesn't exist then just exit immediately.
2015-02-25 03:32:20 +00:00
f, err := os.Open(segment.path)
2014-11-13 05:32:42 +00:00
if os.IsNotExist(err) {
2015-02-25 03:32:20 +00:00
return nil
2014-11-13 05:32:42 +00:00
} else if err != nil {
2015-02-25 03:32:20 +00:00
return err
2014-10-12 18:05:03 +00:00
}
defer func() { _ = f.Close() }()
// Stream out all messages until EOF.
dec := NewMessageDecoder(bufio.NewReader(f))
2014-10-12 18:05:03 +00:00
for {
// Decode message.
var m Message
if err := dec.Decode(&m); err == io.EOF {
break
} else if err != nil {
2015-02-25 03:32:20 +00:00
return fmt.Errorf("decode: %s", err)
2014-10-12 18:05:03 +00:00
}
// Ignore message if it's on or before high water mark.
if m.Index <= index {
continue
}
// Write message out to stream.
_, err := m.WriteTo(w)
if err != nil {
2015-02-25 03:32:20 +00:00
return fmt.Errorf("write to: %s", err)
2014-10-12 18:05:03 +00:00
}
}
2015-02-25 03:32:20 +00:00
return nil
2014-10-12 18:05:03 +00:00
}
// encode writes a message to the end of the topic.
func (t *topic) encode(m *Message) error {
2015-03-01 14:06:25 +00:00
// 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)
}
}
2014-10-12 18:05:03 +00:00
// Encode message.
b := make([]byte, messageHeaderSize+len(m.Data))
2014-10-17 15:53:10 +00:00
copy(b, m.marshalHeader())
2014-10-12 18:05:03 +00:00
copy(b[messageHeaderSize:], m.Data)
2015-03-01 14:06:25 +00:00
// Write to segment.
if _, err := s.file.Write(b); err != nil {
return fmt.Errorf("write segment: %s", err)
2014-10-12 18:05:03 +00:00
}
2014-10-04 17:27:12 +00:00
return nil
}
2015-02-25 03:32:20 +00:00
// 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.
2015-03-01 14:06:25 +00:00
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
2015-02-25 03:32:20 +00:00
}
// segments represents a list of segments sorted by index.
type segments []*segment
2015-03-01 14:06:25 +00:00
// 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]
}
2015-02-25 03:32:20 +00:00
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 {
mu sync.Mutex
path string // topic directory path
index uint64 // starting index
streaming bool // true if reader should wait indefinitely
segment uint64 // current segment index
file *os.File // current segment file handler
closed bool
2014-10-24 04:22:52 +00:00
}
// 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)
}
// 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
}
2014-10-24 04:22:52 +00:00
}
2014-10-12 18:05:03 +00:00
}
// 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()
// Exit if closed.
if r.closed {
return errors.New("topic reader closed")
2014-10-15 03:42:40 +00:00
}
// 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)
}
2014-10-12 18:05:03 +00:00
// 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)
}
2014-10-04 17:27:12 +00:00
// 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
2014-10-17 04:11:28 +00:00
}
return r.file
}
// 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
}
2014-10-12 18:05:03 +00:00
}
}
2014-10-12 18:05:03 +00:00
// 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()
//
2014-10-04 17:27:12 +00:00
}
// Close closes the reader.
func (r *topicReader) Close() error {
r.mu.Lock()
defer r.mu.Unlock()
2014-10-12 18:05:03 +00:00
// Close current handle.
if r.file != nil {
_ = r.file.Close()
r.file = nil
}
2014-10-12 18:05:03 +00:00
// Mark reader as closed.
r.closed = true
2014-10-12 18:05:03 +00:00
return
2014-10-12 18:05:03 +00:00
}
2014-10-15 03:42:40 +00:00
// MessageType represents the type of message.
type MessageType uint16
// BrokerMessageType is a flag set on broker messages to prevent them
// from being passed through to topics.
const BrokerMessageType = 0x8000
2014-10-15 03:42:40 +00:00
const (
InternalMessageType = BrokerMessageType | MessageType(0x00)
SetTopicMaxIndexMessageType = BrokerMessageType | MessageType(0x01)
2014-10-15 03:42:40 +00:00
)
// The size of the encoded message header, in bytes.
const messageHeaderSize = 2 + 8 + 8 + 4
2014-10-15 03:42:40 +00:00
// Message represents a single item in a topic.
type Message struct {
2014-10-17 15:53:10 +00:00
Type MessageType
TopicID uint64
2014-10-17 15:53:10 +00:00
Index uint64
Data []byte
2014-10-15 03:42:40 +00:00
}
// WriteTo encodes and writes the message to a writer. Implements io.WriterTo.
func (m *Message) WriteTo(w io.Writer) (n int64, err error) {
2014-10-17 15:53:10 +00:00
if n, err := w.Write(m.marshalHeader()); err != nil {
return int64(n), err
2014-10-15 03:42:40 +00:00
}
if n, err := w.Write(m.Data); err != nil {
return int64(messageHeaderSize + n), err
2014-10-15 03:42:40 +00:00
}
return int64(messageHeaderSize + len(m.Data)), nil
2014-10-15 03:42:40 +00:00
}
2014-10-17 15:53:10 +00:00
// MarshalBinary returns a binary representation of the message.
// This implements encoding.BinaryMarshaler. An error cannot be returned.
func (m *Message) MarshalBinary() ([]byte, error) {
b := make([]byte, messageHeaderSize+len(m.Data))
copy(b, m.marshalHeader())
copy(b[messageHeaderSize:], m.Data)
return b, nil
}
// UnmarshalBinary reads a message from a binary encoded slice.
// This implements encoding.BinaryUnmarshaler.
func (m *Message) UnmarshalBinary(b []byte) error {
m.unmarshalHeader(b)
if len(b[messageHeaderSize:]) < len(m.Data) {
return fmt.Errorf("message data too short: %d < %d", len(b[messageHeaderSize:]), len(m.Data))
}
copy(m.Data, b[messageHeaderSize:])
return nil
}
// marshalHeader returns a byte slice with the message header.
func (m *Message) marshalHeader() []byte {
2014-10-15 03:42:40 +00:00
b := make([]byte, messageHeaderSize)
binary.BigEndian.PutUint16(b[0:2], uint16(m.Type))
binary.BigEndian.PutUint64(b[2:10], m.TopicID)
binary.BigEndian.PutUint64(b[10:18], m.Index)
binary.BigEndian.PutUint32(b[18:22], uint32(len(m.Data)))
2014-10-15 03:42:40 +00:00
return b
}
2014-10-17 15:53:10 +00:00
// unmarshalHeader reads message header data from binary encoded slice.
// The data field is appropriately sized but is not filled.
func (m *Message) unmarshalHeader(b []byte) {
m.Type = MessageType(binary.BigEndian.Uint16(b[0:2]))
m.TopicID = binary.BigEndian.Uint64(b[2:10])
m.Index = binary.BigEndian.Uint64(b[10:18])
m.Data = make([]byte, binary.BigEndian.Uint32(b[18:22]))
2014-10-17 15:53:10 +00:00
}
2014-10-15 03:42:40 +00:00
// MessageDecoder decodes messages from a reader.
type MessageDecoder struct {
r io.Reader
}
// NewMessageDecoder returns a new instance of the MessageDecoder.
func NewMessageDecoder(r io.Reader) *MessageDecoder {
return &MessageDecoder{r: r}
}
// Decode reads a message from the decoder's reader.
func (dec *MessageDecoder) Decode(m *Message) error {
2014-10-17 15:53:10 +00:00
// Read header bytes.
2014-10-15 03:42:40 +00:00
var b [messageHeaderSize]byte
if _, err := io.ReadFull(dec.r, b[:]); err != nil {
return err
}
2014-10-17 15:53:10 +00:00
m.unmarshalHeader(b[:])
2014-10-15 03:42:40 +00:00
// Read data.
2014-10-17 15:53:10 +00:00
if _, err := io.ReadFull(dec.r, m.Data); err != nil {
2014-10-15 03:42:40 +00:00
return err
}
return nil
}
2014-10-17 04:11:28 +00:00
type flusher interface {
Flush()
}
// uint64Slice attaches the methods of Interface to []int, sorting in increasing order.
type uint64Slice []uint64
2014-10-24 04:22:52 +00:00
func (p uint64Slice) Len() int { return len(p) }
func (p uint64Slice) Less(i, j int) bool { return p[i] < p[j] }
func (p uint64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] }
2014-10-24 04:22:52 +00:00
2014-11-05 05:32:17 +00:00
// mustMarshalJSON encodes a value to JSON.
2014-10-24 04:22:52 +00:00
// This will panic if an error occurs. This should only be used internally when
// an invalid marshal will cause corruption and a panic is appropriate.
2014-11-05 05:32:17 +00:00
func mustMarshalJSON(v interface{}) []byte {
2014-10-12 18:05:03 +00:00
b, err := json.Marshal(v)
2014-10-24 04:22:52 +00:00
if err != nil {
panic("marshal: " + err.Error())
}
2014-10-12 18:05:03 +00:00
return b
2014-10-04 17:27:12 +00:00
}
2014-11-05 05:32:17 +00:00
// mustUnmarshalJSON decodes a value from JSON.
2014-10-24 04:22:52 +00:00
// This will panic if an error occurs. This should only be used internally when
// an invalid unmarshal will cause corruption and a panic is appropriate.
2014-11-05 05:32:17 +00:00
func mustUnmarshalJSON(b []byte, v interface{}) {
2014-10-24 04:22:52 +00:00
if err := json.Unmarshal(b, v); err != nil {
panic("unmarshal: " + err.Error())
}
}
2014-10-03 03:13:42 +00:00
// assert will panic with a given formatted message if the given condition is false.
func assert(condition bool, msg string, v ...interface{}) {
if !condition {
panic(fmt.Sprintf("assert failed: "+msg, v...))
}
}
2014-10-12 18:05:03 +00:00
// 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) }
2014-10-12 18:05:03 +00:00
func warn(v ...interface{}) { fmt.Fprintln(os.Stderr, v...) }
func warnf(msg string, v ...interface{}) { fmt.Fprintf(os.Stderr, msg+"\n", v...) }