influxdb/server.go

1481 lines
37 KiB
Go
Raw Normal View History

2014-10-22 05:32:19 +00:00
package influxdb
import (
2014-10-24 00:54:12 +00:00
"encoding/json"
2014-10-22 05:32:19 +00:00
"fmt"
2014-12-29 23:12:51 +00:00
"net/url"
2014-10-24 05:38:03 +00:00
"os"
2014-11-05 05:32:17 +00:00
"path/filepath"
2014-12-23 06:18:05 +00:00
"regexp"
2014-10-27 23:31:45 +00:00
"sort"
"strconv"
2014-10-22 05:32:19 +00:00
"sync"
"time"
2014-12-23 06:18:05 +00:00
"code.google.com/p/go.crypto/bcrypt"
2014-10-22 05:32:19 +00:00
"github.com/influxdb/influxdb/messaging"
)
2014-10-30 00:21:17 +00:00
const (
// DefaultRootPassword is the password initially set for the root user.
// It is also used when reseting the root user's password.
DefaultRootPassword = "root"
// DefaultRetentionPolicyName is the name of a databases's default shard space.
DefaultRetentionPolicyName = "default"
2014-10-30 00:21:17 +00:00
// DefaultSplitN represents the number of partitions a shard is split into.
DefaultSplitN = 1
// DefaultReplicaN represents the number of replicas data is written to.
DefaultReplicaN = 1
// DefaultShardDuration is the time period held by a shard.
DefaultShardDuration = 7 * (24 * time.Hour)
// DefaultShardRetention is the length of time before a shard is dropped.
DefaultShardRetention = time.Duration(0)
)
2014-10-24 23:45:02 +00:00
const (
2014-12-30 15:50:15 +00:00
// Data node messages
createDataNodeMessageType = messaging.MessageType(0x00)
deleteDataNodeMessageType = messaging.MessageType(0x01)
2014-12-23 06:18:05 +00:00
2014-12-29 23:12:51 +00:00
// Database messages
createDatabaseMessageType = messaging.MessageType(0x10)
deleteDatabaseMessageType = messaging.MessageType(0x11)
2014-12-23 06:18:05 +00:00
2014-12-29 23:12:51 +00:00
// Retention policy messages
createRetentionPolicyMessageType = messaging.MessageType(0x20)
updateRetentionPolicyMessageType = messaging.MessageType(0x21)
deleteRetentionPolicyMessageType = messaging.MessageType(0x22)
setDefaultRetentionPolicyMessageType = messaging.MessageType(0x23)
2014-12-23 06:18:05 +00:00
2014-12-29 23:12:51 +00:00
// User messages
createUserMessageType = messaging.MessageType(0x30)
updateUserMessageType = messaging.MessageType(0x31)
deleteUserMessageType = messaging.MessageType(0x32)
2014-12-23 06:18:05 +00:00
2014-12-29 23:12:51 +00:00
// Shard messages
createShardIfNotExistsMessageType = messaging.MessageType(0x40)
2014-12-29 23:12:51 +00:00
// Series messages
createSeriesIfNotExistsMessageType = messaging.MessageType(0x50)
// Write raw data messages (per-topic)
writeSeriesMessageType = messaging.MessageType(0x80)
2014-10-24 23:45:02 +00:00
)
2014-10-22 05:32:19 +00:00
// Server represents a collection of metadata and raw metric data.
type Server struct {
2014-10-24 05:38:03 +00:00
mu sync.RWMutex
2014-12-29 23:12:51 +00:00
id uint64
2014-10-24 05:38:03 +00:00
path string
done chan struct{} // goroutine close notification
2014-10-24 23:45:02 +00:00
client MessagingClient // broker client
index uint64 // highest broadcast index seen
errors map[uint64]error // message errors
2014-10-24 00:54:12 +00:00
2014-11-05 05:32:17 +00:00
meta *metastore // metadata store
2014-12-30 15:50:15 +00:00
dataNodes map[uint64]*DataNode // data nodes by id
2014-12-29 23:12:51 +00:00
2014-12-23 06:18:05 +00:00
databases map[string]*database // databases by name
databasesByShard map[uint64]*database // databases by shard id
users map[string]*User // user by name
2014-10-22 05:32:19 +00:00
}
// NewServer returns a new instance of Server.
2014-12-30 22:46:50 +00:00
func NewServer() *Server {
2014-10-22 05:32:19 +00:00
return &Server{
meta: &metastore{},
2014-12-30 15:50:15 +00:00
dataNodes: make(map[uint64]*DataNode),
2014-12-23 06:18:05 +00:00
databases: make(map[string]*database),
databasesByShard: make(map[uint64]*database),
users: make(map[string]*User),
errors: make(map[uint64]error),
2014-10-22 05:32:19 +00:00
}
}
2014-12-30 22:46:50 +00:00
// ID returns the data node id for the server.
// Returns zero if the server is closed or the server has not joined a cluster.
func (s *Server) ID() uint64 {
s.mu.Lock()
defer s.mu.Unlock()
return s.id
}
2014-10-23 04:21:48 +00:00
// Path returns the path used when opening the server.
// Returns an empty string when the server is closed.
2014-12-30 22:46:50 +00:00
func (s *Server) Path() string {
s.mu.Lock()
defer s.mu.Unlock()
return s.path
}
2014-10-23 04:21:48 +00:00
// shardPath returns the path for a shard.
func (s *Server) shardPath(id uint64) string {
if s.path == "" {
return ""
}
return filepath.Join(s.path, "shards", strconv.FormatUint(id, 10))
}
2014-10-22 05:32:19 +00:00
// Open initializes the server from a given path.
func (s *Server) Open(path string) error {
2014-10-24 00:54:12 +00:00
// Ensure the server isn't already open and there's a path provided.
2014-10-23 04:21:48 +00:00
if s.opened() {
return ErrServerOpen
} else if path == "" {
return ErrPathRequired
}
// Create required directories.
2014-11-05 05:32:17 +00:00
if err := os.MkdirAll(path, 0700); err != nil {
return err
}
if err := os.MkdirAll(filepath.Join(path, "shards"), 0700); err != nil {
return err
}
2014-11-05 05:32:17 +00:00
// Open metadata store.
if err := s.meta.open(filepath.Join(path, "meta")); err != nil {
return fmt.Errorf("meta: %s", err)
}
// Load state from metastore.
if err := s.load(); err != nil {
return fmt.Errorf("load: %s", err)
}
2014-10-24 05:38:03 +00:00
// Set the server path.
s.path = path
2014-10-22 05:32:19 +00:00
return nil
}
2014-10-23 04:21:48 +00:00
// opened returns true when the server is open.
func (s *Server) opened() bool { return s.path != "" }
2014-10-22 05:32:19 +00:00
// Close shuts down the server.
func (s *Server) Close() error {
2014-10-24 00:54:12 +00:00
s.mu.Lock()
2014-10-24 05:38:03 +00:00
defer s.mu.Unlock()
2014-10-24 00:54:12 +00:00
if !s.opened() {
return ErrServerClosed
}
2014-12-30 22:46:50 +00:00
// Close message processing.
s.setClient(nil)
2014-10-24 00:54:12 +00:00
2014-11-05 05:32:17 +00:00
// Close metastore.
_ = s.meta.close()
2014-10-24 00:54:12 +00:00
// Remove path.
s.path = ""
2014-10-22 05:32:19 +00:00
return nil
}
2014-11-05 05:32:17 +00:00
// load reads the state of the server from the metastore.
func (s *Server) load() error {
return s.meta.view(func(tx *metatx) error {
2014-12-29 23:12:51 +00:00
// Read server id.
s.id = tx.id()
2014-11-05 05:32:17 +00:00
// Load databases.
2014-12-23 06:18:05 +00:00
s.databases = make(map[string]*database)
2014-11-05 05:32:17 +00:00
for _, db := range tx.databases() {
s.databases[db.name] = db
2014-12-03 16:32:53 +00:00
for sh := range db.shards {
s.databasesByShard[sh] = db
}
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// Load users.
s.users = make(map[string]*User)
for _, u := range tx.users() {
s.users[u.Name] = u
2014-11-05 05:32:17 +00:00
}
return nil
})
}
2014-12-30 22:46:50 +00:00
// Client retrieves the current messaging client.
func (s *Server) Client() MessagingClient {
s.mu.RLock()
defer s.mu.RUnlock()
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 broker.
if client != nil {
s.done = make(chan struct{}, 0)
go s.processor(client, s.done)
}
return nil
}
2014-10-24 05:38:03 +00:00
// 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.
2014-10-24 00:54:12 +00:00
// Returns the broker log index of the message or an error.
2014-10-24 05:38:03 +00:00
func (s *Server) broadcast(typ messaging.MessageType, c interface{}) (uint64, error) {
2014-10-24 00:54:12 +00:00
// Encode the command.
2014-10-24 05:38:03 +00:00
data, err := json.Marshal(c)
2014-10-24 00:54:12 +00:00
if err != nil {
return 0, err
}
// Publish the message.
2014-10-24 05:38:03 +00:00
m := &messaging.Message{
Type: typ,
TopicID: messaging.BroadcastTopicID,
Data: data,
}
index, err := s.client.Publish(m)
2014-10-24 00:54:12 +00:00
if err != nil {
2014-10-24 05:38:03 +00:00
return 0, err
2014-10-24 00:54:12 +00:00
}
2014-10-24 05:38:03 +00:00
// Wait for the server to receive the message.
2014-10-24 23:45:02 +00:00
err = s.sync(index)
2014-10-24 05:38:03 +00:00
2014-10-24 23:45:02 +00:00
return index, err
2014-10-24 05:38:03 +00:00
}
// sync blocks until a given index (or a higher index) has been seen.
2014-10-24 23:45:02 +00:00
// Returns any error associated with the command.
func (s *Server) sync(index uint64) error {
2014-10-24 05:38:03 +00:00
for {
2014-10-24 23:45:02 +00:00
// Check if index has occurred. If so, retrieve the error and return.
s.mu.RLock()
if s.index >= index {
err, ok := s.errors[index]
if ok {
delete(s.errors, index)
}
s.mu.RUnlock()
return err
2014-10-24 05:38:03 +00:00
}
2014-10-24 23:45:02 +00:00
s.mu.RUnlock()
// Otherwise wait momentarily and check again.
2014-10-24 05:38:03 +00:00
time.Sleep(1 * time.Millisecond)
2014-10-24 00:54:12 +00:00
}
2014-10-24 05:38:03 +00:00
}
2014-10-24 00:54:12 +00:00
2014-12-31 19:42:53 +00:00
// Initialize creates a new data node and initializes the server's id to 1.
func (s *Server) Initialize(u *url.URL) error {
// Create a new data node.
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)
assert(n != nil && n.ID == 1, "invalid initial server id: %d", n.ID)
// Set the ID on the metastore.
if err := s.meta.mustUpdate(func(tx *metatx) error {
return tx.setID(n.ID)
}); err != nil {
return err
}
// Set the ID on the server.
s.id = 1
return nil
}
2014-12-30 15:50:15 +00:00
// DataNode returns a data node by id.
func (s *Server) DataNode(id uint64) *DataNode {
2014-12-29 23:12:51 +00:00
s.mu.RLock()
defer s.mu.RUnlock()
2014-12-30 15:50:15 +00:00
return s.dataNodes[id]
2014-12-29 23:12:51 +00:00
}
2014-12-30 15:50:15 +00:00
// DataNodeByURL returns a data node by url.
func (s *Server) DataNodeByURL(u *url.URL) *DataNode {
2014-12-29 23:12:51 +00:00
s.mu.RLock()
defer s.mu.RUnlock()
2014-12-30 15:50:15 +00:00
for _, n := range s.dataNodes {
if n.URL.String() == u.String() {
return n
}
}
return nil
2014-12-29 23:12:51 +00:00
}
2014-12-30 15:50:15 +00:00
// DataNodes returns a list of data nodes.
func (s *Server) DataNodes() (a []*DataNode) {
2014-12-29 23:12:51 +00:00
s.mu.RLock()
defer s.mu.RUnlock()
2014-12-30 15:50:15 +00:00
for _, n := range s.dataNodes {
2014-12-29 23:12:51 +00:00
a = append(a, n)
}
2014-12-30 15:50:15 +00:00
sort.Sort(dataNodes(a))
2014-12-29 23:12:51 +00:00
return
}
2014-12-30 15:50:15 +00:00
// CreateDataNode creates a new data node with a given URL.
func (s *Server) CreateDataNode(u *url.URL) error {
c := &createDataNodeCommand{URL: u.String()}
_, err := s.broadcast(createDataNodeMessageType, c)
2014-12-29 23:12:51 +00:00
return err
}
2014-12-30 15:50:15 +00:00
func (s *Server) applyCreateDataNode(m *messaging.Message) (err error) {
var c createDataNodeCommand
2014-12-29 23:12:51 +00:00
mustUnmarshalJSON(m.Data, &c)
s.mu.Lock()
defer s.mu.Unlock()
// Validate parameters.
if c.URL == "" {
2014-12-30 15:50:15 +00:00
return ErrDataNodeURLRequired
2014-12-29 23:12:51 +00:00
}
2014-12-30 15:50:15 +00:00
// Check that another node with the same URL doesn't already exist.
2014-12-29 23:12:51 +00:00
u, _ := url.Parse(c.URL)
2014-12-30 15:50:15 +00:00
for _, n := range s.dataNodes {
if n.URL.String() == u.String() {
return ErrDataNodeExists
}
2014-12-29 23:12:51 +00:00
}
2014-12-30 15:50:15 +00:00
// Create data node.
n := newDataNode()
2014-12-29 23:12:51 +00:00
n.URL = u
// Persist to metastore.
err = s.meta.mustUpdate(func(tx *metatx) error {
n.ID = tx.nextDataNodeID()
return tx.saveDataNode(n)
})
2014-12-29 23:12:51 +00:00
// Add to node on server.
2014-12-30 15:50:15 +00:00
s.dataNodes[n.ID] = n
2014-12-29 23:12:51 +00:00
return
}
2014-12-30 15:50:15 +00:00
type createDataNodeCommand struct {
2014-12-29 23:12:51 +00:00
URL string `json:"url"`
}
2014-12-30 15:50:15 +00:00
// DeleteDataNode deletes an existing data node.
func (s *Server) DeleteDataNode(id uint64) error {
c := &deleteDataNodeCommand{ID: id}
_, err := s.broadcast(deleteDataNodeMessageType, c)
2014-12-29 23:12:51 +00:00
return err
}
2014-12-30 15:50:15 +00:00
func (s *Server) applyDeleteDataNode(m *messaging.Message) (err error) {
var c deleteDataNodeCommand
2014-12-29 23:12:51 +00:00
mustUnmarshalJSON(m.Data, &c)
s.mu.Lock()
defer s.mu.Unlock()
2014-12-30 15:50:15 +00:00
n := s.dataNodes[c.ID]
2014-12-29 23:12:51 +00:00
if n == nil {
2014-12-30 15:50:15 +00:00
return ErrDataNodeNotFound
2014-12-29 23:12:51 +00:00
}
// Remove from metastore.
2014-12-30 15:50:15 +00:00
err = s.meta.mustUpdate(func(tx *metatx) error { return tx.deleteDataNode(c.ID) })
2014-12-29 23:12:51 +00:00
// Delete the node.
2014-12-30 15:50:15 +00:00
delete(s.dataNodes, n.ID)
2014-12-29 23:12:51 +00:00
return
}
2014-12-30 15:50:15 +00:00
type deleteDataNodeCommand struct {
2014-12-29 23:12:51 +00:00
ID uint64 `json:"id"`
}
2014-12-23 06:18:05 +00:00
// DatabaseExists returns true if a database exists.
func (s *Server) DatabaseExists(name string) bool {
s.mu.RLock()
defer s.mu.RUnlock()
return s.databases[name] != nil
2014-10-24 00:54:12 +00:00
}
2014-12-23 06:18:05 +00:00
// Databases returns a sorted list of all database names.
func (s *Server) Databases() (a []string) {
s.mu.RLock()
defer s.mu.RUnlock()
2014-10-28 00:16:03 +00:00
for _, db := range s.databases {
2014-12-23 06:18:05 +00:00
a = append(a, db.name)
2014-10-28 00:16:03 +00:00
}
2014-12-23 06:18:05 +00:00
sort.Strings(a)
return
2014-10-28 00:16:03 +00:00
}
2014-10-24 00:54:12 +00:00
// CreateDatabase creates a new database.
func (s *Server) CreateDatabase(name string) error {
2014-10-25 17:54:23 +00:00
c := &createDatabaseCommand{Name: name}
_, err := s.broadcast(createDatabaseMessageType, c)
2014-10-24 05:38:03 +00:00
return err
2014-10-24 00:54:12 +00:00
}
2014-12-23 06:18:05 +00:00
func (s *Server) applyCreateDatabase(m *messaging.Message) (err error) {
2014-10-24 05:38:03 +00:00
var c createDatabaseCommand
2014-11-05 05:32:17 +00:00
mustUnmarshalJSON(m.Data, &c)
2014-10-24 05:38:03 +00:00
2014-10-24 00:54:12 +00:00
s.mu.Lock()
defer s.mu.Unlock()
if s.databases[c.Name] != nil {
2014-10-24 23:45:02 +00:00
return ErrDatabaseExists
2014-10-24 00:54:12 +00:00
}
// Create database entry.
2014-12-23 06:18:05 +00:00
db := newDatabase()
2014-10-25 04:38:01 +00:00
db.name = c.Name
2014-11-05 05:32:17 +00:00
// Persist to metastore.
2014-12-23 06:18:05 +00:00
err = s.meta.mustUpdate(func(tx *metatx) error { return tx.saveDatabase(db) })
2014-11-05 05:32:17 +00:00
// Add to databases on server.
2014-10-25 04:38:01 +00:00
s.databases[c.Name] = db
2014-11-05 05:32:17 +00:00
2014-12-23 06:18:05 +00:00
return
2014-10-24 23:45:02 +00:00
}
type createDatabaseCommand struct {
Name string `json:"name"`
}
// DeleteDatabase deletes an existing database.
func (s *Server) DeleteDatabase(name string) error {
2014-10-25 17:54:23 +00:00
c := &deleteDatabaseCommand{Name: name}
_, err := s.broadcast(deleteDatabaseMessageType, c)
2014-10-24 23:45:02 +00:00
return err
}
2014-12-23 06:18:05 +00:00
func (s *Server) applyDeleteDatabase(m *messaging.Message) (err error) {
2014-10-24 23:45:02 +00:00
var c deleteDatabaseCommand
2014-11-05 05:32:17 +00:00
mustUnmarshalJSON(m.Data, &c)
2014-10-24 23:45:02 +00:00
s.mu.Lock()
defer s.mu.Unlock()
if s.databases[c.Name] == nil {
return ErrDatabaseNotFound
}
2014-11-05 05:32:17 +00:00
// Remove from metastore.
2014-12-23 06:18:05 +00:00
err = s.meta.mustUpdate(func(tx *metatx) error { return tx.deleteDatabase(c.Name) })
2014-11-05 05:32:17 +00:00
2014-10-24 23:45:02 +00:00
// Delete the database entry.
delete(s.databases, c.Name)
2014-12-23 06:18:05 +00:00
return
2014-10-24 23:45:02 +00:00
}
type deleteDatabaseCommand struct {
Name string `json:"name"`
2014-10-24 00:54:12 +00:00
}
2014-12-23 06:18:05 +00:00
// shardByTimestamp returns a shard that owns a given timestamp for a database.
func (s *Server) shardByTimestamp(database, policy string, id uint32, timestamp time.Time) (*Shard, error) {
db := s.databases[database]
if db == nil {
return nil, ErrDatabaseNotFound
}
return db.shardByTimestamp(policy, id, timestamp)
}
// Shards returns a list of all shards for a database.
// Returns an error if the database doesn't exist.
func (s *Server) Shards(database string) ([]*Shard, error) {
s.mu.RLock()
defer s.mu.RUnlock()
// Lookup database.
db := s.databases[database]
if db == nil {
return nil, ErrDatabaseNotFound
}
// Retrieve shards from database.
shards := make([]*Shard, 0, len(db.shards))
for _, shard := range db.shards {
shards = append(shards, shard)
}
return shards, nil
}
// shardsByTimestamp returns all shards that own a given timestamp for a database.
func (s *Server) shardsByTimestamp(database, policy string, timestamp time.Time) ([]*Shard, error) {
db := s.databases[database]
if db == nil {
return nil, ErrDatabaseNotFound
}
return db.shardsByTimestamp(policy, timestamp)
}
// CreateShardsIfNotExist creates all the shards for a retention policy for the interval a timestamp falls into.
// Note that multiple shards can be created for each bucket of time.
func (s *Server) CreateShardsIfNotExists(database, policy string, timestamp time.Time) error {
c := &createShardIfNotExistsCommand{Database: database, Policy: policy, Timestamp: timestamp}
_, err := s.broadcast(createShardIfNotExistsMessageType, c)
return err
}
// createShardIfNotExists returns the shard for a given retention policy, series, and timestamp.
// If it doesn't exist, it will create all shards for the given timestamp
func (s *Server) createShardIfNotExists(database, policy string, id uint32, timestamp time.Time) (*Shard, error) {
// Check if shard exists first.
sh, err := s.shardByTimestamp(database, policy, id, timestamp)
if err != nil {
return nil, err
} else if sh != nil {
return sh, nil
}
// If the shard doesn't exist then create it.
if err := s.CreateShardsIfNotExists(database, policy, timestamp); err != nil {
return nil, err
}
// Lookup the shard again.
return s.shardByTimestamp(database, policy, id, timestamp)
}
func (s *Server) applyCreateShardIfNotExists(m *messaging.Message) (err error) {
var c createShardIfNotExistsCommand
mustUnmarshalJSON(m.Data, &c)
s.mu.Lock()
defer s.mu.Unlock()
2014-12-23 06:18:05 +00:00
// Retrieve database.
db := s.databases[c.Database]
if s.databases[c.Database] == nil {
return ErrDatabaseNotFound
}
2014-12-23 06:18:05 +00:00
// Validate retention policy.
rp := db.policies[c.Policy]
if rp == nil {
return ErrRetentionPolicyNotFound
}
2014-12-23 06:18:05 +00:00
// If we can match to an existing shard date range then just ignore request.
for _, sh := range rp.Shards {
2014-12-23 15:47:32 +00:00
if timeBetweenInclusive(c.Timestamp, sh.StartTime, sh.EndTime) {
2014-12-23 06:18:05 +00:00
return nil
}
}
// If no shards match then create a new one.
sh := newShard()
sh.ID = m.Index
sh.StartTime = c.Timestamp.Truncate(rp.Duration).UTC()
sh.EndTime = sh.StartTime.Add(rp.Duration).UTC()
// Open shard.
if err := sh.open(s.shardPath(sh.ID)); err != nil {
panic("unable to open shard: " + err.Error())
}
// Persist to metastore if a shard was created.
2014-12-23 18:18:46 +00:00
if err = s.meta.mustUpdate(func(tx *metatx) error {
return tx.saveDatabase(db)
2014-12-23 18:18:46 +00:00
}); err != nil {
_ = sh.close()
return
}
2014-12-23 06:18:05 +00:00
// Add to lookups.
s.databasesByShard[sh.ID] = db
db.shards[sh.ID] = sh
rp.Shards = append(rp.Shards, sh)
2014-12-29 23:12:51 +00:00
// TODO: Subscribe to shard if it matches the server's index.
2014-12-23 18:18:46 +00:00
return
}
type createShardIfNotExistsCommand struct {
Database string `json:"name"`
Policy string `json:"policy"`
Timestamp time.Time `json:"timestamp"`
}
2014-12-23 06:18:05 +00:00
// User returns a user by username
// Returns nil if the user does not exist.
func (s *Server) User(name string) *User {
2014-10-28 23:54:49 +00:00
s.mu.Lock()
defer s.mu.Unlock()
2014-12-23 06:18:05 +00:00
return s.users[name]
2014-10-28 23:54:49 +00:00
}
2014-12-23 06:18:05 +00:00
// Users returns a list of all users, sorted by name.
func (s *Server) Users() (a []*User) {
s.mu.RLock()
defer s.mu.RUnlock()
for _, u := range s.users {
a = append(a, u)
}
2014-12-23 06:18:05 +00:00
sort.Sort(users(a))
return a
}
2014-12-23 06:18:05 +00:00
// CreateUser creates a user on the server.
func (s *Server) CreateUser(username, password string, admin bool) error {
c := &createUserCommand{Username: username, Password: password, Admin: admin}
_, err := s.broadcast(createUserMessageType, c)
2014-10-28 23:54:49 +00:00
return err
}
2014-12-23 06:18:05 +00:00
func (s *Server) applyCreateUser(m *messaging.Message) (err error) {
var c createUserCommand
2014-11-05 05:32:17 +00:00
mustUnmarshalJSON(m.Data, &c)
2014-10-28 23:54:49 +00:00
s.mu.Lock()
defer s.mu.Unlock()
2014-12-23 06:18:05 +00:00
// Validate user.
2014-10-28 23:54:49 +00:00
if c.Username == "" {
return ErrUsernameRequired
2014-12-23 06:18:05 +00:00
} else if s.users[c.Username] != nil {
return ErrUserExists
2014-10-28 23:54:49 +00:00
}
// Generate the hash of the password.
hash, err := HashPassword(c.Password)
if err != nil {
return err
}
2014-12-23 06:18:05 +00:00
// Create the user.
u := &User{
Name: c.Username,
Hash: string(hash),
Admin: c.Admin,
2014-10-28 23:54:49 +00:00
}
2014-11-05 05:32:17 +00:00
// Persist to metastore.
2014-12-23 06:18:05 +00:00
err = s.meta.mustUpdate(func(tx *metatx) error {
return tx.saveUser(u)
2014-11-05 05:32:17 +00:00
})
2014-12-23 06:18:05 +00:00
s.users[u.Name] = u
return
2014-10-28 23:54:49 +00:00
}
2014-12-23 06:18:05 +00:00
type createUserCommand struct {
2014-10-28 23:54:49 +00:00
Username string `json:"username"`
Password string `json:"password"`
2014-12-23 06:18:05 +00:00
Admin bool `json:"admin,omitempty"`
2014-10-28 23:54:49 +00:00
}
2014-12-23 06:18:05 +00:00
// UpdateUser updates an existing user on the server.
func (s *Server) UpdateUser(username, password string) error {
c := &updateUserCommand{Username: username, Password: password}
_, err := s.broadcast(updateUserMessageType, c)
2014-10-29 00:43:03 +00:00
return err
}
2014-12-23 06:18:05 +00:00
func (s *Server) applyUpdateUser(m *messaging.Message) (err error) {
var c updateUserCommand
2014-11-05 05:32:17 +00:00
mustUnmarshalJSON(m.Data, &c)
2014-10-29 00:43:03 +00:00
s.mu.Lock()
defer s.mu.Unlock()
2014-12-23 06:18:05 +00:00
// Validate command.
u := s.users[c.Username]
if u == nil {
return ErrUserNotFound
2014-10-29 00:43:03 +00:00
}
2014-12-23 06:18:05 +00:00
// Update the user's password, if set.
if c.Password != "" {
hash, err := HashPassword(c.Password)
if err != nil {
return err
}
u.Hash = string(hash)
}
2014-11-05 05:32:17 +00:00
2014-12-23 06:18:05 +00:00
// Persist to metastore.
return s.meta.mustUpdate(func(tx *metatx) error {
return tx.saveUser(u)
})
2014-10-29 00:43:03 +00:00
}
2014-12-23 06:18:05 +00:00
type updateUserCommand struct {
2014-10-29 00:43:03 +00:00
Username string `json:"username"`
2014-12-23 06:18:05 +00:00
Password string `json:"password,omitempty"`
2014-10-29 00:43:03 +00:00
}
2014-12-23 06:18:05 +00:00
// DeleteUser removes a user from the server.
func (s *Server) DeleteUser(username string) error {
c := &deleteUserCommand{Username: username}
_, err := s.broadcast(deleteUserMessageType, c)
return err
}
func (s *Server) applyDeleteUser(m *messaging.Message) error {
var c deleteUserCommand
2014-11-05 05:32:17 +00:00
mustUnmarshalJSON(m.Data, &c)
2014-10-28 23:54:49 +00:00
s.mu.Lock()
defer s.mu.Unlock()
2014-12-23 06:18:05 +00:00
// Validate user.
if c.Username == "" {
return ErrUsernameRequired
} else if s.users[c.Username] == nil {
return ErrUserNotFound
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// Remove from metastore.
2014-11-05 05:32:17 +00:00
s.meta.mustUpdate(func(tx *metatx) error {
2014-12-23 06:18:05 +00:00
return tx.deleteUser(c.Username)
2014-11-05 05:32:17 +00:00
})
2014-12-23 06:18:05 +00:00
// Delete the user.
delete(s.users, c.Username)
2014-11-05 05:32:17 +00:00
return nil
}
2014-12-23 06:18:05 +00:00
type deleteUserCommand struct {
2014-11-05 05:32:17 +00:00
Username string `json:"username"`
2014-10-28 23:54:49 +00:00
}
2014-12-23 06:18:05 +00:00
// RetentionPolicy returns a retention policy by name.
// Returns an error if the database doesn't exist.
func (s *Server) RetentionPolicy(database, name string) (*RetentionPolicy, error) {
2014-10-25 04:38:01 +00:00
s.mu.Lock()
defer s.mu.Unlock()
2014-12-23 06:18:05 +00:00
// Lookup database.
db := s.databases[database]
if db == nil {
return nil, ErrDatabaseNotFound
2014-10-25 04:38:01 +00:00
}
2014-12-23 06:18:05 +00:00
return db.policies[name], nil
}
// DefaultRetentionPolicy returns the default retention policy for a database.
// Returns an error if the database doesn't exist.
func (s *Server) DefaultRetentionPolicy(database string) (*RetentionPolicy, error) {
s.mu.Lock()
defer s.mu.Unlock()
// Lookup database.
db := s.databases[database]
if db == nil {
return nil, ErrDatabaseNotFound
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
return db.policies[db.defaultRetentionPolicy], nil
}
2014-11-05 05:32:17 +00:00
2014-12-23 06:18:05 +00:00
// RetentionPolicies returns a list of retention polocies for a database.
// Returns an error if the database doesn't exist.
func (s *Server) RetentionPolicies(database string) ([]*RetentionPolicy, error) {
s.mu.RLock()
defer s.mu.RUnlock()
// Lookup database.
db := s.databases[database]
if db == nil {
return nil, ErrDatabaseNotFound
}
// Retrieve the policies.
a := make([]*RetentionPolicy, 0, len(db.policies))
for _, p := range db.policies {
a = append(a, p)
}
return a, nil
2014-10-25 04:38:01 +00:00
}
2014-12-23 06:18:05 +00:00
// CreateRetentionPolicy creates a retention policy for a database.
func (s *Server) CreateRetentionPolicy(database string, rp *RetentionPolicy) error {
c := &createRetentionPolicyCommand{
Database: database,
Name: rp.Name,
Duration: rp.Duration,
ReplicaN: rp.ReplicaN,
SplitN: rp.SplitN,
}
_, err := s.broadcast(createRetentionPolicyMessageType, c)
return err
2014-10-25 04:38:01 +00:00
}
2014-12-23 06:18:05 +00:00
func (s *Server) applyCreateRetentionPolicy(m *messaging.Message) error {
var c createRetentionPolicyCommand
2014-11-05 05:32:17 +00:00
mustUnmarshalJSON(m.Data, &c)
2014-10-25 17:54:23 +00:00
s.mu.Lock()
defer s.mu.Unlock()
// Retrieve the database.
db := s.databases[c.Database]
if s.databases[c.Database] == nil {
return ErrDatabaseNotFound
2014-12-23 06:18:05 +00:00
} else if c.Name == "" {
return ErrRetentionPolicyNameRequired
} else if db.policies[c.Name] != nil {
return ErrRetentionPolicyExists
2014-10-25 17:54:23 +00:00
}
2014-12-23 15:47:32 +00:00
// Add policy to the database.
2014-12-23 06:18:05 +00:00
db.policies[c.Name] = &RetentionPolicy{
Name: c.Name,
Duration: c.Duration,
ReplicaN: c.ReplicaN,
SplitN: c.SplitN,
2014-11-05 05:32:17 +00:00
}
2014-10-25 17:54:23 +00:00
2014-11-05 05:32:17 +00:00
// Persist to metastore.
s.meta.mustUpdate(func(tx *metatx) error {
return tx.saveDatabase(db)
})
return nil
2014-10-25 17:54:23 +00:00
}
2014-12-23 06:18:05 +00:00
type createRetentionPolicyCommand struct {
Database string `json:"database"`
Name string `json:"name"`
Duration time.Duration `json:"duration"`
ReplicaN uint32 `json:"replicaN"`
SplitN uint32 `json:"splitN"`
}
// UpdateRetentionPolicy updates an existing retention policy on a database.
func (s *Server) UpdateRetentionPolicy(database, name string, rp *RetentionPolicy) error {
c := &updateRetentionPolicyCommand{Database: database, Name: name, NewName: rp.Name}
_, err := s.broadcast(updateRetentionPolicyMessageType, c)
return err
}
type updateRetentionPolicyCommand struct {
2014-10-25 19:30:41 +00:00
Database string `json:"database"`
2014-12-23 06:18:05 +00:00
Name string `json:"name"`
NewName string `json:"newName"`
2014-10-25 19:30:41 +00:00
}
2014-12-23 06:18:05 +00:00
func (s *Server) applyUpdateRetentionPolicy(m *messaging.Message) (err error) {
var c updateRetentionPolicyCommand
2014-11-05 05:32:17 +00:00
mustUnmarshalJSON(m.Data, &c)
s.mu.Lock()
defer s.mu.Unlock()
2014-12-23 06:18:05 +00:00
// Validate command.
db := s.databases[c.Database]
if s.databases[c.Database] == nil {
return ErrDatabaseNotFound
2014-12-23 06:18:05 +00:00
} else if c.Name == "" {
return ErrRetentionPolicyNameRequired
}
2014-12-23 06:18:05 +00:00
// Retrieve the policy.
p := db.policies[c.Name]
if db.policies[c.Name] == nil {
return ErrRetentionPolicyNotFound
}
// Update the policy name, if not blank.
if c.NewName != c.Name && c.NewName != "" {
delete(db.policies, p.Name)
p.Name = c.NewName
db.policies[p.Name] = p
2014-11-05 05:32:17 +00:00
}
// Persist to metastore.
2014-12-23 06:18:05 +00:00
err = s.meta.mustUpdate(func(tx *metatx) error {
2014-11-05 05:32:17 +00:00
return tx.saveDatabase(db)
})
2014-12-23 06:18:05 +00:00
return
}
2014-12-23 06:18:05 +00:00
// DeleteRetentionPolicy removes a retention policy from a database.
func (s *Server) DeleteRetentionPolicy(database, name string) error {
c := &deleteRetentionPolicyCommand{Database: database, Name: name}
_, err := s.broadcast(deleteRetentionPolicyMessageType, c)
return err
}
2014-12-23 06:18:05 +00:00
func (s *Server) applyDeleteRetentionPolicy(m *messaging.Message) (err error) {
var c deleteRetentionPolicyCommand
2014-11-05 05:32:17 +00:00
mustUnmarshalJSON(m.Data, &c)
s.mu.Lock()
defer s.mu.Unlock()
// Retrieve the database.
db := s.databases[c.Database]
if s.databases[c.Database] == nil {
return ErrDatabaseNotFound
2014-12-23 06:18:05 +00:00
} else if c.Name == "" {
return ErrRetentionPolicyNameRequired
} else if db.policies[c.Name] == nil {
return ErrRetentionPolicyNotFound
}
2014-12-23 06:18:05 +00:00
// Remove retention policy.
delete(db.policies, c.Name)
2014-11-05 05:32:17 +00:00
// Persist to metastore.
2014-12-23 06:18:05 +00:00
err = s.meta.mustUpdate(func(tx *metatx) error {
2014-11-05 05:32:17 +00:00
return tx.saveDatabase(db)
})
2014-12-23 06:18:05 +00:00
return
}
type deleteRetentionPolicyCommand struct {
Database string `json:"database"`
Name string `json:"name"`
}
2014-12-23 06:18:05 +00:00
// SetDefaultRetentionPolicy sets the default policy to write data into and query from on a database.
func (s *Server) SetDefaultRetentionPolicy(database, name string) error {
c := &setDefaultRetentionPolicyCommand{Database: database, Name: name}
_, err := s.broadcast(setDefaultRetentionPolicyMessageType, c)
return err
}
func (s *Server) applySetDefaultRetentionPolicy(m *messaging.Message) (err error) {
2014-11-18 23:59:37 +00:00
var c setDefaultRetentionPolicyCommand
mustUnmarshalJSON(m.Data, &c)
s.mu.Lock()
defer s.mu.Unlock()
2014-12-23 06:18:05 +00:00
// Validate command.
2014-11-18 23:59:37 +00:00
db := s.databases[c.Database]
if s.databases[c.Database] == nil {
return ErrDatabaseNotFound
2014-12-23 06:18:05 +00:00
} else if db.policies[c.Name] == nil {
return ErrRetentionPolicyNotFound
2014-11-18 23:59:37 +00:00
}
2014-12-23 06:18:05 +00:00
// Update default policy.
db.defaultRetentionPolicy = c.Name
// Persist to metastore.
2014-12-23 06:18:05 +00:00
err = s.meta.mustUpdate(func(tx *metatx) error {
return tx.saveDatabase(db)
})
2014-12-23 06:18:05 +00:00
return
2014-11-18 23:59:37 +00:00
}
type setDefaultRetentionPolicyCommand struct {
Database string `json:"database"`
Name string `json:"name"`
}
func (s *Server) applyCreateSeriesIfNotExists(m *messaging.Message) error {
var c createSeriesIfNotExistsCommand
mustUnmarshalJSON(m.Data, &c)
s.mu.Lock()
2014-12-23 06:18:05 +00:00
defer s.mu.Unlock()
2014-12-23 06:18:05 +00:00
// Validate command.
db := s.databases[c.Database]
if db == nil {
return ErrDatabaseNotFound
}
2014-12-23 06:18:05 +00:00
return s.meta.mustUpdate(func(tx *metatx) error {
return tx.createSeriesIfNotExists(db.name, c.Name, c.Tags)
})
}
type createSeriesIfNotExistsCommand struct {
Database string `json:"database"`
Name string `json:"name"`
Tags map[string]string `json:"tags"`
}
2014-12-23 06:18:05 +00:00
// WriteSeries writes series data to the database.
func (s *Server) WriteSeries(database, retentionPolicy, name string, tags map[string]string, timestamp time.Time, values map[string]interface{}) error {
2014-12-23 15:47:32 +00:00
// Find the id for the series and tagset
id, err := s.createSeriesIfNotExists(database, name, tags)
2014-12-23 06:18:05 +00:00
if err != nil {
return err
}
2014-12-23 15:47:32 +00:00
// Now write it into the shard.
sh, err := s.createShardIfNotExists(database, retentionPolicy, id, timestamp)
2014-12-23 06:18:05 +00:00
if err != nil {
return fmt.Errorf("create shard(%s/%d): %s", retentionPolicy, timestamp.Format(time.RFC3339Nano), err)
}
2014-12-23 15:47:32 +00:00
// Encode point to a byte slice.
2014-12-23 06:18:05 +00:00
data, err := marshalPoint(id, timestamp, values)
if err != nil {
return err
}
// Publish "write series" message on shard's topic to broker.
m := &messaging.Message{
Type: writeSeriesMessageType,
TopicID: sh.ID,
Data: data,
}
2014-12-23 15:47:32 +00:00
_, err = s.client.Publish(m)
2014-12-23 06:18:05 +00:00
return err
}
func (s *Server) applyWriteSeries(m *messaging.Message) error {
s.mu.RLock()
// Retrieve the database.
db := s.databasesByShard[m.TopicID]
if db == nil {
s.mu.RUnlock()
return ErrDatabaseNotFound
}
// Retrieve the shard.
sh := db.shards[m.TopicID]
if sh == nil {
s.mu.RUnlock()
return ErrShardNotFound
}
s.mu.RUnlock()
// TODO: enable some way to specify if the data should be overwritten
overwrite := true
// Write to shard.
return sh.writeSeries(overwrite, m.Data)
}
2014-12-23 06:18:05 +00:00
// seriesID returns the unique id of a series and tagset and a bool indicating if it was found
2014-12-23 15:47:32 +00:00
func (s *Server) seriesID(database, name string, tags map[string]string) (id uint32) {
2014-12-23 06:18:05 +00:00
s.meta.view(func(tx *metatx) error {
2014-12-23 15:47:32 +00:00
id, _ = tx.seriesID(database, name, tags)
2014-12-23 06:18:05 +00:00
return nil
})
2014-12-23 15:47:32 +00:00
return
2014-12-23 06:18:05 +00:00
}
2014-12-23 15:47:32 +00:00
func (s *Server) createSeriesIfNotExists(database, name string, tags map[string]string) (uint32, error) {
// Try to find series locally first.
if id := s.seriesID(database, name, tags); id != 0 {
2014-12-23 06:18:05 +00:00
return id, nil
}
2014-12-23 15:47:32 +00:00
// If it doesn't exist then create a message and broadcast.
c := &createSeriesIfNotExistsCommand{Database: database, Name: name, Tags: tags}
_, err := s.broadcast(createSeriesIfNotExistsMessageType, c)
2014-12-23 06:18:05 +00:00
if err != nil {
2014-12-23 15:47:32 +00:00
return 0, err
2014-12-23 06:18:05 +00:00
}
2014-12-23 15:47:32 +00:00
// Lookup series again.
id := s.seriesID(database, name, tags)
if id == 0 {
return 0, ErrSeriesNotFound
2014-12-23 06:18:05 +00:00
}
return id, nil
}
2014-12-23 15:47:32 +00:00
func (s *Server) Measurements(database string) (a Measurements) {
s.meta.view(func(tx *metatx) error {
a = tx.measurements(database)
2014-12-23 06:18:05 +00:00
return nil
})
return
}
2014-10-24 00:54:12 +00:00
// processor runs in a separate goroutine and processes all incoming broker messages.
2014-12-30 22:46:50 +00:00
func (s *Server) processor(client MessagingClient, done chan struct{}) {
2014-10-24 00:54:12 +00:00
for {
// Read incoming message.
var m *messaging.Message
select {
case <-done:
return
case m = <-client.C():
}
// Process message.
2014-10-24 23:45:02 +00:00
var err error
2014-10-24 00:54:12 +00:00
switch m.Type {
case writeSeriesMessageType:
err = s.applyWriteSeries(m)
2014-12-30 15:50:15 +00:00
case createDataNodeMessageType:
err = s.applyCreateDataNode(m)
case deleteDataNodeMessageType:
err = s.applyDeleteDataNode(m)
2014-10-24 00:54:12 +00:00
case createDatabaseMessageType:
2014-10-24 23:45:02 +00:00
err = s.applyCreateDatabase(m)
case deleteDatabaseMessageType:
err = s.applyDeleteDatabase(m)
2014-12-23 06:18:05 +00:00
case createUserMessageType:
err = s.applyCreateUser(m)
case updateUserMessageType:
err = s.applyUpdateUser(m)
case deleteUserMessageType:
err = s.applyDeleteUser(m)
case createRetentionPolicyMessageType:
err = s.applyCreateRetentionPolicy(m)
2014-12-23 06:18:05 +00:00
case updateRetentionPolicyMessageType:
err = s.applyUpdateRetentionPolicy(m)
case deleteRetentionPolicyMessageType:
err = s.applyDeleteRetentionPolicy(m)
case createShardIfNotExistsMessageType:
err = s.applyCreateShardIfNotExists(m)
2014-11-18 23:59:37 +00:00
case setDefaultRetentionPolicyMessageType:
err = s.applySetDefaultRetentionPolicy(m)
case createSeriesIfNotExistsMessageType:
err = s.applyCreateSeriesIfNotExists(m)
2014-10-24 05:38:03 +00:00
}
// Sync high water mark and errors.
s.mu.Lock()
s.index = m.Index
if err != nil {
s.errors[m.Index] = err
2014-10-24 00:54:12 +00:00
}
s.mu.Unlock()
2014-10-24 00:54:12 +00:00
}
}
2014-10-22 05:32:19 +00:00
// MessagingClient represents the client used to receive messages from brokers.
type MessagingClient interface {
2014-10-24 05:38:03 +00:00
// Publishes a message to the broker.
Publish(m *messaging.Message) (index uint64, err error)
// The streaming channel for all subscribed messages.
2014-10-22 05:32:19 +00:00
C() <-chan *messaging.Message
}
2014-12-30 15:50:15 +00:00
// DataNode represents a data node in the cluster.
type DataNode struct {
2014-12-29 23:12:51 +00:00
ID uint64
URL *url.URL
}
2014-12-30 15:50:15 +00:00
// newDataNode returns an instance of DataNode.
func newDataNode() *DataNode { return &DataNode{} }
2014-12-29 23:12:51 +00:00
2014-12-30 15:50:15 +00:00
type dataNodes []*DataNode
2014-12-29 23:12:51 +00:00
2014-12-30 15:50:15 +00:00
func (p dataNodes) Len() int { return len(p) }
func (p dataNodes) Less(i, j int) bool { return p[i].ID < p[j].ID }
func (p dataNodes) Swap(i, j int) { p[i], p[j] = p[j], p[i] }
2014-12-29 23:12:51 +00:00
2014-12-23 06:18:05 +00:00
// database represents a collection of retention policies.
type database struct {
name string
2014-11-05 05:32:17 +00:00
2014-12-23 06:18:05 +00:00
policies map[string]*RetentionPolicy // retention policies by name
shards map[uint64]*Shard // shards by id
2014-11-05 05:32:17 +00:00
2014-12-23 06:18:05 +00:00
defaultRetentionPolicy string
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// newDatabase returns an instance of database.
func newDatabase() *database {
return &database{
policies: make(map[string]*RetentionPolicy),
shards: make(map[uint64]*Shard),
2014-11-05 05:32:17 +00:00
}
}
2014-12-23 06:18:05 +00:00
// shardByTimestamp returns a shard that owns a given timestamp.
func (db *database) shardByTimestamp(policy string, id uint32, timestamp time.Time) (*Shard, error) {
p := db.policies[policy]
if p == nil {
return nil, ErrRetentionPolicyNotFound
}
return p.shardByTimestamp(id, timestamp), nil
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// shardsByTimestamp returns all shards that own a given timestamp.
func (db *database) shardsByTimestamp(policy string, timestamp time.Time) ([]*Shard, error) {
p := db.policies[policy]
if p == nil {
return nil, ErrRetentionPolicyNotFound
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
return p.shardsByTimestamp(timestamp), nil
2014-11-05 05:32:17 +00:00
}
2014-12-23 15:47:32 +00:00
// timeBetweenInclusive returns true if t is between min and max, inclusive.
func timeBetweenInclusive(t, min, max time.Time) bool {
2014-12-23 06:18:05 +00:00
return (t.Equal(min) || t.After(min)) && (t.Equal(max) || t.Before(max))
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// MarshalJSON encodes a database into a JSON-encoded byte slice.
func (db *database) MarshalJSON() ([]byte, error) {
// Copy over properties to intermediate type.
var o databaseJSON
o.Name = db.name
o.DefaultRetentionPolicy = db.defaultRetentionPolicy
for _, rp := range db.policies {
o.Policies = append(o.Policies, rp)
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
for _, s := range db.shards {
o.Shards = append(o.Shards, s)
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
return json.Marshal(&o)
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// UnmarshalJSON decodes a JSON-encoded byte slice to a database.
func (db *database) UnmarshalJSON(data []byte) error {
// Decode into intermediate type.
var o databaseJSON
if err := json.Unmarshal(data, &o); err != nil {
return err
}
2014-12-23 06:18:05 +00:00
// Copy over properties from intermediate type.
db.name = o.Name
db.defaultRetentionPolicy = o.DefaultRetentionPolicy
// Copy shard policies.
db.policies = make(map[string]*RetentionPolicy)
for _, rp := range o.Policies {
db.policies[rp.Name] = rp
}
2014-12-23 06:18:05 +00:00
// Copy shards.
db.shards = make(map[uint64]*Shard)
for _, s := range o.Shards {
db.shards[s.ID] = s
2014-11-28 19:14:04 +00:00
}
2014-12-23 06:18:05 +00:00
return nil
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// databaseJSON represents the JSON-serialization format for a database.
type databaseJSON struct {
Name string `json:"name,omitempty"`
DefaultRetentionPolicy string `json:"defaultRetentionPolicy,omitempty"`
Policies []*RetentionPolicy `json:"policies,omitempty"`
Shards []*Shard `json:"shards,omitempty"`
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// Measurement represents a collection of time series in a database
type Measurement struct {
Name string `json:"name,omitempty"`
Series []*Series `json:"series,omitempty"`
Fields []*Fields `json:"fields,omitempty"`
}
2014-12-23 06:18:05 +00:00
type Measurements []*Measurement
2014-12-23 06:18:05 +00:00
func (m Measurement) String() string { return string(mustMarshalJSON(m)) }
2014-12-23 06:18:05 +00:00
// Field represents a series field.
type Field struct {
ID uint8 `json:"id,omitempty"`
Name string `json:"name,omitempty"`
Type FieldType `json:"field"`
2014-11-28 19:14:04 +00:00
}
2014-12-23 06:18:05 +00:00
type FieldType int
2014-11-28 19:14:04 +00:00
2014-12-23 06:18:05 +00:00
const (
Int64 FieldType = iota
Float64
String
Boolean
Binary
)
2014-12-23 06:18:05 +00:00
// Fields represents a list of fields.
type Fields []*Field
2014-11-28 19:14:04 +00:00
2014-12-23 06:18:05 +00:00
// Series belong to a Measurement and represent unique time series in a database
type Series struct {
ID uint32
Tags map[string]string
}
2014-12-23 06:18:05 +00:00
// RetentionPolicy represents a policy for creating new shards in a database and how long they're kept around for.
type RetentionPolicy struct {
// Unique name within database. Required.
Name string
2014-12-23 06:18:05 +00:00
// Length of time to keep data around
Duration time.Duration
2014-12-23 06:18:05 +00:00
ReplicaN uint32
SplitN uint32
2014-12-23 06:18:05 +00:00
Shards []*Shard
}
2014-12-23 06:18:05 +00:00
// NewRetentionPolicy returns a new instance of RetentionPolicy with defaults set.
func NewRetentionPolicy(name string) *RetentionPolicy {
return &RetentionPolicy{
Name: name,
ReplicaN: DefaultReplicaN,
SplitN: DefaultSplitN,
Duration: DefaultShardRetention,
}
}
2014-12-23 06:18:05 +00:00
// shardByTimestamp returns the shard in the space that owns a given timestamp for a given series id.
// Returns nil if the shard does not exist.
func (rp *RetentionPolicy) shardByTimestamp(id uint32, timestamp time.Time) *Shard {
shards := rp.shardsByTimestamp(timestamp)
if len(shards) > 0 {
return shards[int(id)%len(shards)]
}
2014-12-23 06:18:05 +00:00
return nil
}
2014-12-23 06:18:05 +00:00
func (rp *RetentionPolicy) shardsByTimestamp(timestamp time.Time) []*Shard {
shards := make([]*Shard, 0, rp.SplitN)
for _, s := range rp.Shards {
2014-12-23 15:47:32 +00:00
if timeBetweenInclusive(timestamp, s.StartTime, s.EndTime) {
2014-12-23 06:18:05 +00:00
shards = append(shards, s)
}
}
2014-12-23 06:18:05 +00:00
return shards
}
2014-12-23 06:18:05 +00:00
// MarshalJSON encodes a retention policy to a JSON-encoded byte slice.
func (rp *RetentionPolicy) MarshalJSON() ([]byte, error) {
return json.Marshal(&retentionPolicyJSON{
Name: rp.Name,
Duration: rp.Duration,
ReplicaN: rp.ReplicaN,
SplitN: rp.SplitN,
})
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// UnmarshalJSON decodes a JSON-encoded byte slice to a retention policy.
func (rp *RetentionPolicy) UnmarshalJSON(data []byte) error {
// Decode into intermediate type.
var o retentionPolicyJSON
if err := json.Unmarshal(data, &o); err != nil {
return err
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// Copy over properties from intermediate type.
rp.Name = o.Name
rp.ReplicaN = o.ReplicaN
rp.SplitN = o.SplitN
rp.Duration = o.Duration
rp.Shards = o.Shards
return nil
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// retentionPolicyJSON represents an intermediate struct for JSON marshaling.
type retentionPolicyJSON struct {
Name string `json:"name"`
ReplicaN uint32 `json:"replicaN,omitempty"`
SplitN uint32 `json:"splitN,omitempty"`
Duration time.Duration `json:"duration,omitempty"`
Shards []*Shard `json:"shards,omitempty"`
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// RetentionPolicies represents a list of shard policies.
type RetentionPolicies []*RetentionPolicy
// Shards returns a list of all shards for all policies.
func (rps RetentionPolicies) Shards() []*Shard {
var shards []*Shard
for _, rp := range rps {
shards = append(shards, rp.Shards...)
}
return shards
2014-11-05 05:32:17 +00:00
}
2014-12-23 06:18:05 +00:00
// BcryptCost is the cost associated with generating password with Bcrypt.
// This setting is lowered during testing to improve test suite performance.
var BcryptCost = 10
// User represents a user account on the system.
// It can be given read/write permissions to individual databases.
type User struct {
Name string `json:"name"`
Hash string `json:"hash"`
Admin bool `json:"admin,omitempty"`
2014-11-04 04:15:58 +00:00
}
2014-12-23 06:18:05 +00:00
// Authenticate returns nil if the password matches the user's password.
// Returns an error if the password was incorrect.
func (u *User) Authenticate(password string) error {
return bcrypt.CompareHashAndPassword([]byte(u.Hash), []byte(password))
2014-10-24 00:54:12 +00:00
}
2014-12-23 06:18:05 +00:00
// users represents a list of users, sortable by name.
type users []*User
func (p users) Len() int { return len(p) }
func (p users) Less(i, j int) bool { return p[i].Name < p[j].Name }
func (p users) Swap(i, j int) { p[i], p[j] = p[j], p[i] }
type Matcher struct {
IsRegex bool
Name string
2014-10-24 00:54:12 +00:00
}
2014-12-23 06:18:05 +00:00
func (m *Matcher) Matches(name string) bool {
if m.IsRegex {
matches, _ := regexp.MatchString(m.Name, name)
return matches
2014-10-22 05:32:19 +00:00
}
2014-12-23 06:18:05 +00:00
return m.Name == name
2014-10-22 05:32:19 +00:00
}
2014-10-24 05:38:03 +00:00
2014-12-23 06:18:05 +00:00
// HashPassword generates a cryptographically secure hash for password.
// Returns an error if the password is invalid or a hash cannot be generated.
func HashPassword(password string) ([]byte, error) {
// The second arg is the cost of the hashing, higher is slower but makes
// it harder to brute force, since it will be really slow and impractical
return bcrypt.GenerateFromPassword([]byte(password), BcryptCost)
}
// ContinuousQuery represents a query that exists on the server and processes
// each incoming event.
type ContinuousQuery struct {
ID uint32
Query string
// TODO: ParsedQuery *parser.SelectQuery
}