Skip to content

Commit

Permalink
sqlproxyccl: change denylist into an access control list
Browse files Browse the repository at this point in the history
To support an IP Allowlist in the sqlproxy, this change extends
the denylist code to make the Watcher support multiple AccessControllers.
Each AccessController is consulted before allowing a connection through,
and rechecked on any changes to the underlying files.

The sqlproxy will also fail to start if it begins with an invalid allow
or deny list, but if invalid files are written later then it increments
a new error metric so we can be alerted and take action to fix it.

Part of: https://cockroachlabs.atlassian.net/browse/CC-8136

Release note: None
  • Loading branch information
PJ Tatlow committed Mar 16, 2023
1 parent ea8e307 commit a20ff9b
Show file tree
Hide file tree
Showing 13 changed files with 791 additions and 307 deletions.
1 change: 0 additions & 1 deletion pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -816,7 +816,6 @@ GO_TARGETS = [
"//pkg/ccl/sqlitelogictestccl:sqlitelogictestccl_test",
"//pkg/ccl/sqlproxyccl/acl:acl",
"//pkg/ccl/sqlproxyccl/acl:acl_test",
"//pkg/ccl/sqlproxyccl/acl:denylist",
"//pkg/ccl/sqlproxyccl/balancer:balancer",
"//pkg/ccl/sqlproxyccl/balancer:balancer_test",
"//pkg/ccl/sqlproxyccl/interceptor:interceptor",
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/sqlproxyccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,7 @@ go_test(
"@com_github_jackc_pgx_v4//:pgx",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
"@in_gopkg_yaml_v3//:yaml_v3",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//status",
],
Expand Down
23 changes: 4 additions & 19 deletions pkg/ccl/sqlproxyccl/acl/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,28 +1,11 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "denylist",
srcs = [
"denylist.go",
"file.go",
"watcher.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl/denylist",
visibility = ["//visibility:public"],
deps = [
"//pkg/util/log",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_google_btree//:btree",
"@in_gopkg_yaml_v2//:yaml_v2",
],
)

go_library(
name = "acl",
srcs = [
"access_control.go",
"allowlist.go",
"denylist.go",
"file.go",
"watcher.go",
Expand All @@ -31,6 +14,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
Expand All @@ -49,6 +33,7 @@ go_test(
embed = [":acl"],
deps = [
"//pkg/util/leaktest",
"//pkg/util/metric",
"//pkg/util/timeutil",
"@com_github_google_btree//:btree",
"@com_github_stretchr_testify//require",
Expand Down
21 changes: 21 additions & 0 deletions pkg/ccl/sqlproxyccl/acl/access_control.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
// Copyright 2023 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package acl

import "github.com/cockroachdb/cockroach/pkg/util/timeutil"

// ConnectionTags contains connection properties to match against the denylist.
type ConnectionTags struct {
IP string
Cluster string
}

type AccessController interface {
CheckConnection(ConnectionTags, timeutil.TimeSource) error
}
85 changes: 85 additions & 0 deletions pkg/ccl/sqlproxyccl/acl/allowlist.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
// Copyright 2023 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package acl

import (
"net"

"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)

type AllowlistFile struct {
Seq int64 `yaml:"SequenceNumber"`
Allowlist map[string]AllowEntry `yaml:"allowlist"`
}

// Allowlist represents the current IP Allowlist,
// which maps cluster IDs to a list of allowed IP ranges.
type Allowlist struct {
entries map[string]AllowEntry
}

func (al *Allowlist) UnmarshalYAML(unmarshal func(interface{}) error) error {
var f AllowlistFile
if err := unmarshal(&f); err != nil {
return err
}
al.entries = f.Allowlist
return nil
}

func (al *Allowlist) CheckConnection(
connection ConnectionTags, timeSource timeutil.TimeSource,
) error {
entry, ok := al.entries[connection.Cluster]
if !ok {
// No allowlist entry, allow all traffic
return nil
}
ip := net.ParseIP(connection.IP)
if ip == nil {
return errors.Newf("could not parse ip address: '%s'", ip)
}
// Check all ips for this cluster.
// If one of them contains the current IP then it's allowed.
for _, allowedIP := range entry.ips {
if allowedIP.Contains(ip) {
return nil
}
}

return errors.Newf("connection ip '%s' denied: ip address not allowed", connection.IP)
}

type AllowEntry struct {
ips []*net.IPNet
}

// This custom unmarshal code converts each string IP address into a *net.IPNet.
// If it cannot be parsed, it is currently ignored and not added to the AllowEntry.
func (e *AllowEntry) UnmarshalYAML(unmarshal func(interface{}) error) error {
var raw struct {
IPs []string `yaml:"ips"`
}

if err := unmarshal(&raw); err != nil {
return err
}
e.ips = make([]*net.IPNet, 0)

for _, ip := range raw.IPs {
_, ipNet, _ := net.ParseCIDR(ip)
if ipNet != nil {
e.ips = append(e.ips, ipNet)
}
}

return nil
}
107 changes: 89 additions & 18 deletions pkg/ccl/sqlproxyccl/acl/denylist.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,17 +9,61 @@
package acl

import (
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)

// File represents a on-disk version of the denylist config.
// This also serves as a spec of expected yaml file format.
type DenylistFile struct {
Seq int64 `yaml:"SequenceNumber"`
Denylist []*DenyEntry `yaml:"denylist"`
}

// Denylist represents an in-memory cache for the current denylist.
// It also handles the logic of deciding what to be denied.
type Denylist struct {
timeSource timeutil.TimeSource
entries map[DenyEntity]*DenyEntry
entries map[DenyEntity]*DenyEntry
}

func (dl *Denylist) UnmarshalYAML(unmarshal func(interface{}) error) error {
var f DenylistFile
if err := unmarshal(&f); err != nil {
return err
}
dl.entries = make(map[DenyEntity]*DenyEntry)
for _, entry := range f.Denylist {
dl.entries[entry.Entity] = entry
}

return nil
}

func (dl *Denylist) CheckConnection(
connection ConnectionTags, timeSource timeutil.TimeSource,
) error {
ip := DenyEntity{Item: connection.IP, Type: IPAddrType}
if err := dl.denied(ip, timeSource); err != nil {
return errors.Wrapf(err, "connection ip '%v' denied", connection.IP)
}
cluster := DenyEntity{Item: connection.Cluster, Type: ClusterType}
if err := dl.denied(cluster, timeSource); err != nil {
return errors.Wrapf(err, "connection cluster '%v' denied", connection.Cluster)
}
return nil
}

// denied returns an error if the entity is denied access. The error message
// describes the reason for the denial.
func (dl *Denylist) denied(entity DenyEntity, timeSource timeutil.TimeSource) error {
if ent, ok := dl.entries[entity]; ok &&
(ent.Expiration.IsZero() || !ent.Expiration.Before(timeSource.Now())) {
return errors.Newf("%s", ent.Reason)
}
return nil
}

// DenyEntry records info about one denied entity,
Expand All @@ -34,32 +78,59 @@ type DenyEntry struct {
// DenyEntity represent one denied entity.
// This also serves as the spec for the config format.
type DenyEntity struct {
Item string `yaml:"item"`
Type Type `yaml:"type"`
Item string `yaml:"item"`
Type DenyType `yaml:"type"`
}

// Type is the type of the denied entity.
type Type int
// DenyType is the type of the denied entity.
type DenyType int

// Enum values for Type.
// Enum values for DenyType.
const (
IPAddrType Type = iota + 1
IPAddrType DenyType = iota + 1
ClusterType
UnknownType
)

// Denied returns an error if the entity is denied access. The error message
// describes the reason for the denial.
func (dl *Denylist) Denied(entity DenyEntity) error {
if ent, ok := dl.entries[entity]; ok &&
(ent.Expiration.IsZero() || !ent.Expiration.Before(dl.timeSource.Now())) {
return errors.Newf("%s", ent.Reason)
var strToTypeMap = map[string]DenyType{
"ip": IPAddrType,
"cluster": ClusterType,
}

var typeToStrMap = map[DenyType]string{
IPAddrType: "ip",
ClusterType: "cluster",
}

// UnmarshalYAML implements yaml.Unmarshaler interface for type.
func (typ *DenyType) UnmarshalYAML(unmarshal func(interface{}) error) error {
var raw string
err := unmarshal(&raw)
if err != nil {
return err
}

normalized := strings.ToLower(raw)
t, ok := strToTypeMap[normalized]
if !ok {
*typ = UnknownType
} else {
*typ = t
}

return nil
}

func emptyList() *Denylist {
return &Denylist{
timeSource: timeutil.DefaultTimeSource{},
entries: make(map[DenyEntity]*DenyEntry)}
// MarshalYAML implements yaml.Marshaler interface for type.
func (typ DenyType) MarshalYAML() (interface{}, error) {
return typ.String(), nil
}

// String implements Stringer interface for type.
func (typ DenyType) String() string {
s, ok := typeToStrMap[typ]
if !ok {
return "UNKNOWN"
}
return s
}
Loading

0 comments on commit a20ff9b

Please sign in to comment.