mirror of
https://github.com/pomerium/pomerium.git
synced 2025-05-19 12:07:18 +02:00
postgres: databroker storage backend (#3370)
* wip * storage: add filtering to SyncLatest * don't increment the record version, so intermediate changes are requested * databroker: add support for query filtering * fill server and record version * postgres: databroker storage backend * wip * serialize puts * add test * skip tests for macos * add test * return error from protojson * set data * exclude postgres from cover tests
This commit is contained in:
parent
550698b1ca
commit
1c2aad2de6
21 changed files with 1573 additions and 17 deletions
358
pkg/storage/postgres/backend.go
Normal file
358
pkg/storage/postgres/backend.go
Normal file
|
@ -0,0 +1,358 @@
|
|||
package postgres
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/cenkalti/backoff/v4"
|
||||
"github.com/jackc/pgx/v4"
|
||||
"github.com/jackc/pgx/v4/pgxpool"
|
||||
"google.golang.org/protobuf/types/known/timestamppb"
|
||||
|
||||
"github.com/pomerium/pomerium/internal/log"
|
||||
"github.com/pomerium/pomerium/internal/signal"
|
||||
"github.com/pomerium/pomerium/pkg/contextutil"
|
||||
"github.com/pomerium/pomerium/pkg/grpc/databroker"
|
||||
"github.com/pomerium/pomerium/pkg/storage"
|
||||
)
|
||||
|
||||
// Backend is a storage Backend implemented with Postgres.
|
||||
type Backend struct {
|
||||
cfg *config
|
||||
dsn string
|
||||
onChange *signal.Signal
|
||||
|
||||
closeCtx context.Context
|
||||
close context.CancelFunc
|
||||
|
||||
mu sync.RWMutex
|
||||
pool *pgxpool.Pool
|
||||
serverVersion uint64
|
||||
}
|
||||
|
||||
// New creates a new Backend.
|
||||
func New(dsn string, options ...Option) *Backend {
|
||||
backend := &Backend{
|
||||
cfg: getConfig(options...),
|
||||
dsn: dsn,
|
||||
onChange: signal.New(),
|
||||
}
|
||||
backend.closeCtx, backend.close = context.WithCancel(context.Background())
|
||||
go backend.doPeriodically(func(ctx context.Context) error {
|
||||
_, pool, err := backend.init(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return deleteChangesBefore(ctx, pool, time.Now().Add(-backend.cfg.expiry))
|
||||
}, time.Minute)
|
||||
go backend.doPeriodically(func(ctx context.Context) error {
|
||||
_, pool, err := backend.init(backend.closeCtx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
conn, err := pool.Acquire(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer conn.Release()
|
||||
|
||||
_, err = conn.Exec(ctx, `LISTEN `+recordChangeNotifyName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
_, err = conn.Conn().WaitForNotification(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
backend.onChange.Broadcast(ctx)
|
||||
|
||||
return nil
|
||||
}, time.Millisecond*100)
|
||||
return backend
|
||||
}
|
||||
|
||||
// Close closes the underlying database connection.
|
||||
func (backend *Backend) Close() error {
|
||||
backend.mu.Lock()
|
||||
defer backend.mu.Unlock()
|
||||
|
||||
backend.close()
|
||||
|
||||
if backend.pool != nil {
|
||||
backend.pool.Close()
|
||||
backend.pool = nil
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Get gets a record from the database.
|
||||
func (backend *Backend) Get(
|
||||
ctx context.Context,
|
||||
recordType, recordID string,
|
||||
) (*databroker.Record, error) {
|
||||
ctx, cancel := contextutil.Merge(ctx, backend.closeCtx)
|
||||
defer cancel()
|
||||
|
||||
_, conn, err := backend.init(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return getRecord(ctx, conn, recordType, recordID)
|
||||
}
|
||||
|
||||
// GetOptions returns the options for the given record type.
|
||||
func (backend *Backend) GetOptions(
|
||||
ctx context.Context,
|
||||
recordType string,
|
||||
) (*databroker.Options, error) {
|
||||
ctx, cancel := contextutil.Merge(ctx, backend.closeCtx)
|
||||
defer cancel()
|
||||
|
||||
_, conn, err := backend.init(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return getOptions(ctx, conn, recordType)
|
||||
}
|
||||
|
||||
// Lease attempts to acquire a lease for the given name.
|
||||
func (backend *Backend) Lease(
|
||||
ctx context.Context,
|
||||
leaseName, leaseID string,
|
||||
ttl time.Duration,
|
||||
) (acquired bool, err error) {
|
||||
ctx, cancel := contextutil.Merge(ctx, backend.closeCtx)
|
||||
defer cancel()
|
||||
|
||||
_, conn, err := backend.init(ctx)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
leaseHolderID, err := maybeAcquireLease(ctx, conn, leaseName, leaseID, ttl)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
return leaseHolderID == leaseID, nil
|
||||
}
|
||||
|
||||
// Put puts a record into Postgres.
|
||||
func (backend *Backend) Put(
|
||||
ctx context.Context,
|
||||
records []*databroker.Record,
|
||||
) (serverVersion uint64, err error) {
|
||||
ctx, cancel := contextutil.Merge(ctx, backend.closeCtx)
|
||||
defer cancel()
|
||||
|
||||
serverVersion, pool, err := backend.init(ctx)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
err = pool.BeginTxFunc(ctx, pgx.TxOptions{
|
||||
IsoLevel: pgx.Serializable,
|
||||
AccessMode: pgx.ReadWrite,
|
||||
}, func(tx pgx.Tx) error {
|
||||
now := timestamppb.Now()
|
||||
|
||||
recordVersion, err := getLatestRecordVersion(ctx, tx)
|
||||
if err != nil {
|
||||
return fmt.Errorf("storage/postgres: error getting latest record version: %w", err)
|
||||
}
|
||||
|
||||
// add all the records
|
||||
recordTypes := map[string]struct{}{}
|
||||
for i, record := range records {
|
||||
recordTypes[record.GetType()] = struct{}{}
|
||||
|
||||
record = dup(record)
|
||||
record.ModifiedAt = now
|
||||
record.Version = recordVersion + uint64(i) + 1
|
||||
err := putRecordChange(ctx, tx, record)
|
||||
if err != nil {
|
||||
return fmt.Errorf("storage/postgres: error saving record change: %w", err)
|
||||
}
|
||||
|
||||
err = putRecord(ctx, tx, record)
|
||||
if err != nil {
|
||||
return fmt.Errorf("storage/postgres: error saving record: %w", err)
|
||||
}
|
||||
records[i] = record
|
||||
}
|
||||
|
||||
// enforce options for each record type
|
||||
for recordType := range recordTypes {
|
||||
options, err := getOptions(ctx, tx, recordType)
|
||||
if err != nil {
|
||||
return fmt.Errorf("storage/postgres: error getting options: %w", err)
|
||||
}
|
||||
err = enforceOptions(ctx, tx, recordType, options)
|
||||
if err != nil {
|
||||
return fmt.Errorf("storage/postgres: error enforcing options: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return serverVersion, err
|
||||
}
|
||||
|
||||
err = signalRecordChange(ctx, pool)
|
||||
return serverVersion, err
|
||||
}
|
||||
|
||||
// SetOptions sets the options for the given record type.
|
||||
func (backend *Backend) SetOptions(
|
||||
ctx context.Context,
|
||||
recordType string,
|
||||
options *databroker.Options,
|
||||
) error {
|
||||
ctx, cancel := contextutil.Merge(ctx, backend.closeCtx)
|
||||
defer cancel()
|
||||
|
||||
_, conn, err := backend.init(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return setOptions(ctx, conn, recordType, options)
|
||||
}
|
||||
|
||||
// Sync syncs the records.
|
||||
func (backend *Backend) Sync(
|
||||
ctx context.Context,
|
||||
serverVersion, recordVersion uint64,
|
||||
) (storage.RecordStream, error) {
|
||||
// the original ctx will be used for the stream, this ctx used for pre-stream calls
|
||||
callCtx, cancel := contextutil.Merge(ctx, backend.closeCtx)
|
||||
defer cancel()
|
||||
|
||||
currentServerVersion, _, err := backend.init(callCtx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if currentServerVersion != serverVersion {
|
||||
return nil, storage.ErrInvalidServerVersion
|
||||
}
|
||||
|
||||
return newChangedRecordStream(ctx, backend, recordVersion), nil
|
||||
}
|
||||
|
||||
// SyncLatest syncs the latest version of each record.
|
||||
func (backend *Backend) SyncLatest(
|
||||
ctx context.Context,
|
||||
recordType string,
|
||||
expr storage.FilterExpression,
|
||||
) (serverVersion, recordVersion uint64, stream storage.RecordStream, err error) {
|
||||
// the original ctx will be used for the stream, this ctx used for pre-stream calls
|
||||
callCtx, cancel := contextutil.Merge(ctx, backend.closeCtx)
|
||||
defer cancel()
|
||||
|
||||
serverVersion, pool, err := backend.init(callCtx)
|
||||
if err != nil {
|
||||
return 0, 0, nil, err
|
||||
}
|
||||
|
||||
recordVersion, err = getLatestRecordVersion(callCtx, pool)
|
||||
if err != nil {
|
||||
return 0, 0, nil, err
|
||||
}
|
||||
|
||||
if recordType != "" {
|
||||
f := storage.EqualsFilterExpression{
|
||||
Fields: []string{"type"},
|
||||
Value: recordType,
|
||||
}
|
||||
if expr != nil {
|
||||
expr = storage.AndFilterExpression{expr, f}
|
||||
} else {
|
||||
expr = f
|
||||
}
|
||||
}
|
||||
|
||||
stream = newRecordStream(ctx, backend, expr)
|
||||
return serverVersion, recordVersion, stream, nil
|
||||
}
|
||||
|
||||
func (backend *Backend) init(ctx context.Context) (serverVersion uint64, pool *pgxpool.Pool, err error) {
|
||||
backend.mu.RLock()
|
||||
serverVersion = backend.serverVersion
|
||||
pool = backend.pool
|
||||
backend.mu.RUnlock()
|
||||
|
||||
if pool != nil {
|
||||
return serverVersion, pool, nil
|
||||
}
|
||||
|
||||
backend.mu.Lock()
|
||||
defer backend.mu.Unlock()
|
||||
|
||||
// double-checked locking, might have already initialized, so just return
|
||||
serverVersion = backend.serverVersion
|
||||
pool = backend.pool
|
||||
if pool != nil {
|
||||
return serverVersion, pool, nil
|
||||
}
|
||||
|
||||
config, err := pgxpool.ParseConfig(backend.dsn)
|
||||
if err != nil {
|
||||
return serverVersion, nil, err
|
||||
}
|
||||
|
||||
pool, err = pgxpool.ConnectConfig(context.Background(), config)
|
||||
if err != nil {
|
||||
return serverVersion, nil, err
|
||||
}
|
||||
|
||||
err = pool.BeginFunc(ctx, func(tx pgx.Tx) error {
|
||||
var err error
|
||||
serverVersion, err = migrate(ctx, tx)
|
||||
return err
|
||||
})
|
||||
if err != nil {
|
||||
return serverVersion, nil, err
|
||||
}
|
||||
|
||||
backend.serverVersion = serverVersion
|
||||
backend.pool = pool
|
||||
return serverVersion, pool, nil
|
||||
}
|
||||
|
||||
func (backend *Backend) doPeriodically(f func(ctx context.Context) error, dur time.Duration) {
|
||||
ctx := backend.closeCtx
|
||||
|
||||
ticker := time.NewTicker(dur)
|
||||
defer ticker.Stop()
|
||||
|
||||
bo := backoff.NewExponentialBackOff()
|
||||
bo.MaxElapsedTime = 0
|
||||
|
||||
for {
|
||||
err := f(ctx)
|
||||
if err == nil {
|
||||
bo.Reset()
|
||||
select {
|
||||
case <-backend.closeCtx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
}
|
||||
} else {
|
||||
log.Error(ctx).Err(err).Msg("storage/postgres")
|
||||
select {
|
||||
case <-backend.closeCtx.Done():
|
||||
return
|
||||
case <-time.After(bo.NextBackOff()):
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
148
pkg/storage/postgres/backend_test.go
Normal file
148
pkg/storage/postgres/backend_test.go
Normal file
|
@ -0,0 +1,148 @@
|
|||
package postgres
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"runtime"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"google.golang.org/protobuf/proto"
|
||||
"google.golang.org/protobuf/types/known/structpb"
|
||||
|
||||
"github.com/pomerium/pomerium/internal/testutil"
|
||||
"github.com/pomerium/pomerium/pkg/grpc/databroker"
|
||||
"github.com/pomerium/pomerium/pkg/protoutil"
|
||||
"github.com/pomerium/pomerium/pkg/storage"
|
||||
)
|
||||
|
||||
func TestBackend(t *testing.T) {
|
||||
if os.Getenv("GITHUB_ACTION") != "" && runtime.GOOS == "darwin" {
|
||||
t.Skip("Github action can not run docker on MacOS")
|
||||
}
|
||||
|
||||
ctx, clearTimeout := context.WithTimeout(context.Background(), time.Second*10)
|
||||
defer clearTimeout()
|
||||
|
||||
require.NoError(t, testutil.WithTestPostgres(func(dsn string) error {
|
||||
backend := New(dsn)
|
||||
defer backend.Close()
|
||||
|
||||
t.Run("put", func(t *testing.T) {
|
||||
serverVersion, err := backend.Put(ctx, []*databroker.Record{
|
||||
{Type: "test-1", Id: "r1", Data: protoutil.NewAny(protoutil.NewStructMap(map[string]*structpb.Value{
|
||||
"k1": protoutil.NewStructString("v1"),
|
||||
}))},
|
||||
{Type: "test-1", Id: "r2", Data: protoutil.NewAny(protoutil.NewStructMap(map[string]*structpb.Value{
|
||||
"k2": protoutil.NewStructString("v2"),
|
||||
}))},
|
||||
})
|
||||
assert.NotEqual(t, 0, serverVersion)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("capacity", func(t *testing.T) {
|
||||
err := backend.SetOptions(ctx, "capacity-test", &databroker.Options{
|
||||
Capacity: proto.Uint64(3),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
_, err = backend.Put(ctx, []*databroker.Record{{
|
||||
Type: "capacity-test",
|
||||
Id: fmt.Sprint(i),
|
||||
Data: protoutil.NewAny(protoutil.NewStructMap(map[string]*structpb.Value{})),
|
||||
}})
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
_, _, stream, err := backend.SyncLatest(ctx, "capacity-test", nil)
|
||||
require.NoError(t, err)
|
||||
defer stream.Close()
|
||||
|
||||
records, err := storage.RecordStreamToList(stream)
|
||||
require.NoError(t, err)
|
||||
assert.Len(t, records, 3)
|
||||
|
||||
var ids []string
|
||||
for _, r := range records {
|
||||
ids = append(ids, r.GetId())
|
||||
}
|
||||
assert.Equal(t, []string{"7", "8", "9"}, ids, "should contain recent records")
|
||||
})
|
||||
|
||||
t.Run("lease", func(t *testing.T) {
|
||||
acquired, err := backend.Lease(ctx, "lease-test", "client-1", time.Second)
|
||||
assert.NoError(t, err)
|
||||
assert.True(t, acquired)
|
||||
|
||||
acquired, err = backend.Lease(ctx, "lease-test", "client-2", time.Second)
|
||||
assert.NoError(t, err)
|
||||
assert.False(t, acquired)
|
||||
})
|
||||
|
||||
t.Run("latest", func(t *testing.T) {
|
||||
for i := 0; i < 100; i++ {
|
||||
_, err := backend.Put(ctx, []*databroker.Record{{
|
||||
Type: "latest-test",
|
||||
Id: fmt.Sprint(i),
|
||||
Data: protoutil.NewAny(protoutil.NewStructMap(map[string]*structpb.Value{})),
|
||||
}})
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
_, _, stream, err := backend.SyncLatest(ctx, "latest-test", nil)
|
||||
require.NoError(t, err)
|
||||
defer stream.Close()
|
||||
|
||||
count := map[string]int{}
|
||||
|
||||
for stream.Next(true) {
|
||||
count[stream.Record().GetId()]++
|
||||
}
|
||||
assert.NoError(t, err)
|
||||
|
||||
for i := 0; i < 100; i++ {
|
||||
assert.Equal(t, 1, count[fmt.Sprint(i)])
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("changed", func(t *testing.T) {
|
||||
serverVersion, recordVersion, stream, err := backend.SyncLatest(ctx, "", nil)
|
||||
require.NoError(t, err)
|
||||
assert.NoError(t, stream.Close())
|
||||
|
||||
stream, err = backend.Sync(ctx, serverVersion, recordVersion)
|
||||
require.NoError(t, err)
|
||||
defer stream.Close()
|
||||
|
||||
go func() {
|
||||
for i := 0; i < 10; i++ {
|
||||
_, err := backend.Put(ctx, []*databroker.Record{{
|
||||
Type: "sync-test",
|
||||
Id: fmt.Sprint(i),
|
||||
Data: protoutil.NewAny(protoutil.NewStructMap(map[string]*structpb.Value{})),
|
||||
}})
|
||||
assert.NoError(t, err)
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
}
|
||||
}()
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
if assert.True(t, stream.Next(true)) {
|
||||
assert.Equal(t, fmt.Sprint(i), stream.Record().GetId())
|
||||
assert.Equal(t, "sync-test", stream.Record().GetType())
|
||||
} else {
|
||||
break
|
||||
}
|
||||
}
|
||||
assert.False(t, stream.Next(false))
|
||||
assert.NoError(t, stream.Err())
|
||||
})
|
||||
|
||||
return nil
|
||||
}))
|
||||
}
|
51
pkg/storage/postgres/filter.go
Normal file
51
pkg/storage/postgres/filter.go
Normal file
|
@ -0,0 +1,51 @@
|
|||
package postgres
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/pomerium/pomerium/pkg/storage"
|
||||
)
|
||||
|
||||
func addFilterExpressionToQuery(query *string, args *[]interface{}, expr storage.FilterExpression) error {
|
||||
compoundExpression := func(subexprs []storage.FilterExpression, op string) error {
|
||||
*query += "( "
|
||||
for i, subexpr := range subexprs {
|
||||
if i > 0 {
|
||||
*query += " " + op + " "
|
||||
}
|
||||
err := addFilterExpressionToQuery(query, args, subexpr)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
*query += " )"
|
||||
return nil
|
||||
}
|
||||
|
||||
switch expr := expr.(type) {
|
||||
case storage.AndFilterExpression:
|
||||
return compoundExpression(expr, "AND")
|
||||
case storage.OrFilterExpression:
|
||||
return compoundExpression(expr, "OR")
|
||||
case storage.EqualsFilterExpression:
|
||||
switch strings.Join(expr.Fields, ".") {
|
||||
case "type":
|
||||
*query += schemaName + "." + recordsTableName + ".type = " + fmt.Sprintf("$%d", len(*args)+1)
|
||||
*args = append(*args, expr.Value)
|
||||
return nil
|
||||
case "id":
|
||||
*query += schemaName + "." + recordsTableName + ".id = " + fmt.Sprintf("$%d", len(*args)+1)
|
||||
*args = append(*args, expr.Value)
|
||||
return nil
|
||||
case "$index":
|
||||
*query += schemaName + "." + recordsTableName + ".index_cidr >>= " + fmt.Sprintf("$%d", len(*args)+1)
|
||||
*args = append(*args, expr.Value)
|
||||
return nil
|
||||
default:
|
||||
return fmt.Errorf("unsupported equals filter: %v", expr.Fields)
|
||||
}
|
||||
default:
|
||||
return fmt.Errorf("unsupported filter expression: %T", expr)
|
||||
}
|
||||
}
|
32
pkg/storage/postgres/filter_test.go
Normal file
32
pkg/storage/postgres/filter_test.go
Normal file
|
@ -0,0 +1,32 @@
|
|||
package postgres
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/pomerium/pomerium/pkg/storage"
|
||||
)
|
||||
|
||||
func TestAddFilterExpressionToQuery(t *testing.T) {
|
||||
query := ""
|
||||
args := []any{}
|
||||
addFilterExpressionToQuery(&query, &args, storage.AndFilterExpression{
|
||||
storage.OrFilterExpression{
|
||||
storage.EqualsFilterExpression{
|
||||
Fields: []string{"id"},
|
||||
Value: "v1",
|
||||
},
|
||||
storage.EqualsFilterExpression{
|
||||
Fields: []string{"$index"},
|
||||
Value: "v2",
|
||||
},
|
||||
},
|
||||
storage.EqualsFilterExpression{
|
||||
Fields: []string{"type"},
|
||||
Value: "v3",
|
||||
},
|
||||
})
|
||||
assert.Equal(t, "( ( pomerium.records.id = $1 OR pomerium.records.index_cidr >>= $2 ) AND pomerium.records.type = $3 )", query)
|
||||
assert.Equal(t, []any{"v1", "v2", "v3"}, args)
|
||||
}
|
131
pkg/storage/postgres/migrate.go
Normal file
131
pkg/storage/postgres/migrate.go
Normal file
|
@ -0,0 +1,131 @@
|
|||
package postgres
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
|
||||
"github.com/jackc/pgx/v4"
|
||||
|
||||
"github.com/pomerium/pomerium/pkg/cryptutil"
|
||||
)
|
||||
|
||||
var migrations = []func(context.Context, pgx.Tx) error{
|
||||
1: func(ctx context.Context, tx pgx.Tx) error {
|
||||
_, err := tx.Exec(ctx, `
|
||||
CREATE TABLE `+schemaName+`.`+recordsTableName+` (
|
||||
type TEXT NOT NULL,
|
||||
id TEXT NOT NULL,
|
||||
version BIGINT NOT NULL,
|
||||
data JSONB NOT NULL,
|
||||
modified_at TIMESTAMPTZ NOT NULL DEFAULT(NOW()),
|
||||
|
||||
index_cidr INET NULL,
|
||||
|
||||
PRIMARY KEY (type, id)
|
||||
)
|
||||
`)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
_, err = tx.Exec(ctx, `
|
||||
CREATE INDEX ON `+schemaName+`.`+recordsTableName+`
|
||||
USING gist (index_cidr inet_ops);
|
||||
`)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
_, err = tx.Exec(ctx, `
|
||||
CREATE TABLE `+schemaName+`.`+recordChangesTableName+` (
|
||||
type TEXT NOT NULL,
|
||||
id TEXT NOT NULL,
|
||||
version BIGINT NOT NULL,
|
||||
data JSONB NOT NULL,
|
||||
modified_at TIMESTAMPTZ NOT NULL,
|
||||
deleted_at TIMESTAMPTZ NULL,
|
||||
|
||||
PRIMARY KEY (version)
|
||||
)
|
||||
`)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
_, err = tx.Exec(ctx, `
|
||||
CREATE TABLE `+schemaName+`.`+recordOptionsTableName+` (
|
||||
type TEXT NOT NULL,
|
||||
capacity BIGINT NULL,
|
||||
|
||||
PRIMARY KEY (type)
|
||||
)
|
||||
`)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
_, err = tx.Exec(ctx, `
|
||||
CREATE TABLE `+schemaName+`.`+leasesTableName+` (
|
||||
name TEXT NOT NULL,
|
||||
id TEXT NOT NULL,
|
||||
expires_at TIMESTAMPTZ NOT NULL,
|
||||
|
||||
PRIMARY KEY (name)
|
||||
)
|
||||
`)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
},
|
||||
}
|
||||
|
||||
func migrate(ctx context.Context, tx pgx.Tx) (serverVersion uint64, err error) {
|
||||
_, err = tx.Exec(ctx, `CREATE SCHEMA IF NOT EXISTS `+schemaName)
|
||||
if err != nil {
|
||||
return serverVersion, err
|
||||
}
|
||||
|
||||
_, err = tx.Exec(ctx, `
|
||||
CREATE TABLE IF NOT EXISTS `+schemaName+`.`+migrationInfoTableName+` (
|
||||
server_version BIGINT NOT NULL,
|
||||
migration_version SMALLINT NOT NULL
|
||||
)
|
||||
`)
|
||||
if err != nil {
|
||||
return serverVersion, err
|
||||
}
|
||||
|
||||
var migrationVersion uint64
|
||||
err = tx.QueryRow(ctx, `
|
||||
SELECT server_version, migration_version
|
||||
FROM `+schemaName+`.migration_info
|
||||
`).Scan(&serverVersion, &migrationVersion)
|
||||
if errors.Is(err, pgx.ErrNoRows) {
|
||||
serverVersion = uint64(cryptutil.NewRandomUInt32()) // we can't actually store a uint64, just an int64, so just generate a uint32
|
||||
_, err = tx.Exec(ctx, `
|
||||
INSERT INTO `+schemaName+`.`+migrationInfoTableName+` (server_version, migration_version)
|
||||
VALUES ($1, $2)
|
||||
`, serverVersion, 0)
|
||||
}
|
||||
if err != nil {
|
||||
return serverVersion, err
|
||||
}
|
||||
|
||||
for version := migrationVersion + 1; version < uint64(len(migrations)); version++ {
|
||||
err = migrations[version](ctx, tx)
|
||||
if err != nil {
|
||||
return serverVersion, err
|
||||
}
|
||||
_, err = tx.Exec(ctx, `
|
||||
UPDATE `+schemaName+`.`+migrationInfoTableName+`
|
||||
SET migration_version = $1
|
||||
`, version)
|
||||
if err != nil {
|
||||
return serverVersion, err
|
||||
}
|
||||
}
|
||||
|
||||
return serverVersion, nil
|
||||
}
|
30
pkg/storage/postgres/option.go
Normal file
30
pkg/storage/postgres/option.go
Normal file
|
@ -0,0 +1,30 @@
|
|||
package postgres
|
||||
|
||||
import (
|
||||
"time"
|
||||
)
|
||||
|
||||
const defaultExpiry = time.Hour * 24
|
||||
|
||||
type config struct {
|
||||
expiry time.Duration
|
||||
}
|
||||
|
||||
// Option customizes a Backend.
|
||||
type Option func(*config)
|
||||
|
||||
// WithExpiry sets the expiry for changes.
|
||||
func WithExpiry(expiry time.Duration) Option {
|
||||
return func(cfg *config) {
|
||||
cfg.expiry = expiry
|
||||
}
|
||||
}
|
||||
|
||||
func getConfig(options ...Option) *config {
|
||||
cfg := new(config)
|
||||
WithExpiry(defaultExpiry)(cfg)
|
||||
for _, o := range options {
|
||||
o(cfg)
|
||||
}
|
||||
return cfg
|
||||
}
|
323
pkg/storage/postgres/postgres.go
Normal file
323
pkg/storage/postgres/postgres.go
Normal file
|
@ -0,0 +1,323 @@
|
|||
// Package postgres contains an implementation of the storage.Backend backed by postgres.
|
||||
package postgres
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"time"
|
||||
|
||||
"github.com/jackc/pgconn"
|
||||
"github.com/jackc/pgtype"
|
||||
"github.com/jackc/pgx/v4"
|
||||
"google.golang.org/protobuf/encoding/protojson"
|
||||
"google.golang.org/protobuf/proto"
|
||||
"google.golang.org/protobuf/types/known/anypb"
|
||||
"google.golang.org/protobuf/types/known/timestamppb"
|
||||
|
||||
"github.com/pomerium/pomerium/pkg/grpc/databroker"
|
||||
"github.com/pomerium/pomerium/pkg/storage"
|
||||
)
|
||||
|
||||
var (
|
||||
schemaName = "pomerium"
|
||||
migrationInfoTableName = "migration_info"
|
||||
recordsTableName = "records"
|
||||
recordChangesTableName = "record_changes"
|
||||
recordChangeNotifyName = "pomerium_record_change"
|
||||
recordOptionsTableName = "record_options"
|
||||
leasesTableName = "leases"
|
||||
)
|
||||
|
||||
type querier interface {
|
||||
Exec(ctx context.Context, sql string, arguments ...interface{}) (commandTag pgconn.CommandTag, err error)
|
||||
Query(ctx context.Context, sql string, args ...interface{}) (pgx.Rows, error)
|
||||
QueryRow(ctx context.Context, sql string, args ...interface{}) pgx.Row
|
||||
}
|
||||
|
||||
func deleteChangesBefore(ctx context.Context, q querier, cutoff time.Time) error {
|
||||
_, err := q.Exec(ctx, `
|
||||
DELETE FROM `+schemaName+`.`+recordChangesTableName+`
|
||||
WHERE modified_at < $1
|
||||
`, cutoff)
|
||||
return err
|
||||
}
|
||||
|
||||
func dup(record *databroker.Record) *databroker.Record {
|
||||
return proto.Clone(record).(*databroker.Record)
|
||||
}
|
||||
|
||||
func enforceOptions(ctx context.Context, q querier, recordType string, options *databroker.Options) error {
|
||||
if options == nil || options.Capacity == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
_, err := q.Exec(ctx, `
|
||||
DELETE FROM `+schemaName+`.`+recordsTableName+`
|
||||
WHERE type=$1
|
||||
AND id NOT IN (
|
||||
SELECT id
|
||||
FROM `+schemaName+`.`+recordsTableName+`
|
||||
WHERE type=$1
|
||||
ORDER BY version DESC
|
||||
LIMIT $2
|
||||
)
|
||||
`, recordType, options.GetCapacity())
|
||||
return err
|
||||
}
|
||||
|
||||
func getLatestRecordVersion(ctx context.Context, q querier) (recordVersion uint64, err error) {
|
||||
err = q.QueryRow(ctx, `
|
||||
SELECT version
|
||||
FROM `+schemaName+`.`+recordChangesTableName+`
|
||||
ORDER BY version DESC
|
||||
LIMIT 1
|
||||
`).Scan(&recordVersion)
|
||||
if isNotFound(err) {
|
||||
err = nil
|
||||
}
|
||||
return recordVersion, err
|
||||
}
|
||||
|
||||
func getNextChangedRecord(ctx context.Context, q querier, afterRecordVersion uint64) (*databroker.Record, error) {
|
||||
var recordType, recordID string
|
||||
var version uint64
|
||||
var data pgtype.JSONB
|
||||
var modifiedAt pgtype.Timestamptz
|
||||
var deletedAt pgtype.Timestamptz
|
||||
err := q.QueryRow(ctx, `
|
||||
SELECT type, id, version, data, modified_at, deleted_at
|
||||
FROM `+schemaName+`.`+recordChangesTableName+`
|
||||
WHERE version > $1
|
||||
`, afterRecordVersion).Scan(&recordType, &recordID, &version, &data, &modifiedAt, &deletedAt)
|
||||
if isNotFound(err) {
|
||||
return nil, storage.ErrNotFound
|
||||
} else if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var any anypb.Any
|
||||
err = protojson.Unmarshal(data.Bytes, &any)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &databroker.Record{
|
||||
Version: version,
|
||||
Type: recordType,
|
||||
Id: recordID,
|
||||
Data: &any,
|
||||
ModifiedAt: timestamppbFromTimestamptz(modifiedAt),
|
||||
DeletedAt: timestamppbFromTimestamptz(deletedAt),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func getOptions(ctx context.Context, q querier, recordType string) (*databroker.Options, error) {
|
||||
var capacity pgtype.Int8
|
||||
err := q.QueryRow(ctx, `
|
||||
SELECT capacity
|
||||
FROM `+schemaName+`.`+recordOptionsTableName+`
|
||||
WHERE type=$1
|
||||
`, recordType).Scan(&capacity)
|
||||
if err != nil && !isNotFound(err) {
|
||||
return nil, err
|
||||
}
|
||||
options := new(databroker.Options)
|
||||
if capacity.Status == pgtype.Present {
|
||||
options.Capacity = proto.Uint64(uint64(capacity.Int))
|
||||
}
|
||||
return options, nil
|
||||
}
|
||||
|
||||
func getRecord(ctx context.Context, q querier, recordType, recordID string) (*databroker.Record, error) {
|
||||
var version uint64
|
||||
var data pgtype.JSONB
|
||||
var modifiedAt pgtype.Timestamptz
|
||||
err := q.QueryRow(ctx, `
|
||||
SELECT version, data, modified_at
|
||||
FROM `+schemaName+`.`+recordsTableName+`
|
||||
WHERE type=$1 AND id=$2
|
||||
`, recordType, recordID).Scan(&version, &data, &modifiedAt)
|
||||
if isNotFound(err) {
|
||||
return nil, storage.ErrNotFound
|
||||
} else if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var any anypb.Any
|
||||
err = protojson.Unmarshal(data.Bytes, &any)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &databroker.Record{
|
||||
Version: version,
|
||||
Type: recordType,
|
||||
Id: recordID,
|
||||
Data: &any,
|
||||
ModifiedAt: timestamppbFromTimestamptz(modifiedAt),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func listRecords(ctx context.Context, q querier, expr storage.FilterExpression, offset, limit int) ([]*databroker.Record, error) {
|
||||
args := []interface{}{offset, limit}
|
||||
query := `
|
||||
SELECT type, id, version, data, modified_at
|
||||
FROM ` + schemaName + `.` + recordsTableName + `
|
||||
`
|
||||
if expr != nil {
|
||||
query += "WHERE "
|
||||
err := addFilterExpressionToQuery(&query, &args, expr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
query += `
|
||||
ORDER BY type, id
|
||||
LIMIT $2
|
||||
OFFSET $1
|
||||
`
|
||||
rows, err := q.Query(ctx, query, args...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer rows.Close()
|
||||
|
||||
var records []*databroker.Record
|
||||
for rows.Next() {
|
||||
var recordType, id string
|
||||
var version uint64
|
||||
var data pgtype.JSONB
|
||||
var modifiedAt pgtype.Timestamptz
|
||||
err = rows.Scan(&recordType, &id, &version, &data, &modifiedAt)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var any anypb.Any
|
||||
err = protojson.Unmarshal(data.Bytes, &any)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
records = append(records, &databroker.Record{
|
||||
Version: version,
|
||||
Type: recordType,
|
||||
Id: id,
|
||||
Data: &any,
|
||||
ModifiedAt: timestamppbFromTimestamptz(modifiedAt),
|
||||
})
|
||||
}
|
||||
return records, rows.Err()
|
||||
}
|
||||
|
||||
func maybeAcquireLease(ctx context.Context, q querier, leaseName, leaseID string, ttl time.Duration) (leaseHolderID string, err error) {
|
||||
tbl := schemaName + "." + leasesTableName
|
||||
expiresAt := timestamptzFromTimestamppb(timestamppb.New(time.Now().Add(ttl)))
|
||||
now := timestamptzFromTimestamppb(timestamppb.Now())
|
||||
err = q.QueryRow(ctx, `
|
||||
INSERT INTO `+tbl+` (name, id, expires_at)
|
||||
VALUES ($1, $2, $3)
|
||||
ON CONFLICT (name) DO UPDATE
|
||||
SET id=CASE WHEN `+tbl+`.expires_at<$4 OR `+tbl+`.id=$2 THEN $2 ELSE `+tbl+`.id END,
|
||||
expires_at=CASE WHEN `+tbl+`.expires_at<$4 OR `+tbl+`.id=$2 THEN $3 ELSE `+tbl+`.expires_at END
|
||||
RETURNING `+tbl+`.id
|
||||
`, leaseName, leaseID, expiresAt, now).Scan(&leaseHolderID)
|
||||
return leaseHolderID, err
|
||||
}
|
||||
|
||||
func putRecordChange(ctx context.Context, q querier, record *databroker.Record) error {
|
||||
data, err := jsonbFromAny(record.GetData())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
modifiedAt := timestamptzFromTimestamppb(record.GetModifiedAt())
|
||||
deletedAt := timestamptzFromTimestamppb(record.GetDeletedAt())
|
||||
_, err = q.Exec(ctx, `
|
||||
INSERT INTO `+schemaName+`.`+recordChangesTableName+` (type, id, version, data, modified_at, deleted_at)
|
||||
VALUES ($1, $2, $3, $4, $5, $6)
|
||||
`, record.GetType(), record.GetId(), record.GetVersion(), data, modifiedAt, deletedAt)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func putRecord(ctx context.Context, q querier, record *databroker.Record) error {
|
||||
data, err := jsonbFromAny(record.GetData())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
modifiedAt := timestamptzFromTimestamppb(record.GetModifiedAt())
|
||||
if record.GetDeletedAt() == nil {
|
||||
_, err = q.Exec(ctx, `
|
||||
INSERT INTO `+schemaName+`.`+recordsTableName+` (type, id, version, data, modified_at)
|
||||
VALUES ($1, $2, $3, $4, $5)
|
||||
ON CONFLICT (type, id) DO UPDATE
|
||||
SET version=$3, data=$4, modified_at=$5
|
||||
`, record.GetType(), record.GetId(), record.GetVersion(), data, modifiedAt)
|
||||
} else {
|
||||
_, err = q.Exec(ctx, `
|
||||
DELETE FROM `+schemaName+`.`+recordsTableName+`
|
||||
WHERE type=$1 AND id=$2 AND version<$3
|
||||
`, record.GetType(), record.GetId(), record.GetVersion())
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func setOptions(ctx context.Context, q querier, recordType string, options *databroker.Options) error {
|
||||
capacity := pgtype.Int8{Status: pgtype.Null}
|
||||
if options != nil && options.Capacity != nil {
|
||||
capacity.Int = int64(options.GetCapacity())
|
||||
capacity.Status = pgtype.Present
|
||||
}
|
||||
|
||||
_, err := q.Exec(ctx, `
|
||||
INSERT INTO `+schemaName+`.`+recordOptionsTableName+` (type, capacity)
|
||||
VALUES ($1, $2)
|
||||
ON CONFLICT (type) DO UPDATE
|
||||
SET capacity=$2
|
||||
`, recordType, capacity)
|
||||
return err
|
||||
}
|
||||
|
||||
func signalRecordChange(ctx context.Context, q querier) error {
|
||||
_, err := q.Exec(ctx, `NOTIFY `+recordChangeNotifyName)
|
||||
return err
|
||||
}
|
||||
|
||||
func jsonbFromAny(any *anypb.Any) (pgtype.JSONB, error) {
|
||||
if any == nil {
|
||||
return pgtype.JSONB{Status: pgtype.Null}, nil
|
||||
}
|
||||
|
||||
bs, err := protojson.Marshal(any)
|
||||
if err != nil {
|
||||
return pgtype.JSONB{Status: pgtype.Null}, err
|
||||
}
|
||||
|
||||
return pgtype.JSONB{Bytes: bs, Status: pgtype.Present}, nil
|
||||
}
|
||||
|
||||
func timestamppbFromTimestamptz(ts pgtype.Timestamptz) *timestamppb.Timestamp {
|
||||
if ts.Status != pgtype.Present {
|
||||
return nil
|
||||
}
|
||||
return timestamppb.New(ts.Time)
|
||||
}
|
||||
|
||||
func timestamptzFromTimestamppb(ts *timestamppb.Timestamp) pgtype.Timestamptz {
|
||||
if !ts.IsValid() {
|
||||
return pgtype.Timestamptz{Status: pgtype.Null}
|
||||
}
|
||||
return pgtype.Timestamptz{Time: ts.AsTime(), Status: pgtype.Present}
|
||||
}
|
||||
|
||||
func isNotFound(err error) bool {
|
||||
return errors.Is(err, pgx.ErrNoRows) || errors.Is(err, storage.ErrNotFound)
|
||||
}
|
165
pkg/storage/postgres/stream.go
Normal file
165
pkg/storage/postgres/stream.go
Normal file
|
@ -0,0 +1,165 @@
|
|||
package postgres
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/jackc/pgx/v4/pgxpool"
|
||||
|
||||
"github.com/pomerium/pomerium/pkg/contextutil"
|
||||
"github.com/pomerium/pomerium/pkg/grpc/databroker"
|
||||
"github.com/pomerium/pomerium/pkg/storage"
|
||||
)
|
||||
|
||||
const recordBatchSize = 64
|
||||
|
||||
type recordStream struct {
|
||||
backend *Backend
|
||||
expr storage.FilterExpression
|
||||
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
offset int
|
||||
pending []*databroker.Record
|
||||
err error
|
||||
}
|
||||
|
||||
func newRecordStream(
|
||||
ctx context.Context,
|
||||
backend *Backend,
|
||||
expr storage.FilterExpression,
|
||||
) *recordStream {
|
||||
stream := &recordStream{
|
||||
backend: backend,
|
||||
expr: expr,
|
||||
}
|
||||
stream.ctx, stream.cancel = contextutil.Merge(ctx, backend.closeCtx)
|
||||
return stream
|
||||
}
|
||||
|
||||
func (stream *recordStream) Close() error {
|
||||
stream.cancel()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (stream *recordStream) Next(block bool) bool {
|
||||
if stream.err != nil {
|
||||
return false
|
||||
}
|
||||
|
||||
if len(stream.pending) > 1 {
|
||||
stream.pending = stream.pending[1:]
|
||||
return true
|
||||
}
|
||||
|
||||
var pool *pgxpool.Pool
|
||||
_, pool, stream.err = stream.backend.init(stream.ctx)
|
||||
if stream.err != nil {
|
||||
return false
|
||||
}
|
||||
|
||||
stream.pending, stream.err = listRecords(stream.ctx, pool, stream.expr, stream.offset, recordBatchSize)
|
||||
if stream.err != nil {
|
||||
return false
|
||||
}
|
||||
stream.offset += recordBatchSize
|
||||
|
||||
return len(stream.pending) > 0
|
||||
}
|
||||
|
||||
func (stream *recordStream) Record() *databroker.Record {
|
||||
if len(stream.pending) == 0 {
|
||||
return nil
|
||||
}
|
||||
return stream.pending[0]
|
||||
}
|
||||
|
||||
func (stream *recordStream) Err() error {
|
||||
return stream.err
|
||||
}
|
||||
|
||||
const watchPollInterval = 30 * time.Second
|
||||
|
||||
type changedRecordStream struct {
|
||||
backend *Backend
|
||||
recordVersion uint64
|
||||
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
record *databroker.Record
|
||||
err error
|
||||
ticker *time.Ticker
|
||||
changed chan context.Context
|
||||
}
|
||||
|
||||
func newChangedRecordStream(
|
||||
ctx context.Context,
|
||||
backend *Backend,
|
||||
recordVersion uint64,
|
||||
) storage.RecordStream {
|
||||
stream := &changedRecordStream{
|
||||
backend: backend,
|
||||
recordVersion: recordVersion,
|
||||
ticker: time.NewTicker(watchPollInterval),
|
||||
changed: backend.onChange.Bind(),
|
||||
}
|
||||
stream.ctx, stream.cancel = contextutil.Merge(ctx, backend.closeCtx)
|
||||
return stream
|
||||
}
|
||||
|
||||
func (stream *changedRecordStream) Close() error {
|
||||
stream.cancel()
|
||||
stream.ticker.Stop()
|
||||
stream.backend.onChange.Unbind(stream.changed)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (stream *changedRecordStream) Next(block bool) bool {
|
||||
for {
|
||||
if stream.err != nil {
|
||||
return false
|
||||
}
|
||||
|
||||
var pool *pgxpool.Pool
|
||||
_, pool, stream.err = stream.backend.init(stream.ctx)
|
||||
if stream.err != nil {
|
||||
return false
|
||||
}
|
||||
|
||||
stream.record, stream.err = getNextChangedRecord(
|
||||
stream.ctx,
|
||||
pool,
|
||||
stream.recordVersion,
|
||||
)
|
||||
if isNotFound(stream.err) {
|
||||
stream.err = nil
|
||||
} else if stream.err != nil {
|
||||
return false
|
||||
}
|
||||
|
||||
if stream.record != nil {
|
||||
stream.recordVersion = stream.record.GetVersion()
|
||||
return true
|
||||
}
|
||||
|
||||
if !block {
|
||||
return false
|
||||
}
|
||||
|
||||
select {
|
||||
case <-stream.ctx.Done():
|
||||
stream.err = stream.ctx.Err()
|
||||
return false
|
||||
case <-stream.ticker.C:
|
||||
case <-stream.changed:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (stream *changedRecordStream) Record() *databroker.Record {
|
||||
return stream.record
|
||||
}
|
||||
|
||||
func (stream *changedRecordStream) Err() error {
|
||||
return stream.err
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue