mirror of
https://github.com/hoernschen/dendrite.git
synced 2025-07-31 13:22:46 +00:00
Support sqlite in addition to postgres (#869)
* Move current work into single branch * Initial massaging of clientapi etc (not working yet) * Interfaces for accounts/devices databases * Duplicate postgres package for sqlite3 (no changes made to it yet) * Some keydb, accountdb, devicedb, common partition fixes, some more syncapi tweaking * Fix accounts DB, device DB * Update naffka dependency for SQLite * Naffka SQLite * Update naffka to latest master * SQLite support for federationsender * Mostly not-bad support for SQLite in syncapi (although there are problems where lots of events get classed incorrectly as backward extremities, probably because of IN/ANY clauses that are badly supported) * Update Dockerfile -> Go 1.13.7, add build-base (as gcc and friends are needed for SQLite) * Implement GET endpoints for account_data in clientapi * Nuke filtering for now... * Revert "Implement GET endpoints for account_data in clientapi" This reverts commit 4d80dff4583d278620d9b3ed437e9fcd8d4674ee. * Implement GET endpoints for account_data in clientapi (#861) * Implement GET endpoints for account_data in clientapi * Fix accountDB parameter * Remove fmt.Println * Fix insertAccountData SQLite query * Fix accountDB storage interfaces * Add empty push rules into account data on account creation (#862) * Put SaveAccountData into the right function this time * Not sure if roomserver is better or worse now * sqlite work * Allow empty last sent ID for the first event * sqlite: room creation works * Support sending messages * Nuke fmt.println * Move QueryVariadic etc into common, other device fixes * Fix some linter issues * Fix bugs * Fix some linting errors * Fix errcheck lint errors * Make naffka use postgres as fallback, fix couple of compile errors * What on earth happened to the /rooms/{roomID}/send/{eventType} routing Co-authored-by: Neil Alexander <neilalexander@users.noreply.github.com>
This commit is contained in:
parent
6942ee1de0
commit
b6ea1bc67a
103 changed files with 9467 additions and 710 deletions
|
@ -18,6 +18,7 @@ import (
|
|||
"database/sql"
|
||||
"io"
|
||||
"net/http"
|
||||
"net/url"
|
||||
|
||||
"golang.org/x/crypto/ed25519"
|
||||
|
||||
|
@ -68,7 +69,13 @@ func NewBaseDendrite(cfg *config.Dendrite, componentName string) *BaseDendrite {
|
|||
logrus.WithError(err).Panicf("failed to start opentracing")
|
||||
}
|
||||
|
||||
kafkaConsumer, kafkaProducer := setupKafka(cfg)
|
||||
var kafkaConsumer sarama.Consumer
|
||||
var kafkaProducer sarama.SyncProducer
|
||||
if cfg.Kafka.UseNaffka {
|
||||
kafkaConsumer, kafkaProducer = setupNaffka(cfg)
|
||||
} else {
|
||||
kafkaConsumer, kafkaProducer = setupKafka(cfg)
|
||||
}
|
||||
|
||||
return &BaseDendrite{
|
||||
componentName: componentName,
|
||||
|
@ -118,7 +125,7 @@ func (b *BaseDendrite) CreateHTTPFederationSenderAPIs() federationSenderAPI.Fede
|
|||
|
||||
// CreateDeviceDB creates a new instance of the device database. Should only be
|
||||
// called once per component.
|
||||
func (b *BaseDendrite) CreateDeviceDB() *devices.Database {
|
||||
func (b *BaseDendrite) CreateDeviceDB() devices.Database {
|
||||
db, err := devices.NewDatabase(string(b.Cfg.Database.Device), b.Cfg.Matrix.ServerName)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panicf("failed to connect to devices db")
|
||||
|
@ -129,7 +136,7 @@ func (b *BaseDendrite) CreateDeviceDB() *devices.Database {
|
|||
|
||||
// CreateAccountsDB creates a new instance of the accounts database. Should only
|
||||
// be called once per component.
|
||||
func (b *BaseDendrite) CreateAccountsDB() *accounts.Database {
|
||||
func (b *BaseDendrite) CreateAccountsDB() accounts.Database {
|
||||
db, err := accounts.NewDatabase(string(b.Cfg.Database.Account), b.Cfg.Matrix.ServerName)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panicf("failed to connect to accounts db")
|
||||
|
@ -186,28 +193,8 @@ func (b *BaseDendrite) SetupAndServeHTTP(bindaddr string, listenaddr string) {
|
|||
logrus.Infof("Stopped %s server on %s", b.componentName, addr)
|
||||
}
|
||||
|
||||
// setupKafka creates kafka consumer/producer pair from the config. Checks if
|
||||
// should use naffka.
|
||||
// setupKafka creates kafka consumer/producer pair from the config.
|
||||
func setupKafka(cfg *config.Dendrite) (sarama.Consumer, sarama.SyncProducer) {
|
||||
if cfg.Kafka.UseNaffka {
|
||||
db, err := sql.Open("postgres", string(cfg.Database.Naffka))
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panic("Failed to open naffka database")
|
||||
}
|
||||
|
||||
naffkaDB, err := naffka.NewPostgresqlDatabase(db)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panic("Failed to setup naffka database")
|
||||
}
|
||||
|
||||
naff, err := naffka.New(naffkaDB)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panic("Failed to setup naffka")
|
||||
}
|
||||
|
||||
return naff, naff
|
||||
}
|
||||
|
||||
consumer, err := sarama.NewConsumer(cfg.Kafka.Addresses, nil)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panic("failed to start kafka consumer")
|
||||
|
@ -220,3 +207,44 @@ func setupKafka(cfg *config.Dendrite) (sarama.Consumer, sarama.SyncProducer) {
|
|||
|
||||
return consumer, producer
|
||||
}
|
||||
|
||||
// setupNaffka creates kafka consumer/producer pair from the config.
|
||||
func setupNaffka(cfg *config.Dendrite) (sarama.Consumer, sarama.SyncProducer) {
|
||||
var err error
|
||||
var db *sql.DB
|
||||
var naffkaDB *naffka.DatabaseImpl
|
||||
|
||||
uri, err := url.Parse(string(cfg.Database.Naffka))
|
||||
if err != nil || uri.Scheme == "file" {
|
||||
db, err = sql.Open("sqlite3", string(cfg.Database.Naffka))
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panic("Failed to open naffka database")
|
||||
}
|
||||
|
||||
naffkaDB, err = naffka.NewSqliteDatabase(db)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panic("Failed to setup naffka database")
|
||||
}
|
||||
} else {
|
||||
db, err = sql.Open("postgres", string(cfg.Database.Naffka))
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panic("Failed to open naffka database")
|
||||
}
|
||||
|
||||
naffkaDB, err = naffka.NewPostgresqlDatabase(db)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panic("Failed to setup naffka database")
|
||||
}
|
||||
}
|
||||
|
||||
if naffkaDB == nil {
|
||||
panic("naffka connection string not understood")
|
||||
}
|
||||
|
||||
naff, err := naffka.New(naffkaDB)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panic("Failed to setup naffka")
|
||||
}
|
||||
|
||||
return naff, naff
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ import (
|
|||
"golang.org/x/crypto/ed25519"
|
||||
|
||||
"github.com/matrix-org/dendrite/common/keydb/postgres"
|
||||
"github.com/matrix-org/dendrite/common/keydb/sqlite3"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
)
|
||||
|
||||
|
@ -44,6 +45,8 @@ func NewDatabase(
|
|||
switch uri.Scheme {
|
||||
case "postgres":
|
||||
return postgres.NewDatabase(dataSourceName, serverName, serverKey, serverKeyID)
|
||||
case "file":
|
||||
return sqlite3.NewDatabase(dataSourceName, serverName, serverKey, serverKeyID)
|
||||
default:
|
||||
return postgres.NewDatabase(dataSourceName, serverName, serverKey, serverKeyID)
|
||||
}
|
||||
|
|
115
common/keydb/sqlite3/keydb.go
Normal file
115
common/keydb/sqlite3/keydb.go
Normal file
|
@ -0,0 +1,115 @@
|
|||
// Copyright 2017-2018 New Vector Ltd
|
||||
// Copyright 2019-2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package sqlite3
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"math"
|
||||
|
||||
"golang.org/x/crypto/ed25519"
|
||||
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
|
||||
_ "github.com/mattn/go-sqlite3"
|
||||
)
|
||||
|
||||
// A Database implements gomatrixserverlib.KeyDatabase and is used to store
|
||||
// the public keys for other matrix servers.
|
||||
type Database struct {
|
||||
statements serverKeyStatements
|
||||
}
|
||||
|
||||
// NewDatabase prepares a new key database.
|
||||
// It creates the necessary tables if they don't already exist.
|
||||
// It prepares all the SQL statements that it will use.
|
||||
// Returns an error if there was a problem talking to the database.
|
||||
func NewDatabase(
|
||||
dataSourceName string,
|
||||
serverName gomatrixserverlib.ServerName,
|
||||
serverKey ed25519.PublicKey,
|
||||
serverKeyID gomatrixserverlib.KeyID,
|
||||
) (*Database, error) {
|
||||
db, err := sql.Open("sqlite3", dataSourceName)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
d := &Database{}
|
||||
err = d.statements.prepare(db)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// Store our own keys so that we don't end up making HTTP requests to find our
|
||||
// own keys
|
||||
index := gomatrixserverlib.PublicKeyLookupRequest{
|
||||
ServerName: serverName,
|
||||
KeyID: serverKeyID,
|
||||
}
|
||||
value := gomatrixserverlib.PublicKeyLookupResult{
|
||||
VerifyKey: gomatrixserverlib.VerifyKey{
|
||||
Key: gomatrixserverlib.Base64String(serverKey),
|
||||
},
|
||||
ValidUntilTS: math.MaxUint64 >> 1,
|
||||
ExpiredTS: gomatrixserverlib.PublicKeyNotExpired,
|
||||
}
|
||||
err = d.StoreKeys(
|
||||
context.Background(),
|
||||
map[gomatrixserverlib.PublicKeyLookupRequest]gomatrixserverlib.PublicKeyLookupResult{
|
||||
index: value,
|
||||
},
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return d, nil
|
||||
}
|
||||
|
||||
// FetcherName implements KeyFetcher
|
||||
func (d Database) FetcherName() string {
|
||||
return "KeyDatabase"
|
||||
}
|
||||
|
||||
// FetchKeys implements gomatrixserverlib.KeyDatabase
|
||||
func (d *Database) FetchKeys(
|
||||
ctx context.Context,
|
||||
requests map[gomatrixserverlib.PublicKeyLookupRequest]gomatrixserverlib.Timestamp,
|
||||
) (map[gomatrixserverlib.PublicKeyLookupRequest]gomatrixserverlib.PublicKeyLookupResult, error) {
|
||||
return d.statements.bulkSelectServerKeys(ctx, requests)
|
||||
}
|
||||
|
||||
// StoreKeys implements gomatrixserverlib.KeyDatabase
|
||||
func (d *Database) StoreKeys(
|
||||
ctx context.Context,
|
||||
keyMap map[gomatrixserverlib.PublicKeyLookupRequest]gomatrixserverlib.PublicKeyLookupResult,
|
||||
) error {
|
||||
// TODO: Inserting all the keys within a single transaction may
|
||||
// be more efficient since the transaction overhead can be quite
|
||||
// high for a single insert statement.
|
||||
var lastErr error
|
||||
for request, keys := range keyMap {
|
||||
if err := d.statements.upsertServerKeys(ctx, request, keys); err != nil {
|
||||
// Rather than returning immediately on error we try to insert the
|
||||
// remaining keys.
|
||||
// Since we are inserting the keys outside of a transaction it is
|
||||
// possible for some of the inserts to succeed even though some
|
||||
// of the inserts have failed.
|
||||
// Ensuring that we always insert all the keys we can means that
|
||||
// this behaviour won't depend on the iteration order of the map.
|
||||
lastErr = err
|
||||
}
|
||||
}
|
||||
return lastErr
|
||||
}
|
142
common/keydb/sqlite3/server_key_table.go
Normal file
142
common/keydb/sqlite3/server_key_table.go
Normal file
|
@ -0,0 +1,142 @@
|
|||
// Copyright 2017-2018 New Vector Ltd
|
||||
// Copyright 2019-2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package sqlite3
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
|
||||
"github.com/lib/pq"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
)
|
||||
|
||||
const serverKeysSchema = `
|
||||
-- A cache of signing keys downloaded from remote servers.
|
||||
CREATE TABLE IF NOT EXISTS keydb_server_keys (
|
||||
-- The name of the matrix server the key is for.
|
||||
server_name TEXT NOT NULL,
|
||||
-- The ID of the server key.
|
||||
server_key_id TEXT NOT NULL,
|
||||
-- Combined server name and key ID separated by the ASCII unit separator
|
||||
-- to make it easier to run bulk queries.
|
||||
server_name_and_key_id TEXT NOT NULL,
|
||||
-- When the key is valid until as a millisecond timestamp.
|
||||
-- 0 if this is an expired key (in which case expired_ts will be non-zero)
|
||||
valid_until_ts BIGINT NOT NULL,
|
||||
-- When the key expired as a millisecond timestamp.
|
||||
-- 0 if this is an active key (in which case valid_until_ts will be non-zero)
|
||||
expired_ts BIGINT NOT NULL,
|
||||
-- The base64-encoded public key.
|
||||
server_key TEXT NOT NULL,
|
||||
UNIQUE (server_name, server_key_id)
|
||||
);
|
||||
|
||||
CREATE INDEX IF NOT EXISTS keydb_server_name_and_key_id ON keydb_server_keys (server_name_and_key_id);
|
||||
`
|
||||
|
||||
const bulkSelectServerKeysSQL = "" +
|
||||
"SELECT server_name, server_key_id, valid_until_ts, expired_ts, " +
|
||||
" server_key FROM keydb_server_keys" +
|
||||
" WHERE server_name_and_key_id IN ($1)"
|
||||
|
||||
const upsertServerKeysSQL = "" +
|
||||
"INSERT INTO keydb_server_keys (server_name, server_key_id," +
|
||||
" server_name_and_key_id, valid_until_ts, expired_ts, server_key)" +
|
||||
" VALUES ($1, $2, $3, $4, $5, $6)" +
|
||||
" ON CONFLICT (server_name, server_key_id)" +
|
||||
" DO UPDATE SET valid_until_ts = $4, expired_ts = $5, server_key = $6"
|
||||
|
||||
type serverKeyStatements struct {
|
||||
bulkSelectServerKeysStmt *sql.Stmt
|
||||
upsertServerKeysStmt *sql.Stmt
|
||||
}
|
||||
|
||||
func (s *serverKeyStatements) prepare(db *sql.DB) (err error) {
|
||||
_, err = db.Exec(serverKeysSchema)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
if s.bulkSelectServerKeysStmt, err = db.Prepare(bulkSelectServerKeysSQL); err != nil {
|
||||
return
|
||||
}
|
||||
if s.upsertServerKeysStmt, err = db.Prepare(upsertServerKeysSQL); err != nil {
|
||||
return
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (s *serverKeyStatements) bulkSelectServerKeys(
|
||||
ctx context.Context,
|
||||
requests map[gomatrixserverlib.PublicKeyLookupRequest]gomatrixserverlib.Timestamp,
|
||||
) (map[gomatrixserverlib.PublicKeyLookupRequest]gomatrixserverlib.PublicKeyLookupResult, error) {
|
||||
var nameAndKeyIDs []string
|
||||
for request := range requests {
|
||||
nameAndKeyIDs = append(nameAndKeyIDs, nameAndKeyID(request))
|
||||
}
|
||||
stmt := s.bulkSelectServerKeysStmt
|
||||
rows, err := stmt.QueryContext(ctx, pq.StringArray(nameAndKeyIDs))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer rows.Close() // nolint: errcheck
|
||||
results := map[gomatrixserverlib.PublicKeyLookupRequest]gomatrixserverlib.PublicKeyLookupResult{}
|
||||
for rows.Next() {
|
||||
var serverName string
|
||||
var keyID string
|
||||
var key string
|
||||
var validUntilTS int64
|
||||
var expiredTS int64
|
||||
if err = rows.Scan(&serverName, &keyID, &validUntilTS, &expiredTS, &key); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
r := gomatrixserverlib.PublicKeyLookupRequest{
|
||||
ServerName: gomatrixserverlib.ServerName(serverName),
|
||||
KeyID: gomatrixserverlib.KeyID(keyID),
|
||||
}
|
||||
vk := gomatrixserverlib.VerifyKey{}
|
||||
err = vk.Key.Decode(key)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
results[r] = gomatrixserverlib.PublicKeyLookupResult{
|
||||
VerifyKey: vk,
|
||||
ValidUntilTS: gomatrixserverlib.Timestamp(validUntilTS),
|
||||
ExpiredTS: gomatrixserverlib.Timestamp(expiredTS),
|
||||
}
|
||||
}
|
||||
return results, nil
|
||||
}
|
||||
|
||||
func (s *serverKeyStatements) upsertServerKeys(
|
||||
ctx context.Context,
|
||||
request gomatrixserverlib.PublicKeyLookupRequest,
|
||||
key gomatrixserverlib.PublicKeyLookupResult,
|
||||
) error {
|
||||
_, err := s.upsertServerKeysStmt.ExecContext(
|
||||
ctx,
|
||||
string(request.ServerName),
|
||||
string(request.KeyID),
|
||||
nameAndKeyID(request),
|
||||
key.ValidUntilTS,
|
||||
key.ExpiredTS,
|
||||
key.Key.Encode(),
|
||||
)
|
||||
return err
|
||||
}
|
||||
|
||||
func nameAndKeyID(request gomatrixserverlib.PublicKeyLookupRequest) string {
|
||||
return string(request.ServerName) + "\x1F" + string(request.KeyID)
|
||||
}
|
|
@ -29,7 +29,7 @@ CREATE TABLE IF NOT EXISTS ${prefix}_partition_offsets (
|
|||
partition INTEGER NOT NULL,
|
||||
-- The 64-bit offset.
|
||||
partition_offset BIGINT NOT NULL,
|
||||
CONSTRAINT ${prefix}_topic_partition_unique UNIQUE (topic, partition)
|
||||
UNIQUE (topic, partition)
|
||||
);
|
||||
`
|
||||
|
||||
|
@ -38,7 +38,7 @@ const selectPartitionOffsetsSQL = "" +
|
|||
|
||||
const upsertPartitionOffsetsSQL = "" +
|
||||
"INSERT INTO ${prefix}_partition_offsets (topic, partition, partition_offset) VALUES ($1, $2, $3)" +
|
||||
" ON CONFLICT ON CONSTRAINT ${prefix}_topic_partition_unique" +
|
||||
" ON CONFLICT (topic, partition)" +
|
||||
" DO UPDATE SET partition_offset = $3"
|
||||
|
||||
// PartitionOffsetStatements represents a set of statements that can be run on a partition_offsets table.
|
||||
|
|
|
@ -16,6 +16,7 @@ package common
|
|||
|
||||
import (
|
||||
"database/sql"
|
||||
"fmt"
|
||||
|
||||
"github.com/lib/pq"
|
||||
)
|
||||
|
@ -30,11 +31,13 @@ type Transaction interface {
|
|||
|
||||
// EndTransaction ends a transaction.
|
||||
// If the transaction succeeded then it is committed, otherwise it is rolledback.
|
||||
func EndTransaction(txn Transaction, succeeded *bool) {
|
||||
// You MUST check the error returned from this function to be sure that the transaction
|
||||
// was applied correctly. For example, 'database is locked' errors in sqlite will happen here.
|
||||
func EndTransaction(txn Transaction, succeeded *bool) error {
|
||||
if *succeeded {
|
||||
txn.Commit() // nolint: errcheck
|
||||
return txn.Commit() // nolint: errcheck
|
||||
} else {
|
||||
txn.Rollback() // nolint: errcheck
|
||||
return txn.Rollback() // nolint: errcheck
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -47,7 +50,12 @@ func WithTransaction(db *sql.DB, fn func(txn *sql.Tx) error) (err error) {
|
|||
return
|
||||
}
|
||||
succeeded := false
|
||||
defer EndTransaction(txn, &succeeded)
|
||||
defer func() {
|
||||
err2 := EndTransaction(txn, &succeeded)
|
||||
if err == nil && err2 != nil { // failed to commit/rollback
|
||||
err = err2
|
||||
}
|
||||
}()
|
||||
|
||||
err = fn(txn)
|
||||
if err != nil {
|
||||
|
@ -74,3 +82,20 @@ func IsUniqueConstraintViolationErr(err error) bool {
|
|||
pqErr, ok := err.(*pq.Error)
|
||||
return ok && pqErr.Code == "23505"
|
||||
}
|
||||
|
||||
// Hack of the century
|
||||
func QueryVariadic(count int) string {
|
||||
return QueryVariadicOffset(count, 0)
|
||||
}
|
||||
|
||||
func QueryVariadicOffset(count, offset int) string {
|
||||
str := "("
|
||||
for i := 0; i < count; i++ {
|
||||
str += fmt.Sprintf("$%d", i+offset+1)
|
||||
if i < (count - 1) {
|
||||
str += ", "
|
||||
}
|
||||
}
|
||||
str += ")"
|
||||
return str
|
||||
}
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue