diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index bb056b89883f..ffb40cf3d35f 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -381,6 +381,7 @@ /pkg/ccl/jwtauthccl/ @cockroachdb/cloud-identity #!/pkg/ccl/kvccl/ @cockroachdb/kv-noreview /pkg/ccl/kvccl/kvtenantccl/ @cockroachdb/server-prs +/pkg/ccl/ldapccl/ @cockroachdb/prodsec #!/pkg/ccl/upgradeccl/ @cockroachdb/release-eng-prs @cockroachdb/upgrade-prs #!/pkg/ccl/logictestccl/ @cockroachdb/sql-queries-noreview #!/pkg/ccl/sqlitelogictestccl/ @cockroachdb/sql-queries-noreview diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 6d62151277d8..356ceb34838e 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -90,6 +90,9 @@ server.hsts.enabled boolean false if true, HSTS headers will be sent along with server.http.base_path string / path to redirect the user to upon succcessful login application server.identity_map.configuration string system-identity to database-username mappings application server.jwt_authentication.issuer_custom_ca string sets the custom root CA for verifying certificates while fetching JWKS from the JWT issuer application +server.ldap_authentication.client.tls_certificate string sets the client certificate for establishing mTLS connection with LDAP server application +server.ldap_authentication.client.tls_key string sets the client key for establishing mTLS connection with LDAP server application +server.ldap_authentication.domain.custom_ca string sets the custom root CA for verifying domain certificates when establishing connection with LDAP server application server.log_gc.max_deletions_per_cycle integer 1000 the maximum number of entries to delete on each purge of log-like system tables application server.log_gc.period duration 1h0m0s the period at which log-like system tables are checked for old entries application server.max_connections_per_gateway integer -1 the maximum number of SQL connections per gateway allowed at a given time (note: this will only limit future connection attempts and will not affect already established connections). Negative values result in unlimited number of connections. Superusers are not affected by this limit. application diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index fbd6e853d580..5232c0629d6a 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -118,6 +118,9 @@
server.http.base_path
string/path to redirect the user to upon succcessful loginServerless/Dedicated/Self-Hosted
server.identity_map.configuration
stringsystem-identity to database-username mappingsServerless/Dedicated/Self-Hosted
server.jwt_authentication.issuer_custom_ca
stringsets the custom root CA for verifying certificates while fetching JWKS from the JWT issuerServerless/Dedicated/Self-Hosted +
server.ldap_authentication.client.tls_certificate
stringsets the client certificate for establishing mTLS connection with LDAP serverServerless/Dedicated/Self-Hosted +
server.ldap_authentication.client.tls_key
stringsets the client key for establishing mTLS connection with LDAP serverServerless/Dedicated/Self-Hosted +
server.ldap_authentication.domain.custom_ca
stringsets the custom root CA for verifying domain certificates when establishing connection with LDAP serverServerless/Dedicated/Self-Hosted
server.log_gc.max_deletions_per_cycle
integer1000the maximum number of entries to delete on each purge of log-like system tablesServerless/Dedicated/Self-Hosted
server.log_gc.period
duration1h0m0sthe period at which log-like system tables are checked for old entriesServerless/Dedicated/Self-Hosted
server.max_connections_per_gateway
integer-1the maximum number of SQL connections per gateway allowed at a given time (note: this will only limit future connection attempts and will not affect already established connections). Negative values result in unlimited number of connections. Superusers are not affected by this limit.Serverless/Dedicated/Self-Hosted diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 2430998eab01..2de77809d654 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -51,6 +51,7 @@ ALL_TESTS = [ "//pkg/ccl/kvccl/kvfollowerreadsccl:kvfollowerreadsccl_test", "//pkg/ccl/kvccl/kvtenantccl/upgradeccl:upgradeccl_test", "//pkg/ccl/kvccl/kvtenantccl/upgradeinterlockccl:upgradeinterlockccl_test", + "//pkg/ccl/ldapccl:ldapccl_test", "//pkg/ccl/logictestccl/tests/3node-tenant-multiregion:3node-tenant-multiregion_test", "//pkg/ccl/logictestccl/tests/3node-tenant:3node-tenant_test", "//pkg/ccl/logictestccl/tests/5node:5node_test", @@ -898,6 +899,8 @@ GO_TARGETS = [ "//pkg/ccl/kvccl/kvtenantccl/upgradeinterlockccl:upgradeinterlockccl_test", "//pkg/ccl/kvccl/kvtenantccl:kvtenantccl", "//pkg/ccl/kvccl:kvccl", + "//pkg/ccl/ldapccl:ldapccl", + "//pkg/ccl/ldapccl:ldapccl_test", "//pkg/ccl/logictestccl/tests/3node-tenant-multiregion:3node-tenant-multiregion_test", "//pkg/ccl/logictestccl/tests/3node-tenant:3node-tenant_test", "//pkg/ccl/logictestccl/tests/5node:5node_test", diff --git a/pkg/ccl/BUILD.bazel b/pkg/ccl/BUILD.bazel index 66c894316e98..fab8ca42f0b7 100644 --- a/pkg/ccl/BUILD.bazel +++ b/pkg/ccl/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/ccl/jwtauthccl", "//pkg/ccl/kvccl", "//pkg/ccl/kvccl/kvtenantccl", + "//pkg/ccl/ldapccl", "//pkg/ccl/multiregionccl", "//pkg/ccl/multitenantccl", "//pkg/ccl/oidcccl", diff --git a/pkg/ccl/ccl_init.go b/pkg/ccl/ccl_init.go index 2f7e2fb5cc65..0bba874ba2e0 100644 --- a/pkg/ccl/ccl_init.go +++ b/pkg/ccl/ccl_init.go @@ -25,6 +25,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/ccl/jwtauthccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" + _ "github.com/cockroachdb/cockroach/pkg/ccl/ldapccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/multitenantccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/oidcccl" diff --git a/pkg/ccl/ldapccl/BUILD.bazel b/pkg/ccl/ldapccl/BUILD.bazel new file mode 100644 index 000000000000..cd31e7d4bcd1 --- /dev/null +++ b/pkg/ccl/ldapccl/BUILD.bazel @@ -0,0 +1,62 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "ldapccl", + srcs = [ + "authentication_ldap.go", + "ldap_util.go", + "settings.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/ccl/ldapccl", + visibility = ["//visibility:public"], + deps = [ + "//pkg/ccl/utilccl", + "//pkg/clusterversion", + "//pkg/security", + "//pkg/security/username", + "//pkg/server/telemetry", + "//pkg/settings", + "//pkg/settings/cluster", + "//pkg/sql/pgwire", + "//pkg/sql/pgwire/hba", + "//pkg/sql/pgwire/identmap", + "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", + "//pkg/util/log", + "//pkg/util/syncutil", + "//pkg/util/uuid", + "@com_github_cockroachdb_errors//:errors", + "@com_github_go_ldap_ldap_v3//:ldap", + ], +) + +go_test( + name = "ldapccl_test", + size = "small", + srcs = [ + "authentication_ldap_test.go", + "main_test.go", + "settings_test.go", + ], + data = glob(["testdata/**"]), + embed = [":ldapccl"], + deps = [ + "//pkg/base", + "//pkg/ccl", + "//pkg/security/certnames", + "//pkg/security/securityassets", + "//pkg/security/securitytest", + "//pkg/security/username", + "//pkg/server", + "//pkg/sql/pgwire/hba", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/randutil", + "@com_github_cockroachdb_errors//:errors", + "@com_github_go_ldap_ldap_v3//:ldap", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/ccl/ldapccl/authentication_ldap.go b/pkg/ccl/ldapccl/authentication_ldap.go new file mode 100644 index 000000000000..b7db2f8c3f54 --- /dev/null +++ b/pkg/ccl/ldapccl/authentication_ldap.go @@ -0,0 +1,276 @@ +// Copyright 2024 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 ldapccl + +import ( + "context" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/identmap" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" +) + +const ( + counterPrefix = "auth.ldap." + beginAuthCounterName = counterPrefix + "begin_auth" + loginSuccessCounterName = counterPrefix + "login_success" + enableCounterName = counterPrefix + "enable" +) + +var ( + beginAuthUseCounter = telemetry.GetCounterOnce(beginAuthCounterName) + loginSuccessUseCounter = telemetry.GetCounterOnce(loginSuccessCounterName) + enableUseCounter = telemetry.GetCounterOnce(enableCounterName) +) + +// ldapAuthenticator is an object that is used to enable ldap connection +// validation that are used as part of the CRDB client auth flow. +// +// The implementation uses the `go-ldap/ldap/` client package and is supported +// through a number of cluster settings defined in `ldapccl/settings.go`. These +// settings specify how the ldap auth attempt should be executed and if this +// feature is enabled. +type ldapAuthenticator struct { + mu struct { + syncutil.RWMutex + // conf contains all the values that come from cluster settings. + conf ldapAuthenticatorConf + // util contains connection object required for interfacing with ldap server. + util ILDAPUtil + // enabled represents the present state of if this feature is enabled. It + // is set to true once ldap util is initialized. + enabled bool + } + // clusterUUID is used to check the validity of the enterprise license. It is + // set once at initialization. + clusterUUID uuid.UUID +} + +// ldapAuthenticatorConf contains all the values to configure LDAP +// authentication. These values are copied from the matching cluster settings or +// from hba conf options for LDAP entry. +type ldapAuthenticatorConf struct { + domainCACert string + clientTLSCert string + clientTLSKey string + ldapServer string + ldapPort string + ldapBaseDN string + ldapBindDN string + ldapBindPassword string + ldapSearchFilter string + ldapSearchAttribute string +} + +// reloadConfig locks mutex and then refreshes the values in conf from the cluster settings. +func (authenticator *ldapAuthenticator) reloadConfig(ctx context.Context, st *cluster.Settings) { + authenticator.mu.Lock() + defer authenticator.mu.Unlock() + authenticator.reloadConfigLocked(ctx, st) +} + +// reloadConfig refreshes the values in conf from the cluster settings without locking the mutex. +func (authenticator *ldapAuthenticator) reloadConfigLocked( + ctx context.Context, st *cluster.Settings, +) { + conf := ldapAuthenticatorConf{ + domainCACert: LDAPDomainCACertificate.Get(&st.SV), + clientTLSCert: LDAPClientTLSCertSetting.Get(&st.SV), + clientTLSKey: LDAPClientTLSKeySetting.Get(&st.SV), + } + authenticator.mu.conf = conf + + var err error + authenticator.mu.util, err = NewLDAPUtil(ctx, authenticator.mu.conf) + if err != nil { + log.Warningf(ctx, "LDAP authentication: unable to initialize LDAP connection: %v", err) + return + } + + if !authenticator.mu.enabled { + telemetry.Inc(enableUseCounter) + } + authenticator.mu.enabled = true + log.Infof(ctx, "initialized LDAP authenticator") +} + +// setLDAPConfigOptions extracts hba conf parameters required for connecting and +// querying LDAP server from hba conf entry and sets them for LDAP authenticator. +func (authenticator *ldapAuthenticator) setLDAPConfigOptions(entry *hba.Entry) error { + conf := ldapAuthenticatorConf{ + domainCACert: authenticator.mu.conf.domainCACert, + } + for _, opt := range entry.Options { + switch opt[0] { + case "ldapserver": + conf.ldapServer = opt[1] + case "ldapport": + conf.ldapPort = opt[1] + case "ldapbasedn": + conf.ldapBaseDN = opt[1] + case "ldapbinddn": + conf.ldapBindDN = opt[1] + case "ldapbindpasswd": + conf.ldapBindPassword = opt[1] + case "ldapsearchfilter": + conf.ldapSearchFilter = opt[1] + case "ldapsearchattribute": + conf.ldapSearchAttribute = opt[1] + default: + return errors.Newf("invalid LDAP option provided in hba conf: %s", opt[0]) + } + } + authenticator.mu.conf = conf + return nil +} + +// validateLDAPOptions checks the ldap authenticator config values for validity. +func (authenticator *ldapAuthenticator) validateLDAPOptions() error { + const ldapOptionsErrorMsg = "ldap params in HBA conf missing" + if authenticator.mu.conf.ldapServer == "" { + return errors.New(ldapOptionsErrorMsg + " ldap server") + } + if authenticator.mu.conf.ldapPort == "" { + return errors.New(ldapOptionsErrorMsg + " ldap port") + } + if authenticator.mu.conf.ldapBaseDN == "" { + return errors.New(ldapOptionsErrorMsg + " base DN") + } + if authenticator.mu.conf.ldapBindDN == "" { + return errors.New(ldapOptionsErrorMsg + " bind DN") + } + if authenticator.mu.conf.ldapBindPassword == "" { + return errors.New(ldapOptionsErrorMsg + " bind password") + } + if authenticator.mu.conf.ldapSearchFilter == "" { + return errors.New(ldapOptionsErrorMsg + " search filter") + } + if authenticator.mu.conf.ldapSearchAttribute == "" { + return errors.New(ldapOptionsErrorMsg + " search attribute") + } + return nil +} + +// ValidateLDAPLogin validates an attempt to bind to an LDAP server. +// In particular, it checks that: +// * The cluster has an enterprise license. +// * The active cluster version is 24.2 for this feature. +// * LDAP authentication is enabled after settings were reloaded. +// * The auth attempt is not for a reserved user. +// * The hba conf entry options could be parsed to obtain ldap server params. +// * All ldap server params are valid. +// * LDAPs connection can be established with configured server. +// * Configured bind DN and password can be used to search for the sql user DN on ldap server. +// * The obtained user DN could be used to bind with the password from sql connection string. +// It returns authError (which is the error sql clients will see in case of +// failures) and detailedError (which is the internal error from ldap clients +// that might contain sensitive information we do not want to send to sql +// clients but still want to log it). We do not want to send any information +// back to client which was not provided by the client. +func (authenticator *ldapAuthenticator) ValidateLDAPLogin( + ctx context.Context, + st *cluster.Settings, + user username.SQLUsername, + ldapPwd string, + entry *hba.Entry, + _ *identmap.Conf, +) (detailedErrorMsg string, authError error) { + if err := utilccl.CheckEnterpriseEnabled(st, "LDAP authentication"); err != nil { + return "", err + } + if !st.Version.IsActive(ctx, clusterversion.V24_2) { + return "", pgerror.Newf(pgcode.FeatureNotSupported, "LDAP authentication is only supported after v24.2 upgrade is finalized") + } + + authenticator.mu.Lock() + defer authenticator.mu.Unlock() + + if !authenticator.mu.enabled { + return "", errors.Newf("LDAP authentication: not enabled") + } + telemetry.Inc(beginAuthUseCounter) + + if user.IsRootUser() || user.IsReserved() { + return "", errors.WithDetailf( + errors.Newf("LDAP authentication: invalid identity"), + "cannot use LDAP auth to login to a reserved user %s", user.Normalized()) + } + + if err := authenticator.setLDAPConfigOptions(entry); err != nil { + return fmt.Sprintf("error when fetching hba conf options for LDAP: %v", err), + errors.Newf("LDAP authentication: unable to fetch hba conf options") + } + + if err := authenticator.validateLDAPOptions(); err != nil { + return fmt.Sprintf("error validation authenticator options for LDAP: %v", err), + errors.Newf("LDAP authentication: unable to validate authenticator options") + } + + // Establish a LDAPs connection with the set LDAP server and port + err := authenticator.mu.util.InitLDAPsConn(ctx, authenticator.mu.conf) + if err != nil { + return fmt.Sprintf("error when trying to create LDAP connection: %v", err), + errors.Newf("LDAP authentication: unable to establish LDAP connection") + } + + // Fetch the ldap server Distinguished Name using sql username as search value + // for ldap search attribute + userDN, err := authenticator.mu.util.Search(ctx, authenticator.mu.conf, user.Normalized()) + if err != nil { + return fmt.Sprintf("error when searching for user in LDAP server: %v", err), + errors.WithDetailf( + errors.Newf("LDAP authentication: unable to find LDAP user distinguished name"), + "cannot find provided user %s on LDAP server", user.Normalized()) + } + + // Bind as the user to verify their password + err = authenticator.mu.util.Bind(ctx, userDN, ldapPwd) + if err != nil { + return fmt.Sprintf("error when binding as user %s with DN(%s) in LDAP server: %v", user.Normalized(), userDN, err), + errors.WithDetailf( + errors.Newf("LDAP authentication: unable to bind as LDAP user"), + "credentials invalid for LDAP server user %s", user.Normalized()) + } + + telemetry.Inc(loginSuccessUseCounter) + return "", nil +} + +// ConfigureLDAPAuth initializes and returns a ldapAuthenticator. It also sets up listeners so +// that the ldapAuthenticator's config is updated when the cluster settings values change. +var ConfigureLDAPAuth = func( + serverCtx context.Context, + ambientCtx log.AmbientContext, + st *cluster.Settings, + clusterUUID uuid.UUID, +) pgwire.LDAPVerifier { + authenticator := ldapAuthenticator{} + authenticator.clusterUUID = clusterUUID + authenticator.reloadConfig(serverCtx, st) + LDAPDomainCACertificate.SetOnChange(&st.SV, func(ctx context.Context) { + authenticator.reloadConfig(ambientCtx.AnnotateCtx(ctx), st) + }) + return &authenticator +} + +func init() { + pgwire.ConfigureLDAPAuth = ConfigureLDAPAuth +} diff --git a/pkg/ccl/ldapccl/authentication_ldap_test.go b/pkg/ccl/ldapccl/authentication_ldap_test.go new file mode 100644 index 000000000000..0b81a47e2d3d --- /dev/null +++ b/pkg/ccl/ldapccl/authentication_ldap_test.go @@ -0,0 +1,261 @@ +// Copyright 2024 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 ldapccl + +import ( + "context" + "crypto/tls" + "fmt" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" + "github.com/go-ldap/ldap/v3" + "github.com/stretchr/testify/require" +) + +const ( + emptyParam = "empty" + invalidParam = "invalid" +) + +type mockLDAPUtil struct { + conn *ldap.Conn + tlsConfig *tls.Config +} + +// InitLDAPsConn implements the ILDAPUtil interface. +func (lu *mockLDAPUtil) InitLDAPsConn(ctx context.Context, conf ldapAuthenticatorConf) error { + if strings.Contains(conf.ldapServer, invalidParam) { + return errors.Newf(ldapsFailureMessage + ": invalid ldap server provided") + } else if strings.Contains(conf.ldapPort, invalidParam) { + return errors.Newf(ldapsFailureMessage + ": invalid ldap port provided") + } + lu.conn = &ldap.Conn{} + return nil +} + +// Bind implements the ILDAPUtil interface. +func (lu *mockLDAPUtil) Bind(ctx context.Context, userDN string, ldapPwd string) error { + if strings.Contains(userDN, invalidParam) { + return errors.Newf(bindFailureMessage + ": invalid username provided") + } else if strings.Contains(ldapPwd, invalidParam) { + return errors.Newf(bindFailureMessage + ": invalid password provided") + } + + return nil +} + +// Search implements the ILDAPUtil interface. +func (lu *mockLDAPUtil) Search( + ctx context.Context, conf ldapAuthenticatorConf, username string, +) (userDN string, err error) { + if err := lu.Bind(ctx, conf.ldapBindDN, conf.ldapBindPassword); err != nil { + return "", errors.Wrap(err, searchFailureMessage) + } + if strings.Contains(conf.ldapBaseDN, invalidParam) { + return "", errors.Newf(searchFailureMessage+": invalid base DN %q provided", conf.ldapBaseDN) + } + if strings.Contains(conf.ldapSearchFilter, invalidParam) { + return "", errors.Newf(searchFailureMessage+": invalid search filter %q provided", conf.ldapSearchFilter) + } + if strings.Contains(conf.ldapSearchAttribute, invalidParam) { + return "", errors.Newf(searchFailureMessage+": invalid search attribute %q provided", conf.ldapSearchAttribute) + } + if strings.Contains(username, invalidParam) { + return "", errors.Newf(searchFailureMessage+": invalid search value %q provided", username) + } + distinguishedNames := strings.Split(username, ",") + switch { + case len(username) == 0: + return "", errors.Newf(searchFailureMessage+": user %q does not exist", username) + case len(distinguishedNames) > 1: + return "", errors.Newf(searchFailureMessage+": too many matching entries returned for user %q", username) + } + return distinguishedNames[0], nil +} + +var _ ILDAPUtil = &mockLDAPUtil{} + +func constructHBAEntry( + t *testing.T, + hbaEntryBase string, + hbaConfLDAPDefaultOpts map[string]string, + hbaConfLDAPOpts map[string]string, +) hba.Entry { + hbaEntryLDAP := hbaEntryBase + // add options from default and override default options when provided with one + for opt, value := range hbaConfLDAPDefaultOpts { + setValue := value + if hbaConfLDAPOpts[opt] == emptyParam { + continue + } else if hbaConfLDAPOpts[opt] != "" { + setValue = hbaConfLDAPOpts[opt] + } + hbaEntryLDAP += fmt.Sprintf("\"%s=%s\" ", opt, setValue) + } + // add non default options + for additionalOpt, additionalOptValue := range hbaConfLDAPOpts { + if _, ok := hbaConfLDAPDefaultOpts[additionalOpt]; !ok { + hbaEntryLDAP += fmt.Sprintf("\"%s=%s\" ", additionalOpt, additionalOptValue) + } + } + hbaConf, err := hba.ParseAndNormalize(hbaEntryLDAP) + if err != nil { + t.Fatalf("error parsing hba conf: %v", err) + } + if len(hbaConf.Entries) != 1 { + t.Fatalf("hba conf value invalid: should contain only 1 entry") + } + return hbaConf.Entries[0] +} + +func TestLDAPAuthentication(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + // Intercept the call to NewLDAPUtil and return the mocked NewLDAPUtil function + defer testutils.TestingHook( + &NewLDAPUtil, + func(ctx context.Context, conf ldapAuthenticatorConf) (ILDAPUtil, error) { + return &mockLDAPUtil{tlsConfig: &tls.Config{}}, nil + })() + ctx := context.Background() + s := serverutils.StartServerOnly(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + verifier := ConfigureLDAPAuth(ctx, s.AmbientCtx(), s.ClusterSettings(), s.StorageClusterID()) + hbaEntryBase := "host all all all ldap " + hbaConfLDAPDefaultOpts := map[string]string{ + "ldapserver": "localhost", "ldapport": "636", "ldapbasedn": "dc=localhost", "ldapbinddn": "cn=readonly,dc=localhost", + "ldapbindpasswd": "readonly_pwd", "ldapsearchattribute": "uid", "ldapsearchfilter": "(memberOf=cn=users,ou=groups,dc=localhost)", + } + testCases := []struct { + testName string + hbaConfLDAPOpts map[string]string + user string + pwd string + ldapAuthSuccess bool + expectedErr string + expectedErrDetails string + expectedDetailedErrMsg string + }{ + {testName: "proper hba conf and valid user cred", + user: "foo", pwd: "bar", ldapAuthSuccess: true}, + {testName: "proper hba conf and root user cred", + user: "root", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: invalid identity", + expectedErrDetails: "cannot use LDAP auth to login to a reserved user root"}, + {testName: "proper hba conf and node user cred", + user: "node", pwd: "bar", ldapAuthSuccess: false, expectedErr: "LDAP authentication: invalid identity", + expectedErrDetails: "cannot use LDAP auth to login to a reserved user node"}, + {testName: "invalid ldap option", + hbaConfLDAPOpts: map[string]string{"invalidOpt": "invalidVal"}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to fetch hba conf options", + expectedDetailedErrMsg: "error when fetching hba conf options for LDAP: invalid LDAP option provided in hba conf: invalidOpt"}, + {testName: "empty server", + hbaConfLDAPOpts: map[string]string{"ldapserver": emptyParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to validate authenticator options", + expectedDetailedErrMsg: "error validation authenticator options for LDAP: ldap params in HBA conf missing ldap server"}, + {testName: "invalid server", + hbaConfLDAPOpts: map[string]string{"ldapserver": invalidParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to establish LDAP connection", + expectedDetailedErrMsg: "error when trying to create LDAP connection: LDAPs connection failed: invalid ldap server provided"}, + {testName: "empty port", + hbaConfLDAPOpts: map[string]string{"ldapport": emptyParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to validate authenticator options", + expectedDetailedErrMsg: "error validation authenticator options for LDAP: ldap params in HBA conf missing ldap port"}, + {testName: "invalid port", + hbaConfLDAPOpts: map[string]string{"ldapport": invalidParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to establish LDAP connection", + expectedDetailedErrMsg: "error when trying to create LDAP connection: LDAPs connection failed: invalid ldap port provided"}, + {testName: "empty base dn", + hbaConfLDAPOpts: map[string]string{"ldapbasedn": emptyParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to validate authenticator options", + expectedDetailedErrMsg: "error validation authenticator options for LDAP: ldap params in HBA conf missing base DN"}, + {testName: "invalid base dn", + hbaConfLDAPOpts: map[string]string{"ldapbasedn": invalidParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to find LDAP user distinguished name", + expectedErrDetails: "cannot find provided user foo on LDAP server", + expectedDetailedErrMsg: "error when searching for user in LDAP server: LDAP search failed: invalid base DN \"invalid\" provided"}, + {testName: "empty bind dn", + hbaConfLDAPOpts: map[string]string{"ldapbinddn": emptyParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to validate authenticator options", + expectedDetailedErrMsg: "error validation authenticator options for LDAP: ldap params in HBA conf missing bind DN"}, + {testName: "invalid bind dn", + hbaConfLDAPOpts: map[string]string{"ldapbinddn": invalidParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to find LDAP user distinguished name", + expectedErrDetails: "cannot find provided user foo on LDAP server", + expectedDetailedErrMsg: "error when searching for user in LDAP server: LDAP search failed: LDAP bind failed: invalid username provided"}, + {testName: "empty bind pwd", + hbaConfLDAPOpts: map[string]string{"ldapbindpasswd": emptyParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to validate authenticator options", + expectedDetailedErrMsg: "error validation authenticator options for LDAP: ldap params in HBA conf missing bind password"}, + {testName: "invalid bind pwd", + hbaConfLDAPOpts: map[string]string{"ldapbindpasswd": invalidParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to find LDAP user distinguished name", + expectedErrDetails: "cannot find provided user foo on LDAP server", + expectedDetailedErrMsg: "error when searching for user in LDAP server: LDAP search failed: LDAP bind failed: invalid password provided"}, + {testName: "empty search attribute", + hbaConfLDAPOpts: map[string]string{"ldapsearchattribute": emptyParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to validate authenticator options", + expectedDetailedErrMsg: "error validation authenticator options for LDAP: ldap params in HBA conf missing search attribute"}, + {testName: "invalid search attribute", + hbaConfLDAPOpts: map[string]string{"ldapsearchattribute": invalidParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to find LDAP user distinguished name", + expectedErrDetails: "cannot find provided user foo on LDAP server", + expectedDetailedErrMsg: "error when searching for user in LDAP server: LDAP search failed: invalid search attribute \"invalid\" provided"}, + {testName: "empty search filter", + hbaConfLDAPOpts: map[string]string{"ldapsearchfilter": emptyParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to validate authenticator options", + expectedDetailedErrMsg: "error validation authenticator options for LDAP: ldap params in HBA conf missing search filter"}, + {testName: "invalid search filter", + hbaConfLDAPOpts: map[string]string{"ldapsearchfilter": invalidParam}, user: "foo", pwd: "bar", ldapAuthSuccess: false, + expectedErr: "LDAP authentication: unable to find LDAP user distinguished name", + expectedErrDetails: "cannot find provided user foo on LDAP server", + expectedDetailedErrMsg: "error when searching for user in LDAP server: LDAP search failed: invalid search filter \"invalid\" provided"}, + {testName: "invalid ldap user", + user: invalidParam, pwd: "bar", ldapAuthSuccess: false, expectedErr: "LDAP authentication: unable to find LDAP user distinguished name", + expectedErrDetails: "cannot find provided user invalid on LDAP server", + expectedDetailedErrMsg: "error when searching for user in LDAP server: LDAP search failed: invalid search value \"invalid\" provided"}, + {testName: "no such ldap user", + user: "", pwd: "bar", ldapAuthSuccess: false, expectedErr: "LDAP authentication: unable to find LDAP user distinguished name", + expectedErrDetails: "cannot find provided user on LDAP server", + expectedDetailedErrMsg: "error when searching for user in LDAP server: LDAP search failed: user \"\" does not exist"}, + {testName: "too many matching ldap users", + user: "foo,foo2,foo3", pwd: "bar", ldapAuthSuccess: false, expectedErr: "LDAP authentication: unable to find LDAP user distinguished name", + expectedErrDetails: "cannot find provided user foo,foo2,foo3 on LDAP server", + expectedDetailedErrMsg: "error when searching for user in LDAP server: LDAP search failed: too many matching entries returned for user \"foo,foo2,foo3\""}, + {testName: "invalid ldap password", + user: "foo", pwd: invalidParam, ldapAuthSuccess: false, expectedErr: "LDAP authentication: unable to bind as LDAP user", + expectedErrDetails: "credentials invalid for LDAP server user foo", + expectedDetailedErrMsg: "error when binding as user foo with DN(foo) in LDAP server: LDAP bind failed: invalid password provided"}, + } + for i, tc := range testCases { + t.Run(fmt.Sprintf("%d: testName:%v hbConfOpts:%v user:%v password:%v", i, tc.testName, tc.hbaConfLDAPOpts, tc.user, tc.pwd), func(t *testing.T) { + hbaEntry := constructHBAEntry(t, hbaEntryBase, hbaConfLDAPDefaultOpts, tc.hbaConfLDAPOpts) + detailedErrorMsg, err := verifier.ValidateLDAPLogin( + ctx, s.ClusterSettings(), username.MakeSQLUsernameFromPreNormalizedString(tc.user), tc.pwd, &hbaEntry, nil) + + if (err == nil) != tc.ldapAuthSuccess { + t.Fatalf("expected success=%t, got err=%v", tc.ldapAuthSuccess, err) + } + if err != nil { + require.Equal(t, tc.expectedErr, err.Error()) + require.Equal(t, tc.expectedErrDetails, errors.FlattenDetails(err)) + require.Equal(t, tc.expectedDetailedErrMsg, detailedErrorMsg) + } + }) + } +} diff --git a/pkg/ccl/ldapccl/ldap_util.go b/pkg/ccl/ldapccl/ldap_util.go new file mode 100644 index 000000000000..519004946020 --- /dev/null +++ b/pkg/ccl/ldapccl/ldap_util.go @@ -0,0 +1,142 @@ +// Copyright 2024 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 ldapccl + +import ( + "context" + "crypto/tls" + "crypto/x509" + "fmt" + + "github.com/cockroachdb/errors" + "github.com/go-ldap/ldap/v3" +) + +const ( + invalidLDAPConfMessage = "LDAP configuration invalid" + ldapsFailureMessage = "LDAPs connection failed" + bindFailureMessage = "LDAP bind failed" + searchFailureMessage = "LDAP search failed" +) + +type ldapUtil struct { + conn *ldap.Conn + tlsConfig *tls.Config +} + +// InitLDAPsConn implements the ILDAPUtil interface. +func (lu *ldapUtil) InitLDAPsConn(ctx context.Context, conf ldapAuthenticatorConf) (err error) { + // TODO(souravcrl): (Fix 1) DialTLS is slow if we do it for every authN + // attempt. We should look into ways for caching connections and avoiding + // connection timeouts in case LDAP server enforces that for idle connections. + // We still should be able to validate a large number of authN requests + // reusing the same connection(s). + // (Fix 2) Every authN attempt acquires a lock on ldapAuthenticator, so + // only 1 authN attempt is possible at a given time(for entire flow of + // bind+search+bind). We should have a permanent bind connection to search for + // entries and short-lived bind attempts for requested sql authNs. + // (Fix 3) Every CRDB node currently looks into establishing a separate + // connection with LDAP servers significantly increasing total number of open + // connections. This should be capped and configurable as to how many + // connections crdb nodes can take up(either in total or on a per node basis) + // + // ldapAddress := "ldap://ldap.example.com:636" + ldapAddress := conf.ldapServer + ":" + conf.ldapPort + if lu.conn, err = ldap.DialTLS("tcp", ldapAddress, lu.tlsConfig); err != nil { + return errors.Wrap(err, ldapsFailureMessage) + } + return nil +} + +// Bind implements the ILDAPUtil interface. +func (lu *ldapUtil) Bind(ctx context.Context, userDN string, ldapPwd string) (err error) { + if err = lu.conn.Bind(userDN, ldapPwd); err != nil { + return errors.Wrap(err, bindFailureMessage) + } + return nil +} + +// Search implements the ILDAPUtil interface. +func (lu *ldapUtil) Search( + ctx context.Context, conf ldapAuthenticatorConf, username string, +) (userDN string, err error) { + if err := lu.Bind(ctx, conf.ldapBindDN, conf.ldapBindPassword); err != nil { + return "", errors.Wrap(err, searchFailureMessage) + } + // TODO(souravcrl): Currently search could be performed at subtree level but + // this should be configurable through HBA conf using any of the scopes + // provided: https://github.com/go-ldap/ldap/blob/master/search.go#L17-L24 + searchRequest := ldap.NewSearchRequest( + conf.ldapBaseDN, + ldap.ScopeWholeSubtree, ldap.NeverDerefAliases, 0, 0, false, + fmt.Sprintf("(&%s(%s=%s))", conf.ldapSearchFilter, conf.ldapSearchAttribute, ldap.EscapeFilter(username)), + []string{}, + nil, + ) + sr, err := lu.conn.Search(searchRequest) + if err != nil { + return "", errors.Wrap(err, searchFailureMessage) + } + + switch { + case len(sr.Entries) == 0: + return "", errors.Newf(searchFailureMessage+": user %s does not exist", username) + case len(sr.Entries) > 1: + return "", errors.Newf(searchFailureMessage+": too many matching entries returned for user %s", username) + } + + return sr.Entries[0].DN, nil +} + +// ILDAPUtil is an interface for the `ldapauthccl` library to wrap various LDAP +// functionalities exposed by `go-ldap` library as part of CRDB modules for +// authN and authZ. +type ILDAPUtil interface { + // InitLDAPsConn creates a mTLS connection with the LDAP server taking + // arguments for domain CA, ldap client key and cert, ldap server & port + InitLDAPsConn(ctx context.Context, conf ldapAuthenticatorConf) error + // Bind perform a bind given a valid DN and LDAP password + Bind(ctx context.Context, userDN string, ldapPwd string) error + // Search performs search on LDAP server binding with bindDN and bindpwd + // expecting search arguments from HBA conf and crdb database connection + // string and returns the ldap userDN. + Search(ctx context.Context, conf ldapAuthenticatorConf, username string) (userDN string, err error) +} + +var _ ILDAPUtil = &ldapUtil{} + +// NewLDAPUtil initializes ldapUtil which implements the ILDAPUtil wrapper for +// client interface provided by `go-ldap`. This is needed for testing (to +// intercept the call to NewLDAPUtil and return the mocked NewLDAPUtil which has +// mock implementations for ILDAPUtil interface). +var NewLDAPUtil func(context.Context, ldapAuthenticatorConf) (ILDAPUtil, error) = func( + ctx context.Context, + conf ldapAuthenticatorConf, +) (ILDAPUtil, error) { + util := ldapUtil{tlsConfig: &tls.Config{}} + + if conf.domainCACert != "" { + util.tlsConfig.RootCAs = x509.NewCertPool() + if ok := util.tlsConfig.RootCAs.AppendCertsFromPEM([]byte(conf.domainCACert)); !ok { + return nil, errors.Newf(invalidLDAPConfMessage + ": set domain CA cert for ldap server is not valid") + } + } + + if conf.clientTLSCert != "" && conf.clientTLSKey != "" { + clientCert, err := tls.X509KeyPair([]byte(conf.clientTLSCert), []byte(conf.clientTLSKey)) + if err != nil { + return nil, errors.Wrap(err, invalidLDAPConfMessage+": error parsing client cert and key pair for mTLS") + } + util.tlsConfig.Certificates = []tls.Certificate{clientCert} + } else if conf.clientTLSCert != "" || conf.clientTLSKey != "" { + return nil, errors.Newf(invalidLDAPConfMessage + ": both client cert and key pair must be set for mTLS") + } + + return &util, nil +} diff --git a/pkg/ccl/ldapccl/main_test.go b/pkg/ccl/ldapccl/main_test.go new file mode 100644 index 000000000000..894c950bb698 --- /dev/null +++ b/pkg/ccl/ldapccl/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2024 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 ldapccl_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/ccl" + "github.com/cockroachdb/cockroach/pkg/security/securityassets" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + defer ccl.TestingEnableEnterprise()() + securityassets.SetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} + +//go:generate ../../util/leaktest/add-leaktest.sh *_test.go diff --git a/pkg/ccl/ldapccl/settings.go b/pkg/ccl/ldapccl/settings.go new file mode 100644 index 000000000000..d41806d0a45c --- /dev/null +++ b/pkg/ccl/ldapccl/settings.go @@ -0,0 +1,83 @@ +// Copyright 2024 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 ldapccl + +import ( + "crypto/x509" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/errors" +) + +// All cluster settings necessary for the LDAP authN/authZ feature. +const ( + baseLDAPAuthSettingName = "server.ldap_authentication." + ldapDomainCACertificateSettingName = baseLDAPAuthSettingName + "domain.custom_ca" + ldapClientTLSCertSettingName = baseLDAPAuthSettingName + "client.tls_certificate" + ldapClientTLSKeySettingName = baseLDAPAuthSettingName + "client.tls_key" +) + +// LDAPDomainCACertificate is CA cert PEM (appended to system's default CAs) for +// verifying LDAP server domain certificate. +var LDAPDomainCACertificate = settings.RegisterStringSetting( + settings.ApplicationLevel, + ldapDomainCACertificateSettingName, + "sets the custom root CA for verifying domain certificates when establishing connection with LDAP server", + "", + settings.WithPublic, + settings.WithReportable(false), + settings.Sensitive, + settings.WithValidateString(validateCertificate), +) + +// LDAPClientTLSCertSetting is optional cert key that client can use for mTLS. +var LDAPClientTLSCertSetting = settings.RegisterStringSetting( + settings.ApplicationLevel, + ldapClientTLSCertSettingName, + "sets the client certificate for establishing mTLS connection with LDAP server", + "", + settings.WithPublic, + settings.WithReportable(false), + settings.Sensitive, + settings.WithValidateString(validateCertificate), +) + +// LDAPClientTLSKeySetting is optional certificate that client can use for mTLS. +var LDAPClientTLSKeySetting = settings.RegisterStringSetting( + settings.ApplicationLevel, + ldapClientTLSKeySettingName, + "sets the client key for establishing mTLS connection with LDAP server", + "", + settings.WithPublic, + settings.WithReportable(false), + settings.Sensitive, + settings.WithValidateString(validatePrivateKey), +) + +func validateCertificate(_ *settings.Values, s string) error { + if len(s) != 0 { + if ok := x509.NewCertPool().AppendCertsFromPEM([]byte(s)); !ok { + return errors.Newf("LDAP initialization failed: invalid cert PEM block provided") + } + } + return nil +} + +// validatePrivateKey validates if post parsing private key PEM block we can +// obtain a `crypto.PrivateKey` object. It does not check if it pairs with the +// public key from provided certificate. +func validatePrivateKey(_ *settings.Values, s string) error { + if len(s) != 0 { + if _, err := security.PEMToPrivateKey([]byte(s)); err != nil { + return errors.Newf("LDAP initialization failed: invalid private key PEM provided") + } + } + return nil +} diff --git a/pkg/ccl/ldapccl/settings_test.go b/pkg/ccl/ldapccl/settings_test.go new file mode 100644 index 000000000000..9ba4e6e20597 --- /dev/null +++ b/pkg/ccl/ldapccl/settings_test.go @@ -0,0 +1,110 @@ +// Copyright 2024 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 ldapccl + +import ( + "path/filepath" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security/certnames" + "github.com/cockroachdb/cockroach/pkg/security/securityassets" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +func TestValidateLDAPDomainCACertificate(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + caCertPEMBlock, err := securityassets.GetLoader().ReadFile( + filepath.Join(certnames.EmbeddedCertsDir, certnames.EmbeddedCACert)) + require.NoError(t, err) + + tests := []struct { + name string + setting string + wantErr bool + expectedErr string + }{ + {"empty string", "", false, ""}, + {"valid CA certificate", string(caCertPEMBlock), false, ""}, + {"invalid CA certificate", "--BEGIN INVALID CERT `[]!@#%#^$&*", true, + "LDAP initialization failed: invalid cert PEM block provided"}, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if err := validateCertificate(nil, tt.setting); (err != nil) != tt.wantErr { + t.Errorf("validateLDAPDomainCACertificate() error = %v, wantErr %v", err, tt.wantErr) + } else if err != nil { + require.Regexp(t, tt.expectedErr, err.Error()) + } + }) + } +} + +func TestValidateLDAPClientCertificate(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + clientCertPEMBlock, err := securityassets.GetLoader().ReadFile( + filepath.Join(certnames.EmbeddedCertsDir, certnames.EmbeddedRootCert)) + require.NoError(t, err) + + tests := []struct { + name string + setting string + wantErr bool + expectedErr string + }{ + {"empty string", "", false, ""}, + {"valid client certificate", string(clientCertPEMBlock), false, ""}, + {"invalid client certificate", "--BEGIN INVALID CERT `[]!@#%#^$&*", true, + "LDAP initialization failed: invalid cert PEM block provided"}, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if err := validateCertificate(nil, tt.setting); (err != nil) != tt.wantErr { + t.Errorf("validateCertificate() error = %v, wantErr %v", err, tt.wantErr) + } else if err != nil { + require.Regexp(t, tt.expectedErr, err.Error()) + } + }) + } +} + +func TestValidateLDAPClientPrivateKey(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + clientPrivateKeyPEMBlock, err := securityassets.GetLoader().ReadFile( + filepath.Join(certnames.EmbeddedCertsDir, certnames.EmbeddedRootKey)) + require.NoError(t, err) + + tests := []struct { + name string + setting string + wantErr bool + expectedErr string + }{ + {"empty string", "", false, ""}, + {"valid client private key", string(clientPrivateKeyPEMBlock), false, ""}, + {"invalid private key certificate", "---BEGIN INVALID RSA PRIVATE KEY `[]!@#%#^$&*", true, + "LDAP initialization failed: invalid private key PEM provided"}, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if err := validatePrivateKey(nil, tt.setting); (err != nil) != tt.wantErr { + t.Errorf("validatePrivateKey() error = %v, wantErr %v", err, tt.wantErr) + } else if err != nil { + require.Regexp(t, tt.expectedErr, err.Error()) + } + }) + } +} diff --git a/pkg/sql/pgwire/auth_methods.go b/pkg/sql/pgwire/auth_methods.go index 8ab079a9ddf4..86f6461d3ff3 100644 --- a/pkg/sql/pgwire/auth_methods.go +++ b/pkg/sql/pgwire/auth_methods.go @@ -16,6 +16,7 @@ import ( "crypto/tls" "fmt" "math" + "sync" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/security/password" @@ -84,6 +85,13 @@ func loadDefaultMethods() { // The "trust" method accepts any connection attempt that matches // the current rule. RegisterAuthMethod("trust", authTrust, hba.ConnAny, NoOptionsAllowed) + // The "ldap" method requires a clear text password which will be used to bind + // with a LDAP server. The remaining connection parameters are provided in hba + // conf options + // + // Care should be taken by administrators to only accept this auth + // method over secure connections, e.g. those encrypted using SSL. + RegisterAuthMethod("ldap", authLDAP, hba.ConnAny, nil) } // AuthMethod is a top-level factory for composing the various @@ -106,6 +114,7 @@ var _ AuthMethod = authTrust var _ AuthMethod = authReject var _ AuthMethod = authSessionRevivalToken([]byte{}) var _ AuthMethod = authJwtToken +var _ AuthMethod = authLDAP // authPassword is the AuthMethod constructor for HBA method // "password": authenticate using a cleartext password received from @@ -790,3 +799,103 @@ func authJwtToken( }) return b, nil } + +// LDAPVerifier is an interface for `ldapauthccl` pkg to add ldap login support. +type LDAPVerifier interface { + // ValidateLDAPLogin Validates whether the password supplied could be used to + // bind to ldap server with a distinguished name obtained from performing a + // search operation using options provided in the hba conf and supplied sql + // username in db connection string. + ValidateLDAPLogin(_ context.Context, _ *cluster.Settings, + _ username.SQLUsername, + _ string, + _ *hba.Entry, + _ *identmap.Conf, + ) (detailedErrorMsg string, authError error) +} + +// ldapVerifier is a singleton global pgwire object which gets initialized from +// authLDAP method whenever an LDAP auth attempt happens. It depends on ldapccl +// module to be imported properly to override its default ConfigureLDAPAuth +// constructor. +var ldapVerifier LDAPVerifier + +type noLDAPConfigured struct{} + +func (c *noLDAPConfigured) ValidateLDAPLogin( + _ context.Context, + _ *cluster.Settings, + _ username.SQLUsername, + _ string, + _ *hba.Entry, + _ *identmap.Conf, +) (detailedErrorMsg string, authError error) { + return "", errors.New("LDAP based authentication requires CCL features") +} + +// ConfigureLDAPAuth is a hook for the `ldapauthccl` library to add LDAP login +// support. It's called to setup the LDAPVerifier just as it is needed. +var ConfigureLDAPAuth = func( + serverCtx context.Context, + ambientCtx log.AmbientContext, + st *cluster.Settings, + clusterUUID uuid.UUID, +) LDAPVerifier { + return &noLDAPConfigured{} +} + +// authLDAP is the AuthMethod constructor for the CRDB-specific +// ldap auth mechanism. +func authLDAP( + sCtx context.Context, + c AuthConn, + _ tls.ConnectionState, + execCfg *sql.ExecutorConfig, + entry *hba.Entry, + identMap *identmap.Conf, +) (*AuthBehaviors, error) { + sync.OnceFunc(func() { + ldapVerifier = ConfigureLDAPAuth(sCtx, execCfg.AmbientCtx, execCfg.Settings, execCfg.NodeInfo.LogicalClusterID()) + })() + + b := &AuthBehaviors{} + b.SetRoleMapper(UseProvidedIdentity) + b.SetAuthenticator(func(ctx context.Context, user username.SQLUsername, clientConnection bool, _ PasswordRetrievalFn, _ *ldap.DN) error { + c.LogAuthInfof(ctx, "LDAP password provided; attempting to bind to domain") + if !clientConnection { + err := errors.New("LDAP authentication is only available for client connections") + c.LogAuthFailed(ctx, eventpb.AuthFailReason_PRE_HOOK_ERROR, err) + return err + } + // Request password from client. + if err := c.SendAuthRequest(authCleartextPassword, nil /* data */); err != nil { + c.LogAuthFailed(ctx, eventpb.AuthFailReason_PRE_HOOK_ERROR, err) + return err + } + // Wait for the password response from the client. + pwdData, err := c.GetPwdData() + if err != nil { + c.LogAuthFailed(ctx, eventpb.AuthFailReason_PRE_HOOK_ERROR, err) + return err + } + + // Extract the LDAP password. + ldapPwd, err := passwordString(pwdData) + if err != nil { + c.LogAuthFailed(ctx, eventpb.AuthFailReason_PRE_HOOK_ERROR, err) + return err + } + // If there is no ldap pwd, send the Password Auth Failed error to make the + // client prompt for a password. + if len(ldapPwd) == 0 { + return security.NewErrPasswordUserAuthFailed(user) + } + if detailedErrors, authError := ldapVerifier.ValidateLDAPLogin(ctx, execCfg.Settings, user, ldapPwd, entry, identMap); authError != nil { + c.LogAuthFailed(ctx, eventpb.AuthFailReason_CREDENTIALS_INVALID, + errors.Join(authError, errors.Newf("%s", detailedErrors))) + return authError + } + return nil + }) + return b, nil +} diff --git a/pkg/sql/pgwire/testdata/auth/hba_syntax b/pkg/sql/pgwire/testdata/auth/hba_syntax index 26fe3117e2ae..530011d3c247 100644 --- a/pkg/sql/pgwire/testdata/auth/hba_syntax +++ b/pkg/sql/pgwire/testdata/auth/hba_syntax @@ -20,7 +20,7 @@ host all all 0.0.0.0/0 invalid ERROR: unimplemented: unknown auth method "invalid" (SQLSTATE 0A000) HINT: You have attempted to use a feature that is not yet implemented. -- -Supported methods: cert, cert-password, cert-scram-sha-256, password, reject, scram-sha-256, trust +Supported methods: cert, cert-password, cert-scram-sha-256, ldap, password, reject, scram-sha-256, trust # CockroachDB does not (yet?) support per-db HBA rules.