From 883bc5d80ce871e96a589f0cec4a9ccebbbc6eb2 Mon Sep 17 00:00:00 2001
From: Ben Darnell <ben@cockroachlabs.com>
Date: Thu, 16 Nov 2023 20:42:02 +0000
Subject: [PATCH 1/4] bazel: Add boringcrypto experiment to go_sdk_fips

The boringcrypto build tag/experiment is not necessary to enable FIPS
functionality with this toolchain, but it is necessary to expose the
crypto/boring.Enabled method, which is the application-visible
way to confirm that FIPS mode is in use.

Updates #114344
Release note: None
---
 WORKSPACE | 10 ++++++++++
 1 file changed, 10 insertions(+)

diff --git a/WORKSPACE b/WORKSPACE
index b4ea4c1f3d70..ba448a30e701 100644
--- a/WORKSPACE
+++ b/WORKSPACE
@@ -645,4 +645,14 @@ go_download_sdk(
     },
     urls = ["https://storage.googleapis.com/public-bazel-artifacts/go/20231019-214851/{}"],
     version = "1.21.3fips",
+    # In the golang-fips toolchain, FIPS-ready crypto packages are used by default, regardless of build tags.
+    # The boringcrypto experiment does almost nothing in this toolchain, but it does enable the use of the
+    # crypto/boring.Enabled() method which is the only application-visible way to inspect whether FIPS mode
+    # is working correctly.
+    #
+    # The golang-fips toolchain also supports an experiment `strictfipsruntime` which causes a panic at startup
+    # if the kernel is in FIPS mode but OpenSSL cannot be loaded. We do not currently use this experiment
+    # because A) we also want to detect the case when the kernel is not in FIPS mode and B) we want to be
+    # able to provide additional diagnostic information such as the expected version of OpenSSL.
+    experiments = ["boringcrypto"],
 )

From cf1379cb4ee5f28da20b52769ea6522e760e23ca Mon Sep 17 00:00:00 2001
From: Ben Darnell <ben@cockroachlabs.com>
Date: Tue, 14 Nov 2023 11:49:21 -0500
Subject: [PATCH 2/4] cliccl: Add debug enterprise-check-fips command

This command reports on the status of certain prerequisites for our fips-ready
builds.

Updates #114344

Release note (cli change): New command `cockroach debug
enterprise-check-fips` diagnoses errors in FIPS deployments
---
 .github/CODEOWNERS                            |  1 +
 pkg/BUILD.bazel                               |  1 +
 pkg/ccl/cliccl/BUILD.bazel                    |  2 +
 pkg/ccl/cliccl/debug.go                       | 56 +++++++++++++++
 pkg/ccl/securityccl/fipsccl/BUILD.bazel       | 17 +++++
 pkg/ccl/securityccl/fipsccl/build_boring.go   | 72 +++++++++++++++++++
 pkg/ccl/securityccl/fipsccl/build_noboring.go | 29 ++++++++
 pkg/ccl/securityccl/fipsccl/fips_linux.go     | 34 +++++++++
 pkg/ccl/securityccl/fipsccl/fips_nolinux.go   | 17 +++++
 9 files changed, 229 insertions(+)
 create mode 100644 pkg/ccl/securityccl/fipsccl/BUILD.bazel
 create mode 100644 pkg/ccl/securityccl/fipsccl/build_boring.go
 create mode 100644 pkg/ccl/securityccl/fipsccl/build_noboring.go
 create mode 100644 pkg/ccl/securityccl/fipsccl/fips_linux.go
 create mode 100644 pkg/ccl/securityccl/fipsccl/fips_nolinux.go

diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS
index 6c2c05836cd0..e714d3b78898 100644
--- a/.github/CODEOWNERS
+++ b/.github/CODEOWNERS
@@ -530,6 +530,7 @@
 /pkg/scheduledjobs/          @cockroachdb/jobs-prs @cockroachdb/disaster-recovery
 /pkg/security/               @cockroachdb/prodsec @cockroachdb/server-prs
 /pkg/security/clientsecopts/ @cockroachdb/sql-foundations @cockroachdb/prodsec
+/pkg/ccl/securityccl/        @cockroachdb/prodsec
 #!/pkg/settings/             @cockroachdb/unowned
 /pkg/spanconfig/                         @cockroachdb/kv-prs @cockroachdb/sql-foundations
 /pkg/spanconfig/spanconfigbounds/        @cockroachdb/sql-foundations
diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel
index 2ba242228746..1294efd8a28b 100644
--- a/pkg/BUILD.bazel
+++ b/pkg/BUILD.bazel
@@ -895,6 +895,7 @@ GO_TARGETS = [
     "//pkg/ccl/pgcryptoccl:pgcryptoccl_test",
     "//pkg/ccl/schemachangerccl:schemachangerccl",
     "//pkg/ccl/schemachangerccl:schemachangerccl_test",
+    "//pkg/ccl/securityccl/fipsccl:fipsccl",
     "//pkg/ccl/serverccl/adminccl:adminccl_test",
     "//pkg/ccl/serverccl/diagnosticsccl:diagnosticsccl_test",
     "//pkg/ccl/serverccl/statusccl:statusccl_test",
diff --git a/pkg/ccl/cliccl/BUILD.bazel b/pkg/ccl/cliccl/BUILD.bazel
index f4a428830232..ec1d26b52e0f 100644
--- a/pkg/ccl/cliccl/BUILD.bazel
+++ b/pkg/ccl/cliccl/BUILD.bazel
@@ -20,6 +20,7 @@ go_library(
         "//pkg/base",
         "//pkg/ccl/baseccl",
         "//pkg/ccl/cliccl/cliflagsccl",
+        "//pkg/ccl/securityccl/fipsccl",
         "//pkg/ccl/sqlproxyccl",
         "//pkg/ccl/sqlproxyccl/tenantdirsvr",
         "//pkg/ccl/storageccl/engineccl/enginepbccl",
@@ -41,6 +42,7 @@ go_library(
         "@com_github_cockroachdb_errors//oserror",
         "@com_github_cockroachdb_pebble//vfs",
         "@com_github_cockroachdb_redact//:redact",
+        "@com_github_olekukonko_tablewriter//:tablewriter",
         "@com_github_spf13_cobra//:cobra",
     ],
 )
diff --git a/pkg/ccl/cliccl/debug.go b/pkg/ccl/cliccl/debug.go
index 3fa0464cef8e..22be71676a43 100644
--- a/pkg/ccl/cliccl/debug.go
+++ b/pkg/ccl/cliccl/debug.go
@@ -14,12 +14,14 @@ import (
 	"fmt"
 	"os"
 	"path/filepath"
+	"runtime"
 	"sort"
 	"time"
 
 	"github.com/cockroachdb/cockroach/pkg/base"
 	"github.com/cockroachdb/cockroach/pkg/ccl/baseccl"
 	"github.com/cockroachdb/cockroach/pkg/ccl/cliccl/cliflagsccl"
+	"github.com/cockroachdb/cockroach/pkg/ccl/securityccl/fipsccl"
 	"github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl/enginepbccl"
 	"github.com/cockroachdb/cockroach/pkg/cli"
 	"github.com/cockroachdb/cockroach/pkg/cli/clierrorplus"
@@ -31,6 +33,7 @@ import (
 	"github.com/cockroachdb/cockroach/pkg/util/timeutil"
 	"github.com/cockroachdb/errors"
 	"github.com/cockroachdb/errors/oserror"
+	"github.com/olekukonko/tablewriter"
 	"github.com/spf13/cobra"
 )
 
@@ -102,12 +105,24 @@ with their env type and encryption settings (if applicable).
 		RunE: clierrorplus.MaybeDecorateError(runList),
 	}
 
+	checkFipsCmd := &cobra.Command{
+		Use:   "enterprise-check-fips",
+		Short: "print diagnostics for FIPS-ready configuration",
+		Long: `
+Performs various tests of this binary's ability to operate in FIPS-ready
+mode in the current environment.
+`,
+
+		RunE: clierrorplus.MaybeDecorateError(runCheckFips),
+	}
+
 	// Add commands to the root debug command.
 	// We can't add them to the lists of commands (eg: DebugCmdsForPebble) as cli init() is called before us.
 	cli.DebugCmd.AddCommand(encryptionStatusCmd)
 	cli.DebugCmd.AddCommand(encryptionActiveKeyCmd)
 	cli.DebugCmd.AddCommand(encryptionDecryptCmd)
 	cli.DebugCmd.AddCommand(encryptionRegistryList)
+	cli.DebugCmd.AddCommand(checkFipsCmd)
 
 	// Add the encryption flag to commands that need it.
 	// For the encryption-status command.
@@ -376,3 +391,44 @@ func getActiveEncryptionkey(dir string) (string, string, error) {
 
 	return setting.EncryptionType.String(), setting.KeyId, nil
 }
+
+func runCheckFips(cmd *cobra.Command, args []string) error {
+	if runtime.GOOS != "linux" {
+		return errors.New("FIPS-ready mode is only supported on linux")
+	}
+	// Our FIPS-ready deployments have three major requirements:
+	// 1. This binary is built with the golang-fips toolchain and running on linux
+	// 2. FIPS mode is enabled in the kernel.
+	// 3. We can dynamically load the OpenSSL library (which must be the same major version that was present at
+	//    build time). Verifying that the OpenSSL library is FIPS-compliant is outside the scope of this command.
+	table := tablewriter.NewWriter(os.Stdout)
+	table.SetBorder(false)
+	table.SetAlignment(tablewriter.ALIGN_LEFT)
+	emit := func(label string, status bool, detail string) {
+		statusSymbol := "❌"
+		if status {
+			statusSymbol = "✅"
+		}
+		table.Append([]string{label, statusSymbol, detail})
+	}
+
+	emit("FIPS-ready build", fipsccl.IsCompileTimeFIPSReady(), "")
+	buildOpenSSLVersion, soname, err := fipsccl.BuildOpenSSLVersion()
+	if err == nil {
+		table.Append([]string{"Build-time OpenSSL Version", "", buildOpenSSLVersion})
+		table.Append([]string{"OpenSSL library filename", "", soname})
+	}
+
+	isKernelEnabled, err := fipsccl.IsKernelEnabled()
+	detail := ""
+	if err != nil {
+		detail = err.Error()
+	}
+	emit("Kernel FIPS mode enabled", isKernelEnabled, detail)
+
+	emit("OpenSSL loaded", fipsccl.IsOpenSSLLoaded(), "")
+	emit("FIPS ready", fipsccl.IsFIPSReady(), "")
+
+	table.Render()
+	return nil
+}
diff --git a/pkg/ccl/securityccl/fipsccl/BUILD.bazel b/pkg/ccl/securityccl/fipsccl/BUILD.bazel
new file mode 100644
index 000000000000..ed17d4a9ccec
--- /dev/null
+++ b/pkg/ccl/securityccl/fipsccl/BUILD.bazel
@@ -0,0 +1,17 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library")
+
+go_library(
+    name = "fipsccl",
+    srcs = [
+        "build_boring.go",  # keep
+        "build_noboring.go",
+        "fips_linux.go",
+        "fips_nolinux.go",
+    ],
+    cgo = True,
+    importpath = "github.com/cockroachdb/cockroach/pkg/ccl/securityccl/fipsccl",
+    visibility = ["//visibility:public"],
+    deps = [
+        "@com_github_cockroachdb_errors//:errors",
+    ],
+)
diff --git a/pkg/ccl/securityccl/fipsccl/build_boring.go b/pkg/ccl/securityccl/fipsccl/build_boring.go
new file mode 100644
index 000000000000..011928745400
--- /dev/null
+++ b/pkg/ccl/securityccl/fipsccl/build_boring.go
@@ -0,0 +1,72 @@
+// 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
+//
+//go:build boringcrypto
+
+package fipsccl
+
+/*
+#include <openssl/ossl_typ.h>
+
+static unsigned long _fipsccl_openssl_version_number() {
+	return OPENSSL_VERSION_NUMBER;
+}
+*/
+import "C"
+
+import (
+	"crypto/boring"
+	"fmt"
+)
+
+// IsCompileTimeFIPSReady returns true if this binary was built with correct
+// toolchain and options, which is a prerequisite for FIPS-ready mode.
+// Note that we only support the golang-fips toolchain even though the
+// build tag we test for is "boringcrypto". The two are not actually
+// compatible because crypto/boring.Enabled is a bool in one and a function
+// in the other.
+func IsCompileTimeFIPSReady() bool {
+	return true
+}
+
+// IsOpenSSLLoaded returns true if the OpenSSL library has been found and
+// loaded.
+func IsOpenSSLLoaded() bool {
+	return boring.Enabled()
+}
+
+// IsFIPSReady returns true if all of our FIPS readiness checks succeed.
+func IsFIPSReady() bool {
+	// The golang-fips toolchain only attempts to load OpenSSL if the kernel
+	// fips mode is enabled. Therefore we only need this single check for our
+	// overall fips-readiness status. We could redundantly call IsBoringBuild
+	// and IsKernelEnabled, but doing so would risk some divergence between our
+	// implementation and the toolchain itself so it's better at this time to
+	// use the single check.
+	return IsOpenSSLLoaded()
+}
+
+// BuildOpenSSLVersion returns the version number of OpenSSL that was used at
+// build time. The first return value is the hex value of the
+// OPENSSL_VERSION_NUMBER constant (for example, 10100000 for OpenSSL 1.1 and
+// 30000000 for OpenSSL 3.0), and the second is the versioned name of the
+// libcrypto.so file.
+func BuildOpenSSLVersion() (string, string, error) {
+	buildVersion := uint64(C._fipsccl_openssl_version_number())
+	var soname string
+	// Reference:
+	// https://github.com/golang-fips/go/blob/7f64529ab80e5d394bb2496e982d6f6e11023902/patches/001-initial-openssl-for-fips.patch#L3476-L3482
+	if buildVersion < 0x10100000 {
+		soname = "libcrypto.so.10"
+	} else if buildVersion < 0x30000000 {
+		soname = "libcrypto.so.1.1"
+	} else {
+		soname = "libcrypto.so.3"
+	}
+	return fmt.Sprintf("%x", buildVersion), soname, nil
+}
diff --git a/pkg/ccl/securityccl/fipsccl/build_noboring.go b/pkg/ccl/securityccl/fipsccl/build_noboring.go
new file mode 100644
index 000000000000..e80044530ff2
--- /dev/null
+++ b/pkg/ccl/securityccl/fipsccl/build_noboring.go
@@ -0,0 +1,29 @@
+// 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
+//
+//go:build !boringcrypto
+
+package fipsccl
+
+import "github.com/cockroachdb/errors"
+
+func IsCompileTimeFIPSReady() bool {
+	return false
+}
+
+func IsOpenSSLLoaded() bool {
+	return false
+}
+
+func IsFIPSReady() bool {
+	return false
+}
+
+func BuildOpenSSLVersion() (string, string, error) {
+	return "", "", errors.New("openssl support not present")
+}
diff --git a/pkg/ccl/securityccl/fipsccl/fips_linux.go b/pkg/ccl/securityccl/fipsccl/fips_linux.go
new file mode 100644
index 000000000000..6d841573bcbd
--- /dev/null
+++ b/pkg/ccl/securityccl/fipsccl/fips_linux.go
@@ -0,0 +1,34 @@
+// 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 fipsccl
+
+import (
+	"fmt"
+	"os"
+
+	"github.com/cockroachdb/errors"
+)
+
+const fipsSysctlFilename = "/proc/sys/crypto/fips_enabled"
+
+// IsKernelEnabled returns true if FIPS mode is enabled in the kernel
+// (by reading the crypto.fips_enabled sysctl).
+func IsKernelEnabled() (bool, error) {
+	data, err := os.ReadFile(fipsSysctlFilename)
+	if err != nil {
+		return false, err
+	}
+	if len(data) == 0 {
+		return false, errors.New("sysctl file empty")
+	}
+	if data[0] == '1' {
+		return true, nil
+	}
+	return false, fmt.Errorf("sysctl value: %q", data)
+}
diff --git a/pkg/ccl/securityccl/fipsccl/fips_nolinux.go b/pkg/ccl/securityccl/fipsccl/fips_nolinux.go
new file mode 100644
index 000000000000..247548b03b25
--- /dev/null
+++ b/pkg/ccl/securityccl/fipsccl/fips_nolinux.go
@@ -0,0 +1,17 @@
+// 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
+//
+//go:build !linux
+
+package fipsccl
+
+import "github.com/cockroachdb/errors"
+
+func IsKernelEnabled() (bool, error) {
+	return false, errors.New("only supported on linux")
+}

From 1ceb4232e6b5be3d38af28ce8afc73cc308a7374 Mon Sep 17 00:00:00 2001
From: Ben Darnell <ben@cockroachlabs.com>
Date: Fri, 17 Nov 2023 15:52:52 +0000
Subject: [PATCH 3/4] cliccl: Add global --enterprise-require-fips-ready flag

Previously, misconfigurations of the FIPS environment would result in a
silent fallback to non-FIPS-compliant Go cryptography. This flag permits
users who require FIPS compliance to add some checks to CockroachDB
startup to ensure that the Go crypto implementation will not be used.

Updates #114344

Release note (cli change): New flag --enterprise-require-fips-ready
can be added to any CRDB command to prevent startup if certain
prerequisites for FIPS compliance are not met.
---
 pkg/ccl/cliccl/BUILD.bazel |  3 ++
 pkg/ccl/cliccl/flags.go    | 56 ++++++++++++++++++++++++++++++++++++++
 pkg/cli/cli.go             |  5 ++++
 3 files changed, 64 insertions(+)

diff --git a/pkg/ccl/cliccl/BUILD.bazel b/pkg/ccl/cliccl/BUILD.bazel
index ec1d26b52e0f..71116c967f9d 100644
--- a/pkg/ccl/cliccl/BUILD.bazel
+++ b/pkg/ccl/cliccl/BUILD.bazel
@@ -27,10 +27,12 @@ go_library(
         "//pkg/ccl/utilccl",
         "//pkg/ccl/workloadccl/cliccl",
         "//pkg/cli",
+        "//pkg/cli/clierror",
         "//pkg/cli/clierrorplus",
         "//pkg/cli/cliflagcfg",
         "//pkg/cli/cliflags",
         "//pkg/cli/democluster",
+        "//pkg/cli/exit",
         "//pkg/storage",
         "//pkg/storage/enginepb",
         "//pkg/util/log",
@@ -44,6 +46,7 @@ go_library(
         "@com_github_cockroachdb_redact//:redact",
         "@com_github_olekukonko_tablewriter//:tablewriter",
         "@com_github_spf13_cobra//:cobra",
+        "@com_github_spf13_pflag//:pflag",
     ],
 )
 
diff --git a/pkg/ccl/cliccl/flags.go b/pkg/ccl/cliccl/flags.go
index 58bd3e8d99ba..0c19bd81e0b5 100644
--- a/pkg/ccl/cliccl/flags.go
+++ b/pkg/ccl/cliccl/flags.go
@@ -9,11 +9,59 @@
 package cliccl
 
 import (
+	"os"
+	"strconv"
+
+	"github.com/cockroachdb/cockroach/pkg/ccl/securityccl/fipsccl"
 	"github.com/cockroachdb/cockroach/pkg/cli"
+	"github.com/cockroachdb/cockroach/pkg/cli/clierror"
 	"github.com/cockroachdb/cockroach/pkg/cli/cliflagcfg"
 	"github.com/cockroachdb/cockroach/pkg/cli/cliflags"
+	"github.com/cockroachdb/cockroach/pkg/cli/exit"
+	"github.com/cockroachdb/errors"
+	"github.com/spf13/pflag"
 )
 
+type requireFipsFlag bool
+
+// Type implements the pflag.Value interface.
+func (f *requireFipsFlag) Type() string {
+	return "bool"
+}
+
+// String implements the pflag.Value interface.
+func (f *requireFipsFlag) String() string {
+	return strconv.FormatBool(bool(*f))
+}
+
+// Set implements the pflag.Value interface.
+func (f *requireFipsFlag) Set(s string) error {
+	v, err := strconv.ParseBool(s)
+	if err != nil {
+		return err
+	}
+	// We implement the logic of this check in the flag setter itself because it
+	// applies to all commands and we do not have another good way to inject
+	// this behavior globally (PersistentPreRun functions don't help because
+	// they are inherited across different levels of the command hierarchy only
+	// if that level does not have its own hook).
+	if v && !fipsccl.IsFIPSReady() {
+		err := errors.WithHint(errors.New("FIPS readiness checks failed"), "Run `cockroach debug enterprise-check-fips` for details")
+		clierror.OutputError(os.Stderr, err, true, false)
+		exit.WithCode(exit.UnspecifiedError())
+	}
+	*f = requireFipsFlag(v)
+	return nil
+}
+
+var _ pflag.Value = (*requireFipsFlag)(nil)
+
+// IsBoolFlag implements a non-public pflag interface to indicate that this
+// flag is used without an explicit value.
+func (*requireFipsFlag) IsBoolFlag() bool {
+	return true
+}
+
 func init() {
 	// Multi-tenancy proxy command flags.
 	{
@@ -44,4 +92,12 @@ func init() {
 		// Use StringFlagDepth to avoid conflicting with the already registered KVAddrs env var.
 		cliflagcfg.StringFlagDepth(1, f, &testDirectorySvrContext.kvAddrs, cliflags.KVAddrs)
 	})
+
+	// FIPS verification flags.
+	cli.RegisterFlags(func() {
+		cmd := cli.CockroachCmd()
+		var requireFips = requireFipsFlag(false)
+		flag := cmd.PersistentFlags().VarPF(&requireFips, "enterprise-require-fips-ready", "", "abort if FIPS readiness checks fail")
+		flag.NoOptDefVal = "true"
+	})
 }
diff --git a/pkg/cli/cli.go b/pkg/cli/cli.go
index 7beb18e35f4a..ec1ff3f14f79 100644
--- a/pkg/cli/cli.go
+++ b/pkg/cli/cli.go
@@ -228,6 +228,11 @@ var cockroachCmd = &cobra.Command{
 	},
 }
 
+// CockroachCmd returns the root cockroach Command object.
+func CockroachCmd() *cobra.Command {
+	return cockroachCmd
+}
+
 var workloadCmd = workloadcli.WorkloadCmd(true /* userFacing */)
 
 func init() {

From 62af449104d1352fd216f0943102be637dace5c1 Mon Sep 17 00:00:00 2001
From: Ben Darnell <ben@cockroachlabs.com>
Date: Sun, 19 Nov 2023 19:11:14 +0000
Subject: [PATCH 4/4] fipsccl: Add a SQL function to check fips status

This function provides a way to verify FIPS readiness without modifying
the deployment to add the --enterprise-require-fips-ready flag.

Updates #114344

Release note (enterprise change): New SQL function fips_ready can be
used to verify the FIPS readiness of the gateway node.
---
 pkg/ccl/BUILD.bazel                           |  1 +
 pkg/ccl/ccl_init.go                           |  1 +
 .../testdata/logic_test/fips_ready            | 14 ++++
 .../tests/3node-tenant/generated_test.go      |  7 ++
 .../tests/fakedist-disk/BUILD.bazel           |  2 +-
 .../tests/fakedist-disk/generated_test.go     |  7 ++
 .../tests/fakedist-vec-off/BUILD.bazel        |  2 +-
 .../tests/fakedist-vec-off/generated_test.go  |  7 ++
 .../logictestccl/tests/fakedist/BUILD.bazel   |  2 +-
 .../tests/fakedist/generated_test.go          |  7 ++
 .../local-legacy-schema-changer/BUILD.bazel   |  2 +-
 .../generated_test.go                         |  7 ++
 .../tests/local-mixed-23.1/BUILD.bazel        |  2 +-
 .../tests/local-mixed-23.1/generated_test.go  |  7 ++
 .../tests/local-mixed-23.2/BUILD.bazel        |  2 +-
 .../tests/local-mixed-23.2/generated_test.go  |  7 ++
 .../tests/local-vec-off/BUILD.bazel           |  2 +-
 .../tests/local-vec-off/generated_test.go     |  7 ++
 pkg/ccl/logictestccl/tests/local/BUILD.bazel  |  2 +-
 .../tests/local/generated_test.go             |  7 ++
 pkg/ccl/securityccl/fipsccl/BUILD.bazel       |  9 +++
 pkg/ccl/securityccl/fipsccl/sql.go            | 64 +++++++++++++++++++
 pkg/sql/sem/builtins/fixed_oids.go            |  1 +
 23 files changed, 161 insertions(+), 8 deletions(-)
 create mode 100644 pkg/ccl/logictestccl/testdata/logic_test/fips_ready
 create mode 100644 pkg/ccl/securityccl/fipsccl/sql.go

diff --git a/pkg/ccl/BUILD.bazel b/pkg/ccl/BUILD.bazel
index b86474e903d9..a324ae2bb115 100644
--- a/pkg/ccl/BUILD.bazel
+++ b/pkg/ccl/BUILD.bazel
@@ -21,6 +21,7 @@ go_library(
         "//pkg/ccl/oidcccl",
         "//pkg/ccl/partitionccl",
         "//pkg/ccl/pgcryptoccl",
+        "//pkg/ccl/securityccl/fipsccl",
         "//pkg/ccl/storageccl",
         "//pkg/ccl/storageccl/engineccl",
         "//pkg/ccl/streamingccl/streamingest",
diff --git a/pkg/ccl/ccl_init.go b/pkg/ccl/ccl_init.go
index 4a224d5cd10c..ca925de44dd5 100644
--- a/pkg/ccl/ccl_init.go
+++ b/pkg/ccl/ccl_init.go
@@ -27,6 +27,7 @@ import (
 	_ "github.com/cockroachdb/cockroach/pkg/ccl/oidcccl"
 	_ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl"
 	_ "github.com/cockroachdb/cockroach/pkg/ccl/pgcryptoccl"
+	_ "github.com/cockroachdb/cockroach/pkg/ccl/securityccl/fipsccl"
 	_ "github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
 	_ "github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl"
 	_ "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamingest"
diff --git a/pkg/ccl/logictestccl/testdata/logic_test/fips_ready b/pkg/ccl/logictestccl/testdata/logic_test/fips_ready
new file mode 100644
index 000000000000..be6f4294bfa9
--- /dev/null
+++ b/pkg/ccl/logictestccl/testdata/logic_test/fips_ready
@@ -0,0 +1,14 @@
+subtest fips_ready
+
+# We do not have the plumbing that would let test cases know whether they are
+# running in a fips environment or not so this is just a very basic test to
+# make sure that all the registration, oids, etc work properly.
+query _
+SELECT crdb_internal.fips_ready()
+----
+_
+
+user testuser
+
+statement error pq: crdb_internal\.fips_ready\(\): user testuser does not have VIEWCLUSTERSETTING system privilege
+SELECT crdb_internal.fips_ready()
diff --git a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go
index dba1b80d43e3..0a799ad3904d 100644
--- a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go
+++ b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go
@@ -2606,6 +2606,13 @@ func TestTenantLogicCCL_crdb_internal_tenant(
 	runCCLLogicTest(t, "crdb_internal_tenant")
 }
 
+func TestTenantLogicCCL_fips_ready(
+	t *testing.T,
+) {
+	defer leaktest.AfterTest(t)()
+	runCCLLogicTest(t, "fips_ready")
+}
+
 func TestTenantLogicCCL_new_schema_changer(
 	t *testing.T,
 ) {
diff --git a/pkg/ccl/logictestccl/tests/fakedist-disk/BUILD.bazel b/pkg/ccl/logictestccl/tests/fakedist-disk/BUILD.bazel
index 2fef49525e31..00ba9ff29601 100644
--- a/pkg/ccl/logictestccl/tests/fakedist-disk/BUILD.bazel
+++ b/pkg/ccl/logictestccl/tests/fakedist-disk/BUILD.bazel
@@ -9,7 +9,7 @@ go_test(
         "//pkg/ccl/logictestccl:testdata",  # keep
     ],
     exec_properties = {"Pool": "large"},
-    shard_count = 6,
+    shard_count = 7,
     tags = [
         "ccl_test",
         "cpu:2",
diff --git a/pkg/ccl/logictestccl/tests/fakedist-disk/generated_test.go b/pkg/ccl/logictestccl/tests/fakedist-disk/generated_test.go
index ed15a067a5b3..c901c45d24e8 100644
--- a/pkg/ccl/logictestccl/tests/fakedist-disk/generated_test.go
+++ b/pkg/ccl/logictestccl/tests/fakedist-disk/generated_test.go
@@ -78,6 +78,13 @@ func TestLogic_tmp(t *testing.T) {
 	logictest.RunLogicTests(t, logictest.TestServerArgs{}, configIdx, glob)
 }
 
+func TestCCLLogic_fips_ready(
+	t *testing.T,
+) {
+	defer leaktest.AfterTest(t)()
+	runCCLLogicTest(t, "fips_ready")
+}
+
 func TestCCLLogic_new_schema_changer(
 	t *testing.T,
 ) {
diff --git a/pkg/ccl/logictestccl/tests/fakedist-vec-off/BUILD.bazel b/pkg/ccl/logictestccl/tests/fakedist-vec-off/BUILD.bazel
index 731a6dba7e27..92498312b1e4 100644
--- a/pkg/ccl/logictestccl/tests/fakedist-vec-off/BUILD.bazel
+++ b/pkg/ccl/logictestccl/tests/fakedist-vec-off/BUILD.bazel
@@ -9,7 +9,7 @@ go_test(
         "//pkg/ccl/logictestccl:testdata",  # keep
     ],
     exec_properties = {"Pool": "large"},
-    shard_count = 6,
+    shard_count = 7,
     tags = [
         "ccl_test",
         "cpu:2",
diff --git a/pkg/ccl/logictestccl/tests/fakedist-vec-off/generated_test.go b/pkg/ccl/logictestccl/tests/fakedist-vec-off/generated_test.go
index 662396114099..eeda3c15e837 100644
--- a/pkg/ccl/logictestccl/tests/fakedist-vec-off/generated_test.go
+++ b/pkg/ccl/logictestccl/tests/fakedist-vec-off/generated_test.go
@@ -78,6 +78,13 @@ func TestLogic_tmp(t *testing.T) {
 	logictest.RunLogicTests(t, logictest.TestServerArgs{}, configIdx, glob)
 }
 
+func TestCCLLogic_fips_ready(
+	t *testing.T,
+) {
+	defer leaktest.AfterTest(t)()
+	runCCLLogicTest(t, "fips_ready")
+}
+
 func TestCCLLogic_new_schema_changer(
 	t *testing.T,
 ) {
diff --git a/pkg/ccl/logictestccl/tests/fakedist/BUILD.bazel b/pkg/ccl/logictestccl/tests/fakedist/BUILD.bazel
index 6240d6f37d3b..63b47e74f0f6 100644
--- a/pkg/ccl/logictestccl/tests/fakedist/BUILD.bazel
+++ b/pkg/ccl/logictestccl/tests/fakedist/BUILD.bazel
@@ -9,7 +9,7 @@ go_test(
         "//pkg/ccl/logictestccl:testdata",  # keep
     ],
     exec_properties = {"Pool": "large"},
-    shard_count = 7,
+    shard_count = 8,
     tags = [
         "ccl_test",
         "cpu:2",
diff --git a/pkg/ccl/logictestccl/tests/fakedist/generated_test.go b/pkg/ccl/logictestccl/tests/fakedist/generated_test.go
index 4781d36ae881..740b8da2992f 100644
--- a/pkg/ccl/logictestccl/tests/fakedist/generated_test.go
+++ b/pkg/ccl/logictestccl/tests/fakedist/generated_test.go
@@ -78,6 +78,13 @@ func TestLogic_tmp(t *testing.T) {
 	logictest.RunLogicTests(t, logictest.TestServerArgs{}, configIdx, glob)
 }
 
+func TestCCLLogic_fips_ready(
+	t *testing.T,
+) {
+	defer leaktest.AfterTest(t)()
+	runCCLLogicTest(t, "fips_ready")
+}
+
 func TestCCLLogic_new_schema_changer(
 	t *testing.T,
 ) {
diff --git a/pkg/ccl/logictestccl/tests/local-legacy-schema-changer/BUILD.bazel b/pkg/ccl/logictestccl/tests/local-legacy-schema-changer/BUILD.bazel
index 779fa2337028..902f07d858a8 100644
--- a/pkg/ccl/logictestccl/tests/local-legacy-schema-changer/BUILD.bazel
+++ b/pkg/ccl/logictestccl/tests/local-legacy-schema-changer/BUILD.bazel
@@ -9,7 +9,7 @@ go_test(
         "//pkg/ccl/logictestccl:testdata",  # keep
     ],
     exec_properties = {"Pool": "large"},
-    shard_count = 6,
+    shard_count = 7,
     tags = [
         "ccl_test",
         "cpu:1",
diff --git a/pkg/ccl/logictestccl/tests/local-legacy-schema-changer/generated_test.go b/pkg/ccl/logictestccl/tests/local-legacy-schema-changer/generated_test.go
index 74fb8bc7ccb8..78c74e88e1d2 100644
--- a/pkg/ccl/logictestccl/tests/local-legacy-schema-changer/generated_test.go
+++ b/pkg/ccl/logictestccl/tests/local-legacy-schema-changer/generated_test.go
@@ -78,6 +78,13 @@ func TestLogic_tmp(t *testing.T) {
 	logictest.RunLogicTests(t, logictest.TestServerArgs{}, configIdx, glob)
 }
 
+func TestCCLLogic_fips_ready(
+	t *testing.T,
+) {
+	defer leaktest.AfterTest(t)()
+	runCCLLogicTest(t, "fips_ready")
+}
+
 func TestCCLLogic_new_schema_changer(
 	t *testing.T,
 ) {
diff --git a/pkg/ccl/logictestccl/tests/local-mixed-23.1/BUILD.bazel b/pkg/ccl/logictestccl/tests/local-mixed-23.1/BUILD.bazel
index 322c935e4c17..76267f75c810 100644
--- a/pkg/ccl/logictestccl/tests/local-mixed-23.1/BUILD.bazel
+++ b/pkg/ccl/logictestccl/tests/local-mixed-23.1/BUILD.bazel
@@ -9,7 +9,7 @@ go_test(
         "//pkg/ccl/logictestccl:testdata",  # keep
     ],
     exec_properties = {"Pool": "large"},
-    shard_count = 7,
+    shard_count = 8,
     tags = [
         "ccl_test",
         "cpu:1",
diff --git a/pkg/ccl/logictestccl/tests/local-mixed-23.1/generated_test.go b/pkg/ccl/logictestccl/tests/local-mixed-23.1/generated_test.go
index d6082e807060..5edb77cdcb07 100644
--- a/pkg/ccl/logictestccl/tests/local-mixed-23.1/generated_test.go
+++ b/pkg/ccl/logictestccl/tests/local-mixed-23.1/generated_test.go
@@ -85,6 +85,13 @@ func TestCCLLogic_changefeed(
 	runCCLLogicTest(t, "changefeed")
 }
 
+func TestCCLLogic_fips_ready(
+	t *testing.T,
+) {
+	defer leaktest.AfterTest(t)()
+	runCCLLogicTest(t, "fips_ready")
+}
+
 func TestCCLLogic_new_schema_changer(
 	t *testing.T,
 ) {
diff --git a/pkg/ccl/logictestccl/tests/local-mixed-23.2/BUILD.bazel b/pkg/ccl/logictestccl/tests/local-mixed-23.2/BUILD.bazel
index f962607f0235..a8c7ebea2699 100644
--- a/pkg/ccl/logictestccl/tests/local-mixed-23.2/BUILD.bazel
+++ b/pkg/ccl/logictestccl/tests/local-mixed-23.2/BUILD.bazel
@@ -9,7 +9,7 @@ go_test(
         "//pkg/ccl/logictestccl:testdata",  # keep
     ],
     exec_properties = {"Pool": "large"},
-    shard_count = 6,
+    shard_count = 7,
     tags = [
         "ccl_test",
         "cpu:1",
diff --git a/pkg/ccl/logictestccl/tests/local-mixed-23.2/generated_test.go b/pkg/ccl/logictestccl/tests/local-mixed-23.2/generated_test.go
index 83d04f190498..571b77d39762 100644
--- a/pkg/ccl/logictestccl/tests/local-mixed-23.2/generated_test.go
+++ b/pkg/ccl/logictestccl/tests/local-mixed-23.2/generated_test.go
@@ -78,6 +78,13 @@ func TestLogic_tmp(t *testing.T) {
 	logictest.RunLogicTests(t, logictest.TestServerArgs{}, configIdx, glob)
 }
 
+func TestCCLLogic_fips_ready(
+	t *testing.T,
+) {
+	defer leaktest.AfterTest(t)()
+	runCCLLogicTest(t, "fips_ready")
+}
+
 func TestCCLLogic_new_schema_changer(
 	t *testing.T,
 ) {
diff --git a/pkg/ccl/logictestccl/tests/local-vec-off/BUILD.bazel b/pkg/ccl/logictestccl/tests/local-vec-off/BUILD.bazel
index 97bcb27684b8..62f452f8d75d 100644
--- a/pkg/ccl/logictestccl/tests/local-vec-off/BUILD.bazel
+++ b/pkg/ccl/logictestccl/tests/local-vec-off/BUILD.bazel
@@ -9,7 +9,7 @@ go_test(
         "//pkg/ccl/logictestccl:testdata",  # keep
     ],
     exec_properties = {"Pool": "large"},
-    shard_count = 6,
+    shard_count = 7,
     tags = [
         "ccl_test",
         "cpu:1",
diff --git a/pkg/ccl/logictestccl/tests/local-vec-off/generated_test.go b/pkg/ccl/logictestccl/tests/local-vec-off/generated_test.go
index eae4fccbd6fb..d4243389a683 100644
--- a/pkg/ccl/logictestccl/tests/local-vec-off/generated_test.go
+++ b/pkg/ccl/logictestccl/tests/local-vec-off/generated_test.go
@@ -78,6 +78,13 @@ func TestLogic_tmp(t *testing.T) {
 	logictest.RunLogicTests(t, logictest.TestServerArgs{}, configIdx, glob)
 }
 
+func TestCCLLogic_fips_ready(
+	t *testing.T,
+) {
+	defer leaktest.AfterTest(t)()
+	runCCLLogicTest(t, "fips_ready")
+}
+
 func TestCCLLogic_new_schema_changer(
 	t *testing.T,
 ) {
diff --git a/pkg/ccl/logictestccl/tests/local/BUILD.bazel b/pkg/ccl/logictestccl/tests/local/BUILD.bazel
index 70300c42c6f5..4956870d306f 100644
--- a/pkg/ccl/logictestccl/tests/local/BUILD.bazel
+++ b/pkg/ccl/logictestccl/tests/local/BUILD.bazel
@@ -9,7 +9,7 @@ go_test(
         "//pkg/ccl/logictestccl:testdata",  # keep
     ],
     exec_properties = {"Pool": "large"},
-    shard_count = 20,
+    shard_count = 21,
     tags = [
         "ccl_test",
         "cpu:1",
diff --git a/pkg/ccl/logictestccl/tests/local/generated_test.go b/pkg/ccl/logictestccl/tests/local/generated_test.go
index dd66a1be3062..096c7c496613 100644
--- a/pkg/ccl/logictestccl/tests/local/generated_test.go
+++ b/pkg/ccl/logictestccl/tests/local/generated_test.go
@@ -120,6 +120,13 @@ func TestCCLLogic_explain_redact(
 	runCCLLogicTest(t, "explain_redact")
 }
 
+func TestCCLLogic_fips_ready(
+	t *testing.T,
+) {
+	defer leaktest.AfterTest(t)()
+	runCCLLogicTest(t, "fips_ready")
+}
+
 func TestCCLLogic_new_schema_changer(
 	t *testing.T,
 ) {
diff --git a/pkg/ccl/securityccl/fipsccl/BUILD.bazel b/pkg/ccl/securityccl/fipsccl/BUILD.bazel
index ed17d4a9ccec..c71c275b068e 100644
--- a/pkg/ccl/securityccl/fipsccl/BUILD.bazel
+++ b/pkg/ccl/securityccl/fipsccl/BUILD.bazel
@@ -7,11 +7,20 @@ go_library(
         "build_noboring.go",
         "fips_linux.go",
         "fips_nolinux.go",
+        "sql.go",
     ],
     cgo = True,
     importpath = "github.com/cockroachdb/cockroach/pkg/ccl/securityccl/fipsccl",
     visibility = ["//visibility:public"],
     deps = [
+        "//pkg/ccl/utilccl",
+        "//pkg/sql/privilege",
+        "//pkg/sql/roleoption",
+        "//pkg/sql/sem/eval",
+        "//pkg/sql/sem/tree",
+        "//pkg/sql/sem/volatility",
+        "//pkg/sql/syntheticprivilege",
+        "//pkg/sql/types",
         "@com_github_cockroachdb_errors//:errors",
     ],
 )
diff --git a/pkg/ccl/securityccl/fipsccl/sql.go b/pkg/ccl/securityccl/fipsccl/sql.go
new file mode 100644
index 000000000000..5b76ca4c53ad
--- /dev/null
+++ b/pkg/ccl/securityccl/fipsccl/sql.go
@@ -0,0 +1,64 @@
+// 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 fipsccl
+
+import (
+	"context"
+
+	"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
+	"github.com/cockroachdb/cockroach/pkg/sql/privilege"
+	"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
+	"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
+	"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+	"github.com/cockroachdb/cockroach/pkg/sql/sem/volatility"
+	"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege"
+	"github.com/cockroachdb/cockroach/pkg/sql/types"
+)
+
+func init() {
+	overload := tree.Overload{
+		Types:      tree.ParamTypes{},
+		ReturnType: tree.FixedReturnType(types.Bool),
+		Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
+			if err := utilccl.CheckEnterpriseEnabled(
+				evalCtx.Settings, evalCtx.ClusterID, "fips_ready",
+			); err != nil {
+				return nil, err
+			}
+			// It's debatable whether we need a permission check here at all.
+			// It's not very sensitive and is (currently) a very cheap function
+			// call. However, it's something that regular users should have no
+			// reason to look at so in the interest of least privilege we put it
+			// behind the VIEWCLUSTERSETTING privilige.
+			session := evalCtx.SessionAccessor
+			isAdmin, err := session.HasAdminRole(ctx)
+			if err != nil {
+				return nil, err
+			}
+			if !isAdmin {
+				hasView, err := session.HasRoleOption(ctx, roleoption.VIEWCLUSTERSETTING)
+				if err != nil {
+					return nil, err
+				}
+				if !hasView {
+					if err := session.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERSETTING); err != nil {
+						return nil, err
+					}
+				}
+			}
+			return tree.MakeDBool(tree.DBool(IsFIPSReady())), nil
+		},
+		Class:      tree.NormalClass,
+		Volatility: volatility.Stable,
+	}
+
+	utilccl.RegisterCCLBuiltin("crdb_internal.fips_ready",
+		`Returns true if all FIPS readiness checks pass.`,
+		overload)
+}
diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go
index a9cb1ae51e4e..a416bbec92c8 100644
--- a/pkg/sql/sem/builtins/fixed_oids.go
+++ b/pkg/sql/sem/builtins/fixed_oids.go
@@ -2509,6 +2509,7 @@ var builtinOidsArray = []string{
 	2540: `information_schema._pg_datetime_precision(typid: oid, typmod: int4) -> int`,
 	2541: `information_schema._pg_interval_type(typid: oid, typmod: int4) -> string`,
 	2542: `crdb_internal.release_series(version: string) -> string`,
+	2543: `crdb_internal.fips_ready() -> bool`,
 }
 
 var builtinOidsBySignature map[string]oid.Oid