git » chasquid » commit 394067b

userdb: Use protocol buffers instead of our custom format

author Alberto Bertogli
2016-09-18 00:12:41 UTC
committer Alberto Bertogli
2016-10-09 23:51:03 UTC
parent 5c6fb934fe5d587979c1d00e0858b2051c511523

userdb: Use protocol buffers instead of our custom format

Protocol buffers are a more portable, practical and safe format for the user
database.

chasquid.go +1 -4
cmd/chasquid-userdb/chasquid-userdb.go +8 -12
internal/userdb/userdb.go +60 -205
internal/userdb/userdb.pb.go +224 -0
internal/userdb/userdb.proto +28 -0
internal/userdb/userdb_test.go +46 -88
test/.gitignore +4 -0
test/t-01-simple_local/config/domains/testserver/users +0 -2
test/t-01-simple_local/run.sh +1 -0
test/t-02-exim/config/domains/srv-chasquid/users +0 -2
test/t-02-exim/run.sh +1 -0
test/util/lib.sh +8 -0

diff --git a/chasquid.go b/chasquid.go
index 47925f2..0b176c5 100644
--- a/chasquid.go
+++ b/chasquid.go
@@ -136,13 +136,10 @@ func loadDomain(s *Server, name, dir string) {
 
 	if _, err := os.Stat(dir + "/users"); err == nil {
 		glog.Infof("    adding users")
-		udb, warnings, err := userdb.Load(dir + "/users")
+		udb, err := userdb.Load(dir + "/users")
 		if err != nil {
 			glog.Errorf("      error: %v", err)
 		} else {
-			for _, w := range warnings {
-				glog.Warningf("     %v", w)
-			}
 			s.AddUserDB(name, udb)
 			// TODO: periodically reload the database.
 		}
diff --git a/cmd/chasquid-userdb/chasquid-userdb.go b/cmd/chasquid-userdb/chasquid-userdb.go
index cb12424..8f6f7e6 100644
--- a/cmd/chasquid-userdb/chasquid-userdb.go
+++ b/cmd/chasquid-userdb/chasquid-userdb.go
@@ -29,23 +29,19 @@ func main() {
 		os.Exit(1)
 	}
 
-	db, ws, err := userdb.Load(*dbFname)
+	db, err := userdb.Load(*dbFname)
 	if err != nil {
-		fmt.Printf("error loading database: %v\n", err)
-		os.Exit(1)
-	}
-
-	for _, w := range ws {
-		fmt.Printf("warning: %v\n", w)
+		if *adduser != "" && os.IsNotExist(err) {
+			fmt.Printf("creating database\n")
+		} else {
+			fmt.Printf("error loading database: %v\n", err)
+			os.Exit(1)
+		}
 	}
 
 	if *adduser == "" {
 		fmt.Printf("database loaded\n")
-		if len(ws) == 0 {
-			os.Exit(0)
-		} else {
-			os.Exit(1)
-		}
+		return
 	}
 
 	if *password == "" {
diff --git a/internal/userdb/userdb.go b/internal/userdb/userdb.go
index bd3a08a..355cd00 100644
--- a/internal/userdb/userdb.go
+++ b/internal/userdb/userdb.go
@@ -3,25 +3,15 @@
 //
 // Format
 //
-// The user database is a single text file, with one line per user.
-// All contents must be UTF-8.
+// The user database is a file containing a list of users and their passwords,
+// encrypted with some scheme.
+// We use a text-encoded protobuf, the structure can be found in userdb.proto.
 //
-// For extensibility, the first line MUST be:
+// We write text instead of binary to make it easier for administrators to
+// troubleshoot, and since performance is not an issue for our expected usage.
 //
-//   #chasquid-userdb-v1
-//
-// Then, each line is structured as follows:
-//
-//   user SP scheme SP password
-//
-// Where user is the username in question (usually without the domain,
-// although this package is agnostic to it); scheme is the encryption scheme
-// used for the password; and finally the password, encrypted with the
-// referenced scheme and base64-encoded.
-//
-// Lines with parsing errors, including unknown schemes, will be ignored.
 // Users must be UTF-8 and NOT contain whitespace; the library will enforce
-// this as well.
+// this.
 //
 //
 // Schemes
@@ -40,178 +30,87 @@
 //
 package userdb
 
+//go:generate protoc --go_out=. userdb.proto
+
 import (
-	"bufio"
 	"bytes"
 	"crypto/rand"
-	"encoding/base64"
 	"errors"
 	"fmt"
-	"os"
 	"strings"
 	"sync"
 	"unicode/utf8"
 
 	"golang.org/x/crypto/scrypt"
 
-	"blitiri.com.ar/go/chasquid/internal/safeio"
+	"blitiri.com.ar/go/chasquid/internal/protoio"
 )
 
-type user struct {
-	name     string
-	scheme   scheme
-	password string
-}
-
 type DB struct {
 	fname string
-	finfo os.FileInfo
-
-	// Map of username -> user structure
-	users map[string]user
+	db    *ProtoDB
 
-	// Lock protecting the users map.
+	// Lock protecting db.
 	mu sync.RWMutex
 }
 
 var (
-	ErrMissingHeader   = errors.New("missing '#chasquid-userdb-v1' header")
 	ErrInvalidUsername = errors.New("username contains invalid characters")
 )
 
 func New(fname string) *DB {
 	return &DB{
 		fname: fname,
-		users: map[string]user{},
+		db:    &ProtoDB{Users: map[string]*Password{}},
 	}
 }
 
 // Load the database from the given file.
-// Return the database, a list of warnings (if any), and a fatal error if the
-// database could not be loaded.
-func Load(fname string) (*DB, []error, error) {
-	f, err := os.Open(fname)
-	if err != nil {
-		return nil, nil, err
-	}
+// Return the database, and a fatal error if the database could not be
+// loaded.
+func Load(fname string) (*DB, error) {
+	db := New(fname)
+	err := protoio.ReadTextMessage(fname, db.db)
 
-	db := &DB{
-		fname: fname,
-		users: map[string]user{},
+	// Reading may result in an empty protobuf or dictionary; make sure we
+	// return an empty but usable structure.
+	// This simplifies many of our uses, as we can assume the map is not nil.
+	if db.db == nil || db.db.Users == nil {
+		db.db = &ProtoDB{Users: map[string]*Password{}}
 	}
 
-	db.finfo, err = f.Stat()
-	if err != nil {
-		return nil, nil, err
-	}
-
-	// Special case: an empty file is a valid, empty database.
-	// This simplifies clients.
-	if db.finfo.Size() == 0 {
-		return db, nil, nil
-	}
-
-	scanner := bufio.NewScanner(f)
-	scanner.Scan()
-	if scanner.Text() != "#chasquid-userdb-v1" {
-		return nil, nil, ErrMissingHeader
-	}
-
-	var warnings []error
-
-	// Now the users, one per line. Skip invalid ones.
-	for i := 2; scanner.Scan(); i++ {
-		var name, schemeStr, b64passwd string
-		n, err := fmt.Sscanf(scanner.Text(), "%s %s %s",
-			&name, &schemeStr, &b64passwd)
-		if err != nil || n != 3 {
-			warnings = append(warnings, fmt.Errorf(
-				"line %d: error parsing - %d elements - %v", i, n, err))
-			break
-		}
-
-		if !ValidUsername(name) {
-			warnings = append(warnings, fmt.Errorf(
-				"line %d: invalid username", i))
-			continue
-		}
-
-		password, err := base64.StdEncoding.DecodeString(b64passwd)
-		if err != nil {
-			warnings = append(warnings, fmt.Errorf(
-				"line %d: error decoding password: %v", i, err))
-			continue
-		}
-
-		sc, err := schemeFromString(schemeStr)
-		if err != nil {
-			warnings = append(warnings, fmt.Errorf(
-				"line %d: error in scheme: %v", i, err))
-			continue
-		}
-
-		u := user{
-			name:     name,
-			scheme:   sc,
-			password: string(password),
-		}
-		db.users[name] = u
-	}
-
-	if err := scanner.Err(); err != nil {
-		return nil, warnings, err
-	}
-
-	return db, warnings, nil
+	return db, err
 }
 
 // Reload the database, refreshing its contents from the current file on disk.
 // If there are errors reading from the file, they are returned and the
-// database is not changed. Warnings are returned regardless.
-func (db *DB) Reload() ([]error, error) {
-	newdb, warnings, err := Load(db.fname)
+// database is not changed.
+func (db *DB) Reload() error {
+	newdb, err := Load(db.fname)
 	if err != nil {
-		return warnings, err
+		return err
 	}
 
 	db.mu.Lock()
-	db.users = newdb.users
-	db.finfo = newdb.finfo
+	db.db = newdb.db
 	db.mu.Unlock()
 
-	return warnings, nil
+	return nil
 }
 
 // Write the database to disk. It will do a complete rewrite each time, and is
 // not safe to call it from different processes in parallel.
 func (db *DB) Write() error {
-	buf := new(bytes.Buffer)
-	buf.WriteString("#chasquid-userdb-v1\n")
-
 	db.mu.RLock()
 	defer db.mu.RUnlock()
 
-	// TODO: Sort the usernames, just to be friendlier.
-	for _, user := range db.users {
-		if strings.ContainsAny(user.name, illegalUsernameChars) {
-			return ErrInvalidUsername
-		}
-		fmt.Fprintf(buf, "%s %s %s\n",
-			user.name, user.scheme.String(),
-			base64.StdEncoding.EncodeToString([]byte(user.password)))
-	}
-
-	mode := os.FileMode(0660)
-	if db.finfo != nil {
-		mode = db.finfo.Mode()
-	}
-	return safeio.WriteFile(db.fname, buf.Bytes(), mode)
+	return protoio.WriteTextMessage(db.fname, db.db, 0660)
 }
 
 // Does this user exist in the database?
 func (db *DB) Exists(user string) bool {
 	db.mu.RLock()
-	_, ok := db.users[user]
+	_, ok := db.db.Users[user]
 	db.mu.RUnlock()
 
 	return ok
@@ -220,14 +119,27 @@ func (db *DB) Exists(user string) bool {
 // Is this password valid for the user?
 func (db *DB) Authenticate(name, plainPassword string) bool {
 	db.mu.RLock()
-	u, ok := db.users[name]
+	passwd, ok := db.db.Users[name]
 	db.mu.RUnlock()
 
 	if !ok {
 		return false
 	}
 
-	return u.scheme.PasswordMatches(plainPassword, u.password)
+	return passwd.PasswordMatches(plainPassword)
+}
+
+func (p *Password) PasswordMatches(plain string) bool {
+	switch s := p.Scheme.(type) {
+	case nil:
+		return false
+	case *Password_Scrypt:
+		return s.Scrypt.PasswordMatches(plain)
+	case *Password_Plain:
+		return s.Plain.PasswordMatches(plain)
+	default:
+		return false
+	}
 }
 
 // Check if the given user name is valid.
@@ -250,31 +162,29 @@ func (db *DB) AddUser(name, plainPassword string) error {
 		return ErrInvalidUsername
 	}
 
-	s := scryptScheme{
+	s := &Scrypt{
 		// Use hard-coded standard parameters for now.
 		// Follow the recommendations from the scrypt paper.
-		logN: 14, r: 8, p: 1, keyLen: 32,
+		LogN: 14, R: 8, P: 1, KeyLen: 32,
 
 		// 16 bytes of salt (will be filled later).
-		salt: make([]byte, 16),
+		Salt: make([]byte, 16),
 	}
 
-	n, err := rand.Read(s.salt)
+	n, err := rand.Read(s.Salt)
 	if n != 16 || err != nil {
 		return fmt.Errorf("failed to get salt - %d - %v", n, err)
 	}
 
-	encrypted, err := scrypt.Key([]byte(plainPassword), s.salt,
-		1<<s.logN, s.r, s.p, s.keyLen)
+	s.Encrypted, err = scrypt.Key([]byte(plainPassword), s.Salt,
+		1<<s.LogN, int(s.R), int(s.P), int(s.KeyLen))
 	if err != nil {
 		return fmt.Errorf("scrypt failed: %v", err)
 	}
 
 	db.mu.Lock()
-	db.users[name] = user{
-		name:     name,
-		scheme:   s,
-		password: string(encrypted),
+	db.db.Users[name] = &Password{
+		Scheme: &Password_Scrypt{s},
 	}
 	db.mu.Unlock()
 
@@ -285,46 +195,18 @@ func (db *DB) AddUser(name, plainPassword string) error {
 // Encryption schemes
 //
 
-type scheme interface {
-	String() string
-	PasswordMatches(plain, encrypted string) bool
-}
-
 // Plain text scheme. Useful mostly for testing and debugging.
 // TODO: Do we really need this? Removing it would make accidents less likely
 // to happen. Consider doing so when we add another scheme, so we a least have
 // two and multi-scheme support does not bit-rot.
-type plainScheme struct{}
-
-func (s plainScheme) String() string {
-	return "PLAIN"
-}
-
-func (s plainScheme) PasswordMatches(plain, encrypted string) bool {
-	return plain == encrypted
+func (p *Plain) PasswordMatches(plain string) bool {
+	return plain == string(p.Password)
 }
 
 // scrypt scheme, which we use by default.
-type scryptScheme struct {
-	logN   uint // 1<<logN requires this to be uint
-	r, p   int
-	keyLen int
-	salt   []byte
-}
-
-func (s scryptScheme) String() string {
-	// We're encoding the salt in base64, which uses "/+=", and the URL
-	// variant uses "-_=". We use standard encoding, but shouldn't use any of
-	// those as separators, just to be safe.
-	// It's important that the salt be last, as we can only scan
-	// space-delimited strings.
-	return fmt.Sprintf("SCRYPT@n:%d,r:%d,p:%d,l:%d,%s",
-		s.logN, s.r, s.p, s.keyLen,
-		base64.StdEncoding.EncodeToString(s.salt))
-}
-
-func (s scryptScheme) PasswordMatches(plain, encrypted string) bool {
-	dk, err := scrypt.Key([]byte(plain), s.salt, 1<<s.logN, s.r, s.p, s.keyLen)
+func (s *Scrypt) PasswordMatches(plain string) bool {
+	dk, err := scrypt.Key([]byte(plain), s.Salt,
+		1<<s.LogN, int(s.R), int(s.P), int(s.KeyLen))
 
 	if err != nil {
 		// The encryption failed, this is due to the parameters being invalid.
@@ -333,32 +215,5 @@ func (s scryptScheme) PasswordMatches(plain, encrypted string) bool {
 		panic(fmt.Sprintf("scrypt failed: %v", err))
 	}
 
-	return bytes.Equal(dk, []byte(encrypted))
-}
-
-func schemeFromString(s string) (scheme, error) {
-	if s == "PLAIN" {
-		return plainScheme{}, nil
-	} else if strings.HasPrefix(s, "SCRYPT@") {
-		sc := scryptScheme{}
-		var b64salt string
-		n, err := fmt.Sscanf(s, "SCRYPT@n:%d,r:%d,p:%d,l:%d,%s",
-			&sc.logN, &sc.r, &sc.p, &sc.keyLen, &b64salt)
-		if n != 5 || err != nil {
-			return nil, fmt.Errorf("error scanning scrypt: %d %v", n, err)
-		}
-		sc.salt, err = base64.StdEncoding.DecodeString(b64salt)
-		if err != nil {
-			return nil, fmt.Errorf("error decoding salt: %v", err)
-		}
-
-		// Perform some sanity checks on the parameters, just in case.
-		if (sc.logN >= 32) || (sc.r*sc.p >= 1<<30) || (sc.keyLen < 24) {
-			return nil, fmt.Errorf("invalid scrypt parameters")
-		}
-
-		return sc, nil
-	}
-
-	return nil, fmt.Errorf("unknown scheme")
+	return bytes.Equal(dk, []byte(s.Encrypted))
 }
diff --git a/internal/userdb/userdb.pb.go b/internal/userdb/userdb.pb.go
new file mode 100644
index 0000000..c94d8f1
--- /dev/null
+++ b/internal/userdb/userdb.pb.go
@@ -0,0 +1,224 @@
+// Code generated by protoc-gen-go.
+// source: userdb.proto
+// DO NOT EDIT!
+
+/*
+Package userdb is a generated protocol buffer package.
+
+It is generated from these files:
+	userdb.proto
+
+It has these top-level messages:
+	ProtoDB
+	Password
+	Scrypt
+	Plain
+*/
+package userdb
+
+import proto "github.com/golang/protobuf/proto"
+import fmt "fmt"
+import math "math"
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type ProtoDB struct {
+	Users map[string]*Password `protobuf:"bytes,1,rep,name=users" json:"users,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+}
+
+func (m *ProtoDB) Reset()                    { *m = ProtoDB{} }
+func (m *ProtoDB) String() string            { return proto.CompactTextString(m) }
+func (*ProtoDB) ProtoMessage()               {}
+func (*ProtoDB) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
+
+func (m *ProtoDB) GetUsers() map[string]*Password {
+	if m != nil {
+		return m.Users
+	}
+	return nil
+}
+
+type Password struct {
+	// Types that are valid to be assigned to Scheme:
+	//	*Password_Scrypt
+	//	*Password_Plain
+	Scheme isPassword_Scheme `protobuf_oneof:"scheme"`
+}
+
+func (m *Password) Reset()                    { *m = Password{} }
+func (m *Password) String() string            { return proto.CompactTextString(m) }
+func (*Password) ProtoMessage()               {}
+func (*Password) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
+
+type isPassword_Scheme interface {
+	isPassword_Scheme()
+}
+
+type Password_Scrypt struct {
+	Scrypt *Scrypt `protobuf:"bytes,2,opt,name=scrypt,oneof"`
+}
+type Password_Plain struct {
+	Plain *Plain `protobuf:"bytes,3,opt,name=plain,oneof"`
+}
+
+func (*Password_Scrypt) isPassword_Scheme() {}
+func (*Password_Plain) isPassword_Scheme()  {}
+
+func (m *Password) GetScheme() isPassword_Scheme {
+	if m != nil {
+		return m.Scheme
+	}
+	return nil
+}
+
+func (m *Password) GetScrypt() *Scrypt {
+	if x, ok := m.GetScheme().(*Password_Scrypt); ok {
+		return x.Scrypt
+	}
+	return nil
+}
+
+func (m *Password) GetPlain() *Plain {
+	if x, ok := m.GetScheme().(*Password_Plain); ok {
+		return x.Plain
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*Password) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _Password_OneofMarshaler, _Password_OneofUnmarshaler, _Password_OneofSizer, []interface{}{
+		(*Password_Scrypt)(nil),
+		(*Password_Plain)(nil),
+	}
+}
+
+func _Password_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*Password)
+	// scheme
+	switch x := m.Scheme.(type) {
+	case *Password_Scrypt:
+		b.EncodeVarint(2<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Scrypt); err != nil {
+			return err
+		}
+	case *Password_Plain:
+		b.EncodeVarint(3<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Plain); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("Password.Scheme has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _Password_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*Password)
+	switch tag {
+	case 2: // scheme.scrypt
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Scrypt)
+		err := b.DecodeMessage(msg)
+		m.Scheme = &Password_Scrypt{msg}
+		return true, err
+	case 3: // scheme.plain
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Plain)
+		err := b.DecodeMessage(msg)
+		m.Scheme = &Password_Plain{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _Password_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*Password)
+	// scheme
+	switch x := m.Scheme.(type) {
+	case *Password_Scrypt:
+		s := proto.Size(x.Scrypt)
+		n += proto.SizeVarint(2<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Password_Plain:
+		s := proto.Size(x.Plain)
+		n += proto.SizeVarint(3<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+type Scrypt struct {
+	LogN      uint64 `protobuf:"varint,1,opt,name=logN" json:"logN,omitempty"`
+	R         int32  `protobuf:"varint,2,opt,name=r" json:"r,omitempty"`
+	P         int32  `protobuf:"varint,3,opt,name=p" json:"p,omitempty"`
+	KeyLen    int32  `protobuf:"varint,4,opt,name=keyLen" json:"keyLen,omitempty"`
+	Salt      []byte `protobuf:"bytes,5,opt,name=salt,proto3" json:"salt,omitempty"`
+	Encrypted []byte `protobuf:"bytes,6,opt,name=encrypted,proto3" json:"encrypted,omitempty"`
+}
+
+func (m *Scrypt) Reset()                    { *m = Scrypt{} }
+func (m *Scrypt) String() string            { return proto.CompactTextString(m) }
+func (*Scrypt) ProtoMessage()               {}
+func (*Scrypt) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
+
+type Plain struct {
+	Password []byte `protobuf:"bytes,1,opt,name=password,proto3" json:"password,omitempty"`
+}
+
+func (m *Plain) Reset()                    { *m = Plain{} }
+func (m *Plain) String() string            { return proto.CompactTextString(m) }
+func (*Plain) ProtoMessage()               {}
+func (*Plain) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
+
+func init() {
+	proto.RegisterType((*ProtoDB)(nil), "userdb.ProtoDB")
+	proto.RegisterType((*Password)(nil), "userdb.Password")
+	proto.RegisterType((*Scrypt)(nil), "userdb.Scrypt")
+	proto.RegisterType((*Plain)(nil), "userdb.Plain")
+}
+
+func init() { proto.RegisterFile("userdb.proto", fileDescriptor0) }
+
+var fileDescriptor0 = []byte{
+	// 289 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x44, 0x91, 0x41, 0x4b, 0x03, 0x31,
+	0x10, 0x85, 0x4d, 0xdb, 0xc4, 0x76, 0xba, 0x4a, 0x99, 0x83, 0x84, 0xe2, 0x41, 0x56, 0x94, 0x9e,
+	0x16, 0xa9, 0x17, 0xf1, 0x58, 0x14, 0x44, 0x44, 0x24, 0xe2, 0x0f, 0xd8, 0xba, 0x41, 0xc5, 0x75,
+	0x37, 0x24, 0x5b, 0x65, 0xaf, 0x5e, 0xfc, 0xdb, 0x26, 0xb3, 0xe9, 0xf6, 0x36, 0xef, 0x7b, 0x33,
+	0x6f, 0x26, 0x04, 0x92, 0x8d, 0xd3, 0xb6, 0x58, 0x67, 0xc6, 0xd6, 0x4d, 0x8d, 0xa2, 0x53, 0xe9,
+	0x1f, 0x83, 0xfd, 0xa7, 0x40, 0x6e, 0x56, 0x78, 0x01, 0x3c, 0x50, 0x27, 0xd9, 0xc9, 0x70, 0x31,
+	0x5d, 0xce, 0xb3, 0x38, 0x11, 0xfd, 0xec, 0x25, 0x98, 0xb7, 0x55, 0x63, 0x5b, 0xd5, 0x35, 0xce,
+	0xef, 0x01, 0x76, 0x10, 0x67, 0x30, 0xfc, 0xd4, 0xad, 0x9f, 0x66, 0x8b, 0x89, 0x0a, 0x25, 0x9e,
+	0x03, 0xff, 0xce, 0xcb, 0x8d, 0x96, 0x03, 0xcf, 0xa6, 0xcb, 0x59, 0x9f, 0x98, 0x3b, 0xf7, 0x53,
+	0xdb, 0x42, 0x75, 0xf6, 0xf5, 0xe0, 0x8a, 0xa5, 0x1a, 0xc6, 0x5b, 0x8c, 0x0b, 0x10, 0xee, 0xd5,
+	0xb6, 0xa6, 0x89, 0x83, 0x87, 0xdb, 0xc1, 0x67, 0xa2, 0x77, 0x7b, 0x2a, 0xfa, 0x78, 0x06, 0xdc,
+	0x94, 0xf9, 0x47, 0x25, 0x87, 0xd4, 0x78, 0xd0, 0x6f, 0x08, 0xd0, 0xf7, 0x75, 0xee, 0x6a, 0x1c,
+	0x02, 0xdf, 0xf5, 0x97, 0x4e, 0x7f, 0x19, 0x88, 0x2e, 0x05, 0x11, 0x46, 0x65, 0xfd, 0xf6, 0x48,
+	0x07, 0x8f, 0x14, 0xd5, 0x98, 0x00, 0xb3, 0xb4, 0x94, 0x2b, 0x66, 0x83, 0x32, 0x94, 0xec, 0x95,
+	0xc1, 0x23, 0x10, 0xfe, 0x51, 0x0f, 0xba, 0x92, 0x23, 0x42, 0x51, 0x85, 0x1c, 0x97, 0x97, 0x8d,
+	0xe4, 0x9e, 0x26, 0x8a, 0x6a, 0x3c, 0x86, 0x89, 0xae, 0x68, 0x8d, 0x2e, 0xa4, 0x20, 0x63, 0x07,
+	0xd2, 0x53, 0xe0, 0x74, 0x20, 0xce, 0x61, 0x6c, 0xe2, 0xa3, 0xe9, 0x8c, 0x44, 0xf5, 0x7a, 0x2d,
+	0xe8, 0xa7, 0x2e, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0xe4, 0x93, 0xae, 0x19, 0xb9, 0x01, 0x00,
+	0x00,
+}
diff --git a/internal/userdb/userdb.proto b/internal/userdb/userdb.proto
new file mode 100644
index 0000000..ca663e3
--- /dev/null
+++ b/internal/userdb/userdb.proto
@@ -0,0 +1,28 @@
+
+syntax = "proto3";
+
+package userdb;
+
+message ProtoDB {
+	map<string, Password> users = 1;
+}
+
+message Password {
+	oneof scheme {
+		Scrypt scrypt = 2;
+		Plain plain = 3;
+	}
+}
+
+message Scrypt {
+	uint64 logN = 1;
+	int32 r = 2;
+	int32 p = 3;
+	int32 keyLen = 4;
+	bytes salt = 5;
+	bytes encrypted = 6;
+}
+
+message Plain {
+	bytes password = 1;
+}
diff --git a/internal/userdb/userdb_test.go b/internal/userdb/userdb_test.go
index cec6e58..2c2a0e6 100644
--- a/internal/userdb/userdb_test.go
+++ b/internal/userdb/userdb_test.go
@@ -4,6 +4,7 @@ import (
 	"fmt"
 	"io/ioutil"
 	"os"
+	"reflect"
 	"testing"
 )
 
@@ -32,17 +33,17 @@ func mustCreateDB(t *testing.T, content string) string {
 }
 
 func dbEquals(a, b *DB) bool {
-	if a.users == nil || b.users == nil {
-		return a.users == nil && b.users == nil
+	if a.db == nil || b.db == nil {
+		return a.db == nil && b.db == nil
 	}
 
-	if len(a.users) != len(b.users) {
+	if len(a.db.Users) != len(b.db.Users) {
 		return false
 	}
 
-	for k, av := range a.users {
-		bv, ok := b.users[k]
-		if !ok || av.name != bv.name || av.password != bv.password {
+	for k, av := range a.db.Users {
+		bv, ok := b.db.Users[k]
+		if !ok || !reflect.DeepEqual(av, bv) {
 			return false
 		}
 	}
@@ -51,66 +52,30 @@ func dbEquals(a, b *DB) bool {
 }
 
 var emptyDB = &DB{
-	users: map[string]user{},
+	db: &ProtoDB{Users: map[string]*Password{}},
 }
 
-const (
-	scryptNoSalt = ("#chasquid-userdb-v1\n" +
-		"user1 SCRYPT@n:14,r:8,p:1,l:32, " +
-		"WyZPRd08NPAkWgBuqB5kwK4fEuB6FHu/X1pA1SxnXhc=")
-	scryptInvalidSalt = ("#chasquid-userdb-v1\n" +
-		"user1 SCRYPT@n:99,r:8,p:1,l:16,not-valid$base64!nono== " +
-		"WyZPRd08NPAkWgBuqB5kwK4fEuB6FHu/X1pA1SxnXhc=")
-	scryptMissingR = ("#chasquid-userdb-v1\n" +
-		"user1 SCRYPT@n:14,r:,p:1,l:32,gY3a3PIzehu7xu6KM9PeOQ== " +
-		"WyZPRd08NPAkWgBuqB5kwK4fEuB6FHu/X1pA1SxnXhc=")
-	scryptBadN = ("#chasquid-userdb-v1\n" +
-		"user1 SCRYPT@n:99,r:8,p:1,l:32,gY3a3PIzehu7xu6KM9PeOQ== " +
-		"WyZPRd08NPAkWgBuqB5kwK4fEuB6FHu/X1pA1SxnXhc=")
-	scryptShortKeyLen = ("#chasquid-userdb-v1\n" +
-		"user1 SCRYPT@n:99,r:8,p:1,l:16,gY3a3PIzehu7xu6KM9PeOQ== " +
-		"WyZPRd08NPAkWgBuqB5kwK4fEuB6FHu/X1pA1SxnXhc=")
-)
-
 // Test various cases of loading an empty/broken database.
-func TestLoad(t *testing.T) {
+func TestEmptyLoad(t *testing.T) {
 	cases := []struct {
 		desc     string
 		content  string
 		fatal    bool
 		fatalErr error
-		warns    bool
 	}{
-		{"empty file", "", false, nil, false},
-		{"header \\n", "#chasquid-userdb-v1\n", false, nil, false},
-		{"header \\r\\n", "#chasquid-userdb-v1\r\n", false, nil, false},
-		{"header EOF", "#chasquid-userdb-v1", false, nil, false},
-		{"missing header", "this is not the header",
-			true, ErrMissingHeader, false},
-		{"invalid user", "#chasquid-userdb-v1\nnam\xa0e PLAIN pass\n",
-			false, nil, true},
-		{"too few fields", "#chasquid-userdb-v1\nfield1 field2\n",
-			false, nil, true},
-		{"too many fields", "#chasquid-userdb-v1\nf1 f2 f3 f4\n",
-			false, nil, true},
-		{"unknown scheme", "#chasquid-userdb-v1\nuser SCHEME pass\n",
-			false, nil, true},
-		{"scrypt no salt", scryptNoSalt, false, nil, true},
-		{"scrypt invalid salt", scryptInvalidSalt, false, nil, true},
-		{"scrypt missing R", scryptMissingR, false, nil, true},
-		{"scrypt bad N", scryptBadN, false, nil, true},
-		{"scrypt short key len", scryptShortKeyLen, false, nil, true},
+		{"empty file", "", false, nil},
+		{"invalid ", "users: < invalid >", true, nil},
 	}
 
 	for _, c := range cases {
-		testOneLoad(t, c.desc, c.content, c.fatal, c.fatalErr, c.warns)
+		testOneLoad(t, c.desc, c.content, c.fatal, c.fatalErr)
 	}
 }
 
-func testOneLoad(t *testing.T, desc, content string, fatal bool, fatalErr error, warns bool) {
+func testOneLoad(t *testing.T, desc, content string, fatal bool, fatalErr error) {
 	fname := mustCreateDB(t, content)
 	defer removeIfSuccessful(t, fname)
-	db, warnings, err := Load(fname)
+	db, err := Load(fname)
 	if fatal {
 		if err == nil {
 			t.Errorf("case %q: expected error loading, got nil", desc)
@@ -122,24 +87,13 @@ func testOneLoad(t *testing.T, desc, content string, fatal bool, fatalErr error,
 		t.Fatalf("case %q: error loading database: %v", desc, err)
 	}
 
-	if warns && warnings == nil {
-		t.Errorf("case %q: expected warnings, got nil", desc)
-	} else if !warns {
-		for _, w := range warnings {
-			t.Errorf("case %q: warning loading database: %v", desc, w)
-		}
-	}
-
 	if db != nil && !dbEquals(db, emptyDB) {
-		t.Errorf("case %q: DB not empty: %#v", desc, db)
+		t.Errorf("case %q: DB not empty: %#v", desc, db.db.Users)
 	}
 }
 
 func mustLoad(t *testing.T, fname string) *DB {
-	db, warnings, err := Load(fname)
-	for _, w := range warnings {
-		t.Errorf("warning loading database: %v", w)
-	}
+	db, err := Load(fname)
 	if err != nil {
 		t.Fatalf("error loading database: %v", err)
 	}
@@ -178,8 +132,8 @@ func TestWrite(t *testing.T) {
 		if !db.Exists(name) {
 			t.Errorf("user %q not in database", name)
 		}
-		if _, ok := db.users[name].scheme.(scryptScheme); !ok {
-			t.Errorf("user %q not using scrypt: %#v", name, db.users[name])
+		if db.db.Users[name].GetScheme() == nil {
+			t.Errorf("user %q not using scrypt: %#v", name, db.db.Users[name])
 		}
 	}
 
@@ -210,13 +164,10 @@ func TestNew(t *testing.T) {
 	db1.AddUser("user", "passwd")
 	db1.Write()
 
-	db2, ws, err := Load(fname)
+	db2, err := Load(fname)
 	if err != nil {
 		t.Fatalf("error loading: %v", err)
 	}
-	if len(ws) != 0 {
-		t.Errorf("warnings loading: %v", ws)
-	}
 
 	if !dbEquals(db1, db2) {
 		t.Errorf("databases differ. db1:%v  !=  db2:%v", db1, db2)
@@ -236,12 +187,11 @@ func TestInvalidUsername(t *testing.T) {
 			t.Errorf("AddUser(%q) worked, expected it to fail", name)
 		}
 	}
+}
 
-	// Add an invalid user from behind, and check that Write fails.
-	db.users["in valid"] = user{"in valid", plainScheme{}, "password"}
-	err := db.Write()
-	if err == nil {
-		t.Errorf("Write worked, expected it to fail")
+func plainPassword(p string) *Password {
+	return &Password{
+		Scheme: &Password_Plain{&Plain{[]byte(p)}},
 	}
 }
 
@@ -252,7 +202,7 @@ func TestPlainScheme(t *testing.T) {
 	defer removeIfSuccessful(t, fname)
 	db := mustLoad(t, fname)
 
-	db.users["user"] = user{"user", plainScheme{}, "pass word"}
+	db.db.Users["user"] = plainPassword("pass word")
 	err := db.Write()
 	if err != nil {
 		t.Errorf("Write failed: %v", err)
@@ -268,35 +218,43 @@ func TestPlainScheme(t *testing.T) {
 }
 
 func TestReload(t *testing.T) {
-	content := "#chasquid-userdb-v1\nu1 PLAIN pass\n"
+	content := "users:< key: 'u1' value:< plain:< password: 'pass' >>>"
 	fname := mustCreateDB(t, content)
 	defer removeIfSuccessful(t, fname)
 	db := mustLoad(t, fname)
 
-	// Add some things to the file, including a broken line.
-	content += "u2 UNKNOWN pass\n"
-	content += "u3 PLAIN pass\n"
-	ioutil.WriteFile(fname, []byte(content), db.finfo.Mode())
+	// Add a valid line to the file.
+	content += "users:< key: 'u2' value:< plain:< password: 'pass' >>>"
+	ioutil.WriteFile(fname, []byte(content), 0660)
 
-	warnings, err := db.Reload()
+	err := db.Reload()
 	if err != nil {
 		t.Errorf("Reload failed: %v", err)
 	}
-	if len(warnings) != 1 {
-		t.Errorf("expected 1 warning, got %v", warnings)
+	if len(db.db.Users) != 2 {
+		t.Errorf("expected 2 users, got %d", len(db.db.Users))
+	}
+
+	// And now a broken one.
+	content += "users:< invalid >"
+	ioutil.WriteFile(fname, []byte(content), 0660)
+
+	err = db.Reload()
+	if err == nil {
+		t.Errorf("expected error, got nil")
 	}
-	if len(db.users) != 2 {
-		t.Errorf("expected 2 users, got %d", len(db.users))
+	if len(db.db.Users) != 2 {
+		t.Errorf("expected 2 users, got %d", len(db.db.Users))
 	}
 
-	// Cause an error loading, check the database is not changed.
+	// Cause an even bigger error loading, check the database is not changed.
 	db.fname = "/does/not/exist"
-	warnings, err = db.Reload()
+	err = db.Reload()
 	if err == nil {
 		t.Errorf("expected error, got nil")
 	}
-	if len(db.users) != 2 {
-		t.Errorf("expected 2 users, got %d", len(db.users))
+	if len(db.db.Users) != 2 {
+		t.Errorf("expected 2 users, got %d", len(db.db.Users))
 	}
 
 }
diff --git a/test/.gitignore b/test/.gitignore
new file mode 100644
index 0000000..243d233
--- /dev/null
+++ b/test/.gitignore
@@ -0,0 +1,4 @@
+
+# Ignore the user databases - we create them each time.
+t-*/config/**/users
+
diff --git a/test/t-01-simple_local/config/domains/testserver/users b/test/t-01-simple_local/config/domains/testserver/users
deleted file mode 100644
index bd031fe..0000000
--- a/test/t-01-simple_local/config/domains/testserver/users
+++ /dev/null
@@ -1,2 +0,0 @@
-#chasquid-userdb-v1
-user SCRYPT@n:14,r:8,p:1,l:32,r00XqNmRkV505R2X6KT8+Q== aAiBBIVNNzmDXwxLLdJezFuxGtc2/wcHsy3FiOMAH4c=
diff --git a/test/t-01-simple_local/run.sh b/test/t-01-simple_local/run.sh
index d52e6dd..f773ff8 100755
--- a/test/t-01-simple_local/run.sh
+++ b/test/t-01-simple_local/run.sh
@@ -6,6 +6,7 @@ set -e
 init
 
 generate_certs_for testserver
+add_user testserver user secretpassword
 
 mkdir -p .logs
 chasquid -v=2 --log_dir=.logs --config_dir=config &
diff --git a/test/t-02-exim/config/domains/srv-chasquid/users b/test/t-02-exim/config/domains/srv-chasquid/users
deleted file mode 100644
index bd031fe..0000000
--- a/test/t-02-exim/config/domains/srv-chasquid/users
+++ /dev/null
@@ -1,2 +0,0 @@
-#chasquid-userdb-v1
-user SCRYPT@n:14,r:8,p:1,l:32,r00XqNmRkV505R2X6KT8+Q== aAiBBIVNNzmDXwxLLdJezFuxGtc2/wcHsy3FiOMAH4c=
diff --git a/test/t-02-exim/run.sh b/test/t-02-exim/run.sh
index a1c2caa..f689395 100755
--- a/test/t-02-exim/run.sh
+++ b/test/t-02-exim/run.sh
@@ -33,6 +33,7 @@ mkdir -p .exim4
 EXIMDIR="$PWD/.exim4" envsubst < config/exim4.in > .exim4/config
 
 generate_certs_for srv-chasquid
+add_user srv-chasquid user secretpassword
 
 # Launch chasquid at port 1025 (in config).
 # Use outgoing port 2025 which is where exim will be at.
diff --git a/test/util/lib.sh b/test/util/lib.sh
index 07c4913..e6ddb7e 100644
--- a/test/util/lib.sh
+++ b/test/util/lib.sh
@@ -32,6 +32,14 @@ function chasquid() {
 		go run ${TBASE}/../../chasquid.go "$@"
 }
 
+function add_user() {
+	go run ${TBASE}/../../cmd/chasquid-userdb/chasquid-userdb.go \
+		--database "config/domains/${1}/users" \
+		--add_user "${2}" \
+		--password "${3}" \
+		>> .add_user_logs
+}
+
 function run_msmtp() {
 	# msmtp will check that the rc file is only user readable.
 	chmod 600 msmtprc