From 8271bf200e5f56c8c98e5fead03a1529a04991f4 Mon Sep 17 00:00:00 2001 From: Andy Yang Date: Tue, 29 Aug 2023 16:19:50 -0700 Subject: [PATCH 1/4] pgcryptocipherccl: move pgcryptocipher package to CCL Release note: None --- .github/CODEOWNERS | 1 + pkg/BUILD.bazel | 6 +++--- .../pgcryptoccl/pgcryptocipherccl}/BUILD.bazel | 9 +++++---- .../pgcryptoccl/pgcryptocipherccl}/cipher_method.go | 12 +++++------- .../pgcryptocipherccl}/cipher_method_test.go | 12 +++++------- pkg/ccl/pgcryptoccl/pgcryptocipherccl/doc.go | 11 +++++++++++ .../pgcryptoccl/pgcryptocipherccl}/padding.go | 12 +++++------- .../pgcryptoccl/pgcryptocipherccl}/padding_test.go | 12 +++++------- pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/doc.go | 13 ------------- 9 files changed, 40 insertions(+), 48 deletions(-) rename pkg/{sql/sem/builtins/pgcrypto/pgcryptocipher => ccl/pgcryptoccl/pgcryptocipherccl}/BUILD.bazel (73%) rename pkg/{sql/sem/builtins/pgcrypto/pgcryptocipher => ccl/pgcryptoccl/pgcryptocipherccl}/cipher_method.go (85%) rename pkg/{sql/sem/builtins/pgcrypto/pgcryptocipher => ccl/pgcryptoccl/pgcryptocipherccl}/cipher_method_test.go (83%) create mode 100644 pkg/ccl/pgcryptoccl/pgcryptocipherccl/doc.go rename pkg/{sql/sem/builtins/pgcrypto/pgcryptocipher => ccl/pgcryptoccl/pgcryptocipherccl}/padding.go (77%) rename pkg/{sql/sem/builtins/pgcrypto/pgcryptocipher => ccl/pgcryptoccl/pgcryptocipherccl}/padding_test.go (88%) delete mode 100644 pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/doc.go diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 4c3cb15985eb..26e3b5d5b5f6 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -373,6 +373,7 @@ /pkg/ccl/multitenant/tenantcostserver/ @cockroachdb/sqlproxy-prs /pkg/ccl/oidcccl/ @cockroachdb/obs-inf-prs /pkg/ccl/partitionccl/ @cockroachdb/sql-foundations +/pkg/ccl/pgcryptoccl/ @cockroachdb/sql-foundations #!/pkg/ccl/serverccl/ @cockroachdb/unowned /pkg/ccl/serverccl/diagnosticsccl/ @cockroachdb/obs-inf-prs diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 68bc75bc9161..22aaf629fe34 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -66,6 +66,7 @@ ALL_TESTS = [ "//pkg/ccl/multitenantccl/tenantcostserver:tenantcostserver_test", "//pkg/ccl/oidcccl:oidcccl_test", "//pkg/ccl/partitionccl:partitionccl_test", + "//pkg/ccl/pgcryptoccl/pgcryptocipherccl:pgcryptocipherccl_test", "//pkg/ccl/schemachangerccl:schemachangerccl_test", "//pkg/ccl/serverccl/adminccl:adminccl_test", "//pkg/ccl/serverccl/diagnosticsccl:diagnosticsccl_test", @@ -538,7 +539,6 @@ ALL_TESTS = [ "//pkg/sql/schemachanger/screl:screl_test", "//pkg/sql/schemachanger/scrun:scrun_test", "//pkg/sql/schemachanger:schemachanger_test", - "//pkg/sql/sem/builtins/pgcrypto/pgcryptocipher:pgcryptocipher_test", "//pkg/sql/sem/builtins/pgformat:pgformat_test", "//pkg/sql/sem/builtins:builtins_disallowed_imports_test", "//pkg/sql/sem/builtins:builtins_test", @@ -869,6 +869,8 @@ GO_TARGETS = [ "//pkg/ccl/oidcccl:oidcccl_test", "//pkg/ccl/partitionccl:partitionccl", "//pkg/ccl/partitionccl:partitionccl_test", + "//pkg/ccl/pgcryptoccl/pgcryptocipherccl:pgcryptocipherccl", + "//pkg/ccl/pgcryptoccl/pgcryptocipherccl:pgcryptocipherccl_test", "//pkg/ccl/schemachangerccl:schemachangerccl", "//pkg/ccl/schemachangerccl:schemachangerccl_test", "//pkg/ccl/serverccl/adminccl:adminccl_test", @@ -2039,8 +2041,6 @@ GO_TARGETS = [ "//pkg/sql/sem/asof:asof", "//pkg/sql/sem/builtins/builtinconstants:builtinconstants", "//pkg/sql/sem/builtins/builtinsregistry:builtinsregistry", - "//pkg/sql/sem/builtins/pgcrypto/pgcryptocipher:pgcryptocipher", - "//pkg/sql/sem/builtins/pgcrypto/pgcryptocipher:pgcryptocipher_test", "//pkg/sql/sem/builtins/pgformat:pgformat", "//pkg/sql/sem/builtins/pgformat:pgformat_test", "//pkg/sql/sem/builtins:builtins", diff --git a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/BUILD.bazel b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/BUILD.bazel similarity index 73% rename from pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/BUILD.bazel rename to pkg/ccl/pgcryptoccl/pgcryptocipherccl/BUILD.bazel index 03de81848e6e..08028d31d0dc 100644 --- a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/BUILD.bazel +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/BUILD.bazel @@ -1,13 +1,13 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( - name = "pgcryptocipher", + name = "pgcryptocipherccl", srcs = [ "cipher_method.go", "doc.go", "padding.go", ], - importpath = "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher", + importpath = "github.com/cockroachdb/cockroach/pkg/ccl/pgcryptoccl/pgcryptocipherccl", visibility = ["//visibility:public"], deps = [ "//pkg/sql/pgwire/pgcode", @@ -18,13 +18,14 @@ go_library( ) go_test( - name = "pgcryptocipher_test", + name = "pgcryptocipherccl_test", srcs = [ "cipher_method_test.go", "padding_test.go", ], args = ["-test.timeout=295s"], - embed = [":pgcryptocipher"], + embed = [":pgcryptocipherccl"], + tags = ["ccl_test"], deps = [ "//pkg/util/leaktest", "@com_github_stretchr_testify//require", diff --git a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/cipher_method.go b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method.go similarity index 85% rename from pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/cipher_method.go rename to pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method.go index ff40c78c37e5..0910f282ae87 100644 --- a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/cipher_method.go +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method.go @@ -1,14 +1,12 @@ // Copyright 2023 The Cockroach Authors. // -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. +// 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 // -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package pgcryptocipher +package pgcryptocipherccl import ( "regexp" diff --git a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/cipher_method_test.go b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method_test.go similarity index 83% rename from pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/cipher_method_test.go rename to pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method_test.go index 4d73dbdb2e89..6e81acbf21a5 100644 --- a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/cipher_method_test.go +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method_test.go @@ -1,14 +1,12 @@ // Copyright 2023 The Cockroach Authors. // -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. +// 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 // -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package pgcryptocipher +package pgcryptocipherccl import ( "testing" diff --git a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/doc.go b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/doc.go new file mode 100644 index 000000000000..3de5cac51858 --- /dev/null +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/doc.go @@ -0,0 +1,11 @@ +// 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 pgcryptocipherccl contains the implementation of pgcrypto +// cipher functions. +package pgcryptocipherccl diff --git a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/padding.go b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/padding.go similarity index 77% rename from pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/padding.go rename to pkg/ccl/pgcryptoccl/pgcryptocipherccl/padding.go index 3bbc36271dd9..02ed2282a45e 100644 --- a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/padding.go +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/padding.go @@ -1,14 +1,12 @@ // Copyright 2023 The Cockroach Authors. // -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. +// 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 // -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package pgcryptocipher +package pgcryptocipherccl import ( "bytes" diff --git a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/padding_test.go b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/padding_test.go similarity index 88% rename from pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/padding_test.go rename to pkg/ccl/pgcryptoccl/pgcryptocipherccl/padding_test.go index c982d15141cc..7ce8c552585e 100644 --- a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/padding_test.go +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/padding_test.go @@ -1,14 +1,12 @@ // Copyright 2023 The Cockroach Authors. // -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. +// 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 // -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package pgcryptocipher +package pgcryptocipherccl import ( "testing" diff --git a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/doc.go b/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/doc.go deleted file mode 100644 index c5a2e30dd54a..000000000000 --- a/pkg/sql/sem/builtins/pgcrypto/pgcryptocipher/doc.go +++ /dev/null @@ -1,13 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -// Package pgcryptocipher contains the implementation of pgcrypto -// cipher functions. -package pgcryptocipher From c42a7a716e9c618418895d22aff7d08151917908 Mon Sep 17 00:00:00 2001 From: Andy Yang Date: Tue, 27 Jun 2023 01:45:09 -0400 Subject: [PATCH 2/4] pgcryptocipherccl: update error messages for invalid cipher method Release note: None --- pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method.go | 6 +++--- pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method_test.go | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method.go b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method.go index 0910f282ae87..dbda78c4403f 100644 --- a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method.go +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method.go @@ -59,7 +59,7 @@ func parseCipherMethod(s string) (cipherMethod, error) { case "bf": return cipherMethod{}, unimplemented.NewWithIssue(105466, "Blowfish is insecure and not supported") default: - return cipherMethod{}, pgerror.Newf(pgcode.InvalidParameterValue, `cipher method has unsupported algorithm: "%s"`, algorithm) + return cipherMethod{}, pgerror.Newf(pgcode.InvalidParameterValue, `cipher method has invalid algorithm: "%s"`, algorithm) } switch mode := submatches[cipherMethodRE.SubexpIndex("mode")]; strings.ToLower(mode) { @@ -67,7 +67,7 @@ func parseCipherMethod(s string) (cipherMethod, error) { case "ecb": return cipherMethod{}, unimplemented.NewWithIssue(105466, "ECB mode is insecure and not supported") default: - return cipherMethod{}, pgerror.Newf(pgcode.InvalidParameterValue, `cipher method has unsupported mode: "%s"`, mode) + return cipherMethod{}, pgerror.Newf(pgcode.InvalidParameterValue, `cipher method has invalid mode: "%s"`, mode) } switch padding := submatches[cipherMethodRE.SubexpIndex("padding")]; strings.ToLower(padding) { @@ -75,7 +75,7 @@ func parseCipherMethod(s string) (cipherMethod, error) { case "none": ret.padding = noPadding default: - return cipherMethod{}, pgerror.Newf(pgcode.InvalidParameterValue, `cipher method has unsupported padding: "%s"`, padding) + return cipherMethod{}, pgerror.Newf(pgcode.InvalidParameterValue, `cipher method has invalid padding: "%s"`, padding) } return ret, nil diff --git a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method_test.go b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method_test.go index 6e81acbf21a5..12ec8288a595 100644 --- a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method_test.go +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_method_test.go @@ -66,9 +66,9 @@ func TestParseCipherMethod(t *testing.T) { // Invalid values "aes/pad=pkcs": `cipher method has wrong format: "aes/pad=pkcs"`, - "aescbc": `cipher method has unsupported algorithm: "aescbc"`, - "aes-ctr": `cipher method has unsupported mode: "ctr"`, - "aes/pad:zero": `cipher method has unsupported padding: "zero"`, + "aescbc": `cipher method has invalid algorithm: "aescbc"`, + "aes-ctr": `cipher method has invalid mode: "ctr"`, + "aes/pad:zero": `cipher method has invalid padding: "zero"`, } { t.Run(input, func(t *testing.T) { _, err := parseCipherMethod(input) From a99de9f7764da1f6c5f530a5ee1d67f520d18b99 Mon Sep 17 00:00:00 2001 From: adityamaru Date: Wed, 30 Aug 2023 18:15:08 -0400 Subject: [PATCH 3/4] server,sql: add status server endpoint to request profiler details This change introduces a new status server endpoint to request job profiler details. This endpoint will redirect the request to the current coordinator node of the job in question. This will be useful because in a followup we will load the resumer from the coordinator node's job registry and trigger its specific job profiler detail collection logic. This is the first step of a few to move to a "fetch model" rather than have each resumer dump their execution details at some arbitrary cadence. The core logic involved in collecting profiler details has not changed, it has been moved in its entirety from pkg/sql to pkg/server. The `crdb_internal.request_job_execution_details` builtin now resolves the job's coordinator ID and calls the new status server endpoint. Informs: #109671 Release note: None --- docs/generated/http/full.md | 40 ++++ pkg/jobs/utils.go | 18 ++ pkg/server/BUILD.bazel | 6 + pkg/server/job_profiler.go | 175 ++++++++++++++++++ pkg/server/job_profiler_test.go | 133 +++++++++++++ pkg/server/serverpb/status.go | 1 + pkg/server/serverpb/status.proto | 12 ++ pkg/server/status.go | 12 +- pkg/server/tenant.go | 5 + pkg/server/testing_knobs.go | 4 + pkg/sql/BUILD.bazel | 1 - pkg/sql/jobs_profiler_execution_details.go | 109 +---------- .../jobs_profiler_execution_details_test.go | 29 +-- pkg/sql/sem/builtins/builtins.go | 6 +- pkg/sql/sem/eval/context.go | 5 +- 15 files changed, 430 insertions(+), 126 deletions(-) create mode 100644 pkg/server/job_profiler.go create mode 100644 pkg/server/job_profiler_test.go diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index ab0fec95efb4..793001d37454 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -5204,6 +5204,46 @@ Support status: [reserved](#support-status) +## RequestJobProfilerExecutionDetails + +`GET /_status/request_job_profiler_execution_details/{job_id}` + + + +Support status: [reserved](#support-status) + +#### Request Parameters + + + + + + + +| Field | Type | Label | Description | Support status | +| ----- | ---- | ----- | ----------- | -------------- | +| job_id | [int64](#cockroach.server.serverpb.RequestJobProfilerExecutionDetailsRequest-int64) | | | [reserved](#support-status) | + + + + + + + +#### Response Parameters + + + + + + + + + + + + + ## GetJobProfilerExecutionDetails `GET /_status/job_profiler_execution_details/{job_id}` diff --git a/pkg/jobs/utils.go b/pkg/jobs/utils.go index 6f7d894ce2a2..db59226ccf90 100644 --- a/pkg/jobs/utils.go +++ b/pkg/jobs/utils.go @@ -156,6 +156,24 @@ func JobExists( return row != nil, nil } +// JobCoordinatorID returns the coordinator node ID of the job. +func JobCoordinatorID( + ctx context.Context, jobID jobspb.JobID, txn *kv.Txn, ex isql.Executor, +) (int32, error) { + row, err := ex.QueryRow(ctx, "fetch-job-coordinator", txn, `SELECT claim_instance_id FROM system.jobs WHERE id = $1`, jobID) + if err != nil { + return 0, err + } + if row == nil { + return 0, errors.Errorf("coordinator not found for job %d", jobID) + } + coordinatorID, ok := tree.AsDInt(row[0]) + if !ok { + return 0, errors.AssertionFailedf("expected coordinator ID to be an int, got %T", row[0]) + } + return int32(coordinatorID), nil +} + // isJobTypeColumnDoesNotExistError returns true if the error is of the form // `column "job_type" does not exist`. func isJobTypeColumnDoesNotExistError(err error) bool { diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 79c8378af069..7427ffdce323 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -33,6 +33,7 @@ go_library( "init.go", "init_handshake.go", "initial_sql.go", + "job_profiler.go", "key_visualizer_server.go", "listen_and_update_addrs.go", "load_endpoint.go", @@ -336,6 +337,7 @@ go_library( "//pkg/util/tracing/tracingpb", "//pkg/util/tracing/tracingservicepb", "//pkg/util/tracing/tracingui", + "//pkg/util/tracing/zipper", "//pkg/util/uint128", "//pkg/util/uuid", "@com_github_cenkalti_backoff//:backoff", @@ -434,6 +436,7 @@ go_test( "index_usage_stats_test.go", "init_handshake_test.go", "intent_test.go", + "job_profiler_test.go", "load_endpoint_test.go", "main_test.go", "migration_test.go", @@ -478,10 +481,13 @@ go_test( "//pkg/base/serverident", "//pkg/build", "//pkg/cli/exit", + "//pkg/cloud/impl:cloudimpl", "//pkg/clusterversion", "//pkg/config", "//pkg/config/zonepb", "//pkg/gossip", + "//pkg/jobs", + "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvtenant", diff --git a/pkg/server/job_profiler.go b/pkg/server/job_profiler.go new file mode 100644 index 000000000000..f0f176f0957f --- /dev/null +++ b/pkg/server/job_profiler.go @@ -0,0 +1,175 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package server + +import ( + "context" + "fmt" + "strconv" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/server/srverrors" + "github.com/cockroachdb/cockroach/pkg/sql/isql" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing/zipper" + "github.com/cockroachdb/errors" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +// RequestJobProfilerExecutionDetails requests the profiler details for a job. +// This method ensures that the details are requested on the current coordinator +// node of the job to allow for the collection of Resumer specific details. +func (s *statusServer) RequestJobProfilerExecutionDetails( + ctx context.Context, req *serverpb.RequestJobProfilerExecutionDetailsRequest, +) (*serverpb.RequestJobProfilerExecutionDetailsResponse, error) { + ctx = s.AnnotateCtx(ctx) + // TODO(adityamaru): Figure out the correct privileges required to request execution details. + _, err := s.privilegeChecker.RequireAdminUser(ctx) + if err != nil { + return nil, err + } + + execCfg := s.sqlServer.execCfg + var coordinatorID int32 + err = execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + // Find the jobs' current coordinator node ID. + coordinatorID, err = jobs.JobCoordinatorID(ctx, jobspb.JobID(req.JobId), + txn, execCfg.InternalDB.Executor()) + return err + }) + if err != nil { + return nil, err + } + + nodeID, local, err := s.parseNodeID(strconv.Itoa(int(coordinatorID))) + if err != nil { + return nil, status.Errorf(codes.InvalidArgument, err.Error()) + } + + // If this node is the current coordinator of the job then we can collect the + // profiler details. + if local { + jobID := jobspb.JobID(req.JobId) + if !execCfg.Settings.Version.IsActive(ctx, clusterversion.V23_2) { + return nil, errors.Newf("execution details can only be requested on a cluster with version >= %s", + clusterversion.V23_2.String()) + } + e := makeJobProfilerExecutionDetailsBuilder(execCfg.SQLStatusServer, execCfg.InternalDB, jobID, execCfg.JobRegistry) + + // TODO(adityamaru): When we start collecting more information we can consider + // parallelize the collection of the various pieces. + e.addDistSQLDiagram(ctx) + e.addLabelledGoroutines(ctx) + e.addClusterWideTraces(ctx) + + // TODO(dt,adityamaru): add logic to reach out the registry and call resumer + // specific execution details collection logic. + + return &serverpb.RequestJobProfilerExecutionDetailsResponse{}, nil + } + + // Forward the request to the coordinator node + status, err := s.dialNode(ctx, nodeID) + if err != nil { + return nil, srverrors.ServerError(ctx, err) + } + return status.RequestJobProfilerExecutionDetails(ctx, req) +} + +// executionDetailsBuilder can be used to read and write execution details corresponding +// to a job. +type executionDetailsBuilder struct { + srv serverpb.SQLStatusServer + db isql.DB + jobID jobspb.JobID + registry *jobs.Registry +} + +// makeJobProfilerExecutionDetailsBuilder returns an instance of an executionDetailsBuilder. +func makeJobProfilerExecutionDetailsBuilder( + srv serverpb.SQLStatusServer, db isql.DB, jobID jobspb.JobID, registry *jobs.Registry, +) executionDetailsBuilder { + e := executionDetailsBuilder{ + srv: srv, db: db, jobID: jobID, registry: registry, + } + return e +} + +// addLabelledGoroutines collects and persists goroutines from all nodes in the +// cluster that have a pprof label tying it to the job whose execution details +// are being collected. +func (e *executionDetailsBuilder) addLabelledGoroutines(ctx context.Context) { + profileRequest := serverpb.ProfileRequest{ + NodeId: "all", + Type: serverpb.ProfileRequest_GOROUTINE, + Labels: true, + LabelFilter: fmt.Sprintf("%d", e.jobID), + } + resp, err := e.srv.Profile(ctx, &profileRequest) + if err != nil { + log.Errorf(ctx, "failed to collect goroutines for job %d: %v", e.jobID, err.Error()) + return + } + filename := fmt.Sprintf("goroutines.%s.txt", timeutil.Now().Format("20060102_150405.00")) + if err := jobs.WriteExecutionDetailFile(ctx, filename, resp.Data, e.db, e.jobID); err != nil { + log.Errorf(ctx, "failed to write goroutine for job %d: %v", e.jobID, err.Error()) + } +} + +// addDistSQLDiagram generates and persists a `distsql..html` file. +func (e *executionDetailsBuilder) addDistSQLDiagram(ctx context.Context) { + query := `SELECT plan_diagram FROM [SHOW JOB $1 WITH EXECUTION DETAILS]` + row, err := e.db.Executor().QueryRowEx(ctx, "profiler-bundler-add-diagram", nil, /* txn */ + sessiondata.NoSessionDataOverride, query, e.jobID) + if err != nil { + log.Errorf(ctx, "failed to write DistSQL diagram for job %d: %v", e.jobID, err.Error()) + return + } + if row != nil && row[0] != tree.DNull { + dspDiagramURL := string(tree.MustBeDString(row[0])) + filename := fmt.Sprintf("distsql.%s.html", timeutil.Now().Format("20060102_150405.00")) + if err := jobs.WriteExecutionDetailFile(ctx, filename, + []byte(fmt.Sprintf(``, dspDiagramURL)), + e.db, e.jobID); err != nil { + log.Errorf(ctx, "failed to write DistSQL diagram for job %d: %v", e.jobID, err.Error()) + } + } +} + +// addClusterWideTraces generates and persists a `trace..zip` file +// that captures the active tracing spans of a job on all nodes in the cluster. +func (e *executionDetailsBuilder) addClusterWideTraces(ctx context.Context) { + z := zipper.MakeInternalExecutorInflightTraceZipper(e.db.Executor()) + + traceID, err := jobs.GetJobTraceID(ctx, e.db, e.jobID) + if err != nil { + log.Warningf(ctx, "failed to fetch job trace ID: %+v", err.Error()) + return + } + zippedTrace, err := z.Zip(ctx, int64(traceID)) + if err != nil { + log.Errorf(ctx, "failed to collect cluster wide traces for job %d: %v", e.jobID, err.Error()) + return + } + + filename := fmt.Sprintf("trace.%s.zip", timeutil.Now().Format("20060102_150405.00")) + if err := jobs.WriteExecutionDetailFile(ctx, filename, zippedTrace, e.db, e.jobID); err != nil { + log.Errorf(ctx, "failed to write traces for job %d: %v", e.jobID, err.Error()) + } +} diff --git a/pkg/server/job_profiler_test.go b/pkg/server/job_profiler_test.go new file mode 100644 index 000000000000..63fbcb852eea --- /dev/null +++ b/pkg/server/job_profiler_test.go @@ -0,0 +1,133 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package server + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + _ "github.com/cockroachdb/cockroach/pkg/cloud/impl" // register ExternalStorage providers. + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +// fakeExecResumer calls optional callbacks during the job lifecycle. +type fakeExecResumer struct { + OnResume func(context.Context) error + FailOrCancel func(context.Context) error +} + +func (d fakeExecResumer) ForceRealSpan() bool { + return true +} + +func (d fakeExecResumer) DumpTraceAfterRun() bool { + return true +} + +var _ jobs.Resumer = fakeExecResumer{} +var _ jobs.TraceableJob = fakeExecResumer{} + +func (d fakeExecResumer) Resume(ctx context.Context, execCtx interface{}) error { + if d.OnResume != nil { + if err := d.OnResume(ctx); err != nil { + return err + } + } + return nil +} + +func (d fakeExecResumer) OnFailOrCancel(ctx context.Context, _ interface{}, _ error) error { + if d.FailOrCancel != nil { + return d.FailOrCancel(ctx) + } + return nil +} + +// TestJobExecutionDetailsRouting tests that the request job execution details +// endpoint redirects the request to the current coordinator node of the job. +func TestJobExecutionDetailsRouting(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + hasStartedCh := make(chan struct{}) + defer close(hasStartedCh) + jobs.RegisterConstructor(jobspb.TypeImport, func(j *jobs.Job, _ *cluster.Settings) jobs.Resumer { + return fakeExecResumer{ + OnResume: func(ctx context.Context) error { + hasStartedCh <- struct{}{} + return nil + }, + } + }, jobs.UsesTenantCostControl) + defer jobs.ResetConstructors()() + + dialedNodeID := roachpb.NodeID(-1) + ctx := context.Background() + tc := serverutils.StartCluster(t, 2, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + Server: &TestingKnobs{ + DialNodeCallback: func(ctx context.Context, nodeID roachpb.NodeID) error { + // We only care about the first call to dialNode. + if dialedNodeID == -1 { + dialedNodeID = nodeID + } + return nil + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + s := tc.Server(1) + sqlDB := s.SQLConn(t, "n1") + defer sqlDB.Close() + + _, err := sqlDB.Exec(`CREATE TABLE defaultdb.t (id INT)`) + require.NoError(t, err) + _, err = sqlDB.Exec(`INSERT INTO defaultdb.t SELECT generate_series(1, 100)`) + require.NoError(t, err) + var importJobID int + err = sqlDB.QueryRow(`IMPORT INTO defaultdb.t CSV DATA ('nodelocal://1/foo') WITH DETACHED`).Scan(&importJobID) + require.NoError(t, err) + <-hasStartedCh + + // Get the job's current coordinator ID. + var claimInstanceID int + err = sqlDB.QueryRow(`SELECT claim_instance_id FROM system.jobs WHERE id = $1`, importJobID).Scan(&claimInstanceID) + require.NoError(t, err) + + nonCoordinatorIDIdx := 0 + if claimInstanceID == 1 { + // We want to pick the non-coordinator node to send our request to. Idx is + // zero-indexed, so if the coordinator is node 1, we want to pick node 2. + nonCoordinatorIDIdx = 1 + } + nonCoordServer := tc.Server(nonCoordinatorIDIdx) + var resp serverpb.RequestJobProfilerExecutionDetailsResponse + path := fmt.Sprintf("/_status/request_job_profiler_execution_details/%d", importJobID) + err = serverutils.GetJSONProto(nonCoordServer, path, &resp) + require.NoError(t, err) + require.Equal(t, serverpb.RequestJobProfilerExecutionDetailsResponse{}, resp) + require.Equal(t, claimInstanceID, int(dialedNodeID)) +} diff --git a/pkg/server/serverpb/status.go b/pkg/server/serverpb/status.go index e33f0f863ce4..029c9647b694 100644 --- a/pkg/server/serverpb/status.go +++ b/pkg/server/serverpb/status.go @@ -46,6 +46,7 @@ type SQLStatusServer interface { LogFile(context.Context, *LogFileRequest) (*LogEntriesResponse, error) Logs(context.Context, *LogsRequest) (*LogEntriesResponse, error) NodesUI(context.Context, *NodesRequest) (*NodesResponseExternal, error) + RequestJobProfilerExecutionDetails(context.Context, *RequestJobProfilerExecutionDetailsRequest) (*RequestJobProfilerExecutionDetailsResponse, error) } // OptionalNodesStatusServer is a StatusServer that is only optionally present diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index d4682da316ef..4266c847bb28 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -2076,6 +2076,12 @@ message NetworkConnectivityResponse { ]; } +message RequestJobProfilerExecutionDetailsRequest { + int64 job_id = 1; +} + +message RequestJobProfilerExecutionDetailsResponse {} + message GetJobProfilerExecutionDetailRequest { int64 job_id = 1; string filename = 2; @@ -2563,6 +2569,12 @@ service Status { }; } + rpc RequestJobProfilerExecutionDetails(RequestJobProfilerExecutionDetailsRequest) returns(RequestJobProfilerExecutionDetailsResponse) { + option (google.api.http) = { + get: "/_status/request_job_profiler_execution_details/{job_id}" + }; + } + rpc GetJobProfilerExecutionDetails(GetJobProfilerExecutionDetailRequest) returns (GetJobProfilerExecutionDetailResponse) { option (google.api.http) = { diff --git a/pkg/server/status.go b/pkg/server/status.go index fc0f6975b80b..41e9e4e75969 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -479,6 +479,8 @@ type statusServer struct { // 256 concurrent queries actively running on a node, then it would // take 2^16 seconds (18 hours) to hit any one of them. cancelSemaphore *quotapool.IntPool + + knobs *TestingKnobs } // systemStatusServer is an extension of the standard @@ -565,6 +567,7 @@ func newStatusServer( internalExecutor *sql.InternalExecutor, serverIterator ServerIterator, clock *hlc.Clock, + knobs *TestingKnobs, ) *statusServer { ambient.AddLogTag("status", nil) if !rpcCtx.TenantID.IsSystem() { @@ -591,6 +594,7 @@ func newStatusServer( // See the docstring on cancelSemaphore for details about this initialization. cancelSemaphore: quotapool.NewIntPool("pgwire-cancel", 256), + knobs: knobs, } return server @@ -636,6 +640,7 @@ func newSystemStatusServer( internalExecutor, serverIterator, clock, + knobs, ) return &systemStatusServer{ @@ -686,6 +691,11 @@ func (s *statusServer) parseNodeID(nodeIDParam string) (roachpb.NodeID, bool, er func (s *statusServer) dialNode( ctx context.Context, nodeID roachpb.NodeID, ) (serverpb.StatusClient, error) { + if s.knobs != nil && s.knobs.DialNodeCallback != nil { + if err := s.knobs.DialNodeCallback(ctx, nodeID); err != nil { + return nil, err + } + } conn, err := s.serverIterator.dialNode(ctx, serverID(nodeID)) if err != nil { return nil, err @@ -1614,7 +1624,7 @@ func (s *statusServer) fetchProfileFromAllNodes( nodeFn := func(ctx context.Context, client interface{}, nodeID roachpb.NodeID) (interface{}, error) { statusClient := client.(serverpb.StatusClient) var pd *profData - err = timeutil.RunWithTimeout(ctx, opName, 1*time.Minute, func(ctx context.Context) error { + err := timeutil.RunWithTimeout(ctx, opName, 1*time.Minute, func(ctx context.Context) error { resp, err := statusClient.Profile(ctx, &serverpb.ProfileRequest{ NodeId: fmt.Sprintf("%d", nodeID), Type: req.Type, diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 36d6cb7e282f..3e05bbc79675 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -316,6 +316,10 @@ func newTenantServer( // construct the status server with a nil sqlServer, and then assign it once // an SQL server gets created. We are going to assume that the status server // won't require the SQL server object until later. + var serverKnobs TestingKnobs + if s, ok := baseCfg.TestingKnobs.Server.(*TestingKnobs); ok { + serverKnobs = *s + } sStatus := newStatusServer( baseCfg.AmbientCtx, baseCfg.Settings, @@ -331,6 +335,7 @@ func newTenantServer( args.circularInternalExecutor, serverIterator, args.clock, + &serverKnobs, ) args.sqlStatusServer = sStatus diff --git a/pkg/server/testing_knobs.go b/pkg/server/testing_knobs.go index 715e9ea51b67..9eaa7980c0b8 100644 --- a/pkg/server/testing_knobs.go +++ b/pkg/server/testing_knobs.go @@ -157,6 +157,10 @@ type TestingKnobs struct { // on a remote node in a cluster fan-out. It is invoked by the nodeFn argument // of server.iterateNodes. IterateNodesNodeCallback func(ctx context.Context, nodeID roachpb.NodeID) error + + // DialNodeCallback is used to mock dial errors when dialing a node. It is + // invoked by the dialNode method of server.serverIterator. + DialNodeCallback func(ctx context.Context, nodeID roachpb.NodeID) error } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 2dffffd1a009..d1a596add750 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -574,7 +574,6 @@ go_library( "//pkg/util/tracing", "//pkg/util/tracing/collector", "//pkg/util/tracing/tracingpb", - "//pkg/util/tracing/zipper", "//pkg/util/tsearch", "//pkg/util/uint128", "//pkg/util/uuid", diff --git a/pkg/sql/jobs_profiler_execution_details.go b/pkg/sql/jobs_profiler_execution_details.go index a5f931f5e896..d4e43b966f46 100644 --- a/pkg/sql/jobs_profiler_execution_details.go +++ b/pkg/sql/jobs_profiler_execution_details.go @@ -13,7 +13,6 @@ package sql import ( "context" gojson "encoding/json" - "fmt" "net/url" "strconv" @@ -26,11 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing/zipper" "github.com/cockroachdb/errors" ) @@ -182,103 +177,9 @@ func (p *planner) RequestExecutionDetailFiles(ctx context.Context, jobID jobspb. clusterversion.V23_2.String()) } - e := makeJobProfilerExecutionDetailsBuilder(execCfg.SQLStatusServer, execCfg.InternalDB, jobID) - - // Check if the job exists otherwise we can bail early. - exists, err := jobs.JobExists(ctx, jobID, p.Txn(), e.db.Executor()) - if err != nil { - return err - } - if !exists { - return errors.Newf("job %d not found; cannot request execution details", jobID) - } - - // TODO(adityamaru): When we start collecting more information we can consider - // parallelize the collection of the various pieces. - e.addDistSQLDiagram(ctx) - e.addLabelledGoroutines(ctx) - e.addClusterWideTraces(ctx) - - return nil -} - -// executionDetailsBuilder can be used to read and write execution details corresponding -// to a job. -type executionDetailsBuilder struct { - srv serverpb.SQLStatusServer - db isql.DB - jobID jobspb.JobID -} - -// makeJobProfilerExecutionDetailsBuilder returns an instance of an executionDetailsBuilder. -func makeJobProfilerExecutionDetailsBuilder( - srv serverpb.SQLStatusServer, db isql.DB, jobID jobspb.JobID, -) executionDetailsBuilder { - e := executionDetailsBuilder{ - srv: srv, db: db, jobID: jobID, - } - return e -} - -// addLabelledGoroutines collects and persists goroutines from all nodes in the -// cluster that have a pprof label tying it to the job whose execution details -// are being collected. -func (e *executionDetailsBuilder) addLabelledGoroutines(ctx context.Context) { - profileRequest := serverpb.ProfileRequest{ - NodeId: "all", - Type: serverpb.ProfileRequest_GOROUTINE, - Labels: true, - LabelFilter: fmt.Sprintf("%d", e.jobID), - } - resp, err := e.srv.Profile(ctx, &profileRequest) - if err != nil { - log.Errorf(ctx, "failed to collect goroutines for job %d: %v", e.jobID, err.Error()) - return - } - filename := fmt.Sprintf("goroutines.%s.txt", timeutil.Now().Format("20060102_150405.00")) - if err := jobs.WriteExecutionDetailFile(ctx, filename, resp.Data, e.db, e.jobID); err != nil { - log.Errorf(ctx, "failed to write goroutine for job %d: %v", e.jobID, err.Error()) - } -} - -// addDistSQLDiagram generates and persists a `distsql..html` file. -func (e *executionDetailsBuilder) addDistSQLDiagram(ctx context.Context) { - query := `SELECT plan_diagram FROM [SHOW JOB $1 WITH EXECUTION DETAILS]` - row, err := e.db.Executor().QueryRowEx(ctx, "profiler-bundler-add-diagram", nil, /* txn */ - sessiondata.NoSessionDataOverride, query, e.jobID) - if err != nil { - log.Errorf(ctx, "failed to write DistSQL diagram for job %d: %v", e.jobID, err.Error()) - return - } - if row != nil && row[0] != tree.DNull { - dspDiagramURL := string(tree.MustBeDString(row[0])) - filename := fmt.Sprintf("distsql.%s.html", timeutil.Now().Format("20060102_150405.00")) - if err := jobs.WriteExecutionDetailFile(ctx, filename, - []byte(fmt.Sprintf(``, dspDiagramURL)), - e.db, e.jobID); err != nil { - log.Errorf(ctx, "failed to write DistSQL diagram for job %d: %v", e.jobID, err.Error()) - } - } -} - -// addClusterWideTraces generates and persists a `trace..zip` file -// that captures the active tracing spans of a job on all nodes in the cluster. -func (e *executionDetailsBuilder) addClusterWideTraces(ctx context.Context) { - z := zipper.MakeInternalExecutorInflightTraceZipper(e.db.Executor()) - - traceID, err := jobs.GetJobTraceID(ctx, e.db, e.jobID) - if err != nil { - log.Warningf(ctx, "failed to fetch job trace ID: %+v", err.Error()) - return - } - zippedTrace, err := z.Zip(ctx, int64(traceID)) - if err != nil { - log.Errorf(ctx, "failed to collect cluster wide traces for job %d: %v", e.jobID, err.Error()) - return - } - - filename := fmt.Sprintf("trace.%s.zip", timeutil.Now().Format("20060102_150405.00")) - if err := jobs.WriteExecutionDetailFile(ctx, filename, zippedTrace, e.db, e.jobID); err != nil { - log.Errorf(ctx, "failed to write traces for job %d: %v", e.jobID, err.Error()) - } + _, err := execCfg.SQLStatusServer.RequestJobProfilerExecutionDetails(ctx, + &serverpb.RequestJobProfilerExecutionDetailsRequest{ + JobId: int64(jobID), + }) + return err } diff --git a/pkg/sql/jobs_profiler_execution_details_test.go b/pkg/sql/jobs_profiler_execution_details_test.go index 1241dc322fad..eb6c4f62936e 100644 --- a/pkg/sql/jobs_profiler_execution_details_test.go +++ b/pkg/sql/jobs_profiler_execution_details_test.go @@ -107,9 +107,7 @@ func checkForPlanDiagrams( }) } -// TestJobsExecutionDetails tests that a job's execution details are retrieved -// and rendered correctly. -func TestJobsExecutionDetails(t *testing.T) { +func TestShowJobsWithExecutionDetails(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -224,7 +222,7 @@ func TestReadWriteProfilerExecutionDetails(t *testing.T) { }) t.Run("execution details for invalid job ID", func(t *testing.T) { - runner.ExpectErr(t, `job -123 not found; cannot request execution details`, `SELECT crdb_internal.request_job_execution_details(-123)`) + runner.ExpectErr(t, `coordinator not found for job -123`, `SELECT crdb_internal.request_job_execution_details(-123)`) }) t.Run("read/write terminal trace", func(t *testing.T) { @@ -321,6 +319,10 @@ func TestListProfilerExecutionDetails(t *testing.T) { execCfg := s.ExecutorConfig().(sql.ExecutorConfig) expectedDiagrams := 1 + writtenDiagram := make(chan struct{}) + defer close(writtenDiagram) + continueCh := make(chan struct{}) + defer close(continueCh) jobs.RegisterConstructor(jobspb.TypeImport, func(j *jobs.Job, _ *cluster.Settings) jobs.Resumer { return fakeExecResumer{ OnResume: func(ctx context.Context) error { @@ -329,6 +331,8 @@ func TestListProfilerExecutionDetails(t *testing.T) { p.PhysicalInfrastructure = infra jobsprofiler.StorePlanDiagram(ctx, s.Stopper(), &p, s.InternalDB().(isql.DB), j.ID()) checkForPlanDiagrams(ctx, t, s.InternalDB().(isql.DB), j.ID(), expectedDiagrams) + writtenDiagram <- struct{}{} + <-continueCh if err := execCfg.JobRegistry.CheckPausepoint("fakeresumer.pause"); err != nil { return err } @@ -344,22 +348,25 @@ func TestListProfilerExecutionDetails(t *testing.T) { runner.Exec(t, `SET CLUSTER SETTING jobs.debug.pausepoints = 'fakeresumer.pause'`) var importJobID int runner.QueryRow(t, `IMPORT INTO t CSV DATA ('nodelocal://1/foo') WITH DETACHED`).Scan(&importJobID) - jobutils.WaitForJobToPause(t, runner, jobspb.JobID(importJobID)) + <-writtenDiagram runner.Exec(t, `SELECT crdb_internal.request_job_execution_details($1)`, importJobID) files := listExecutionDetails(t, s, jobspb.JobID(importJobID)) - require.Len(t, files, 5) - require.Regexp(t, "[0-9]/resumer-trace/.*~cockroach\\.sql\\.jobs\\.jobspb\\.TraceData\\.binpb", files[0]) - require.Regexp(t, "[0-9]/resumer-trace/.*~cockroach\\.sql\\.jobs\\.jobspb\\.TraceData\\.binpb", files[1]) - require.Regexp(t, "distsql\\..*\\.html", files[2]) - require.Regexp(t, "goroutines\\..*\\.txt", files[3]) - require.Regexp(t, "trace\\..*\\.zip", files[4]) + require.Len(t, files, 3) + require.Regexp(t, "distsql\\..*\\.html", files[0]) + require.Regexp(t, "goroutines\\..*\\.txt", files[1]) + require.Regexp(t, "trace\\..*\\.zip", files[2]) + + continueCh <- struct{}{} + jobutils.WaitForJobToPause(t, runner, jobspb.JobID(importJobID)) // Resume the job, so it can write another DistSQL diagram and goroutine // snapshot. runner.Exec(t, `SET CLUSTER SETTING jobs.debug.pausepoints = ''`) expectedDiagrams = 2 runner.Exec(t, `RESUME JOB $1`, importJobID) + <-writtenDiagram + continueCh <- struct{}{} jobutils.WaitForJobToSucceed(t, runner, jobspb.JobID(importJobID)) runner.Exec(t, `SELECT crdb_internal.request_job_execution_details($1)`, importJobID) files = listExecutionDetails(t, s, jobspb.JobID(importJobID)) diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index c0b172988521..31b81c15ef12 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -7786,16 +7786,12 @@ specified store on the node it's run from. One of 'mvccGC', 'merge', 'split', if err != nil { return nil, err } - if !isAdmin { return nil, errors.New("must be admin to request a job profiler bundle") } jobID := int(tree.MustBeDInt(args[0])) - if err := evalCtx.JobsProfiler.RequestExecutionDetailFiles( - ctx, - jobspb.JobID(jobID), - ); err != nil { + if err := evalCtx.JobsProfiler.RequestExecutionDetailFiles(ctx, jobspb.JobID(jobID)); err != nil { return nil, err } diff --git a/pkg/sql/sem/eval/context.go b/pkg/sql/sem/eval/context.go index a228beb2e9d1..bbd6dde2a0d8 100644 --- a/pkg/sql/sem/eval/context.go +++ b/pkg/sql/sem/eval/context.go @@ -296,10 +296,7 @@ type JobsProfiler interface { GenerateExecutionDetailsJSON(ctx context.Context, evalCtx *Context, jobID jobspb.JobID) ([]byte, error) // RequestExecutionDetailFiles triggers the collection of execution details - // for the specified jobID that are then persisted to `system.job_info`. This - // currently includes the following pieces of information: - // - // - Latest DistSQL diagram of the job + // for the specified jobID that are then persisted to `system.job_info`. RequestExecutionDetailFiles(ctx context.Context, jobID jobspb.JobID) error } From d639116adbc6b6b6aa206d69d7f0272c1e720875 Mon Sep 17 00:00:00 2001 From: Andy Yang Date: Tue, 23 May 2023 21:05:34 -0400 Subject: [PATCH 4/4] builtins: implement encrypt and decrypt pgcrypto functions This patch implements `encrypt`, `encrypt_iv`, `decrypt`, and `decrypt_iv` from pgcrypto. These functions require an enterprise license on a CCL distribution. Release note (enterprise change): The pgcrypto functions `encrypt`, `encrypt_iv`, `decrypt`, and `decrypt_iv` are now implemented. These functions require an enterprise license on a CCL distribution. --- docs/generated/sql/functions.md | 36 +++++ pkg/BUILD.bazel | 4 + pkg/ccl/BUILD.bazel | 1 + pkg/ccl/ccl_init.go | 1 + .../testdata/logic_test/pgcrypto_builtins | 136 ++++++++++++++++ .../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-22.2-23.1/BUILD.bazel | 2 +- .../local-mixed-22.2-23.1/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/pgcryptoccl/BUILD.bazel | 38 +++++ pkg/ccl/pgcryptoccl/main_test.go | 31 ++++ pkg/ccl/pgcryptoccl/pgcryptoccl.go | 69 ++++++++ pkg/ccl/pgcryptoccl/pgcryptoccl_test.go | 60 +++++++ .../pgcryptoccl/pgcryptocipherccl/BUILD.bazel | 2 + .../pgcryptoccl/pgcryptocipherccl/cipher.go | 153 ++++++++++++++++++ .../pgcryptocipherccl/cipher_test.go | 129 +++++++++++++++ .../pgcryptoccl/pgcryptocipherccl/padding.go | 10 +- .../testdata/logic_test/pgcrypto_builtins | 20 +++ pkg/sql/sem/builtins/BUILD.bazel | 1 + pkg/sql/sem/builtins/fixed_oids.go | 4 + pkg/sql/sem/builtins/pgcrypto/BUILD.bazel | 13 ++ pkg/sql/sem/builtins/pgcrypto/pgcrypto.go | 47 ++++++ pkg/sql/sem/builtins/pgcrypto_builtins.go | 118 ++++++++++++++ 34 files changed, 933 insertions(+), 10 deletions(-) create mode 100644 pkg/ccl/logictestccl/testdata/logic_test/pgcrypto_builtins create mode 100644 pkg/ccl/pgcryptoccl/BUILD.bazel create mode 100644 pkg/ccl/pgcryptoccl/main_test.go create mode 100644 pkg/ccl/pgcryptoccl/pgcryptoccl.go create mode 100644 pkg/ccl/pgcryptoccl/pgcryptoccl_test.go create mode 100644 pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher.go create mode 100644 pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_test.go create mode 100644 pkg/sql/sem/builtins/pgcrypto/BUILD.bazel create mode 100644 pkg/sql/sem/builtins/pgcrypto/pgcrypto.go diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 31147c9f2f1c..1d4fb35607ed 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -388,10 +388,46 @@ crypt(password: string, salt: string) → string

Generates a hash based on a password and salt. The hash algorithm and number of rounds if applicable are encoded in the salt.

Immutable +decrypt(data: bytes, key: bytes, type: string) → bytes

Decrypt data with key using the cipher method specified by type.

+

The cipher type must have the format <algorithm>[-<mode>][/pad:<padding>] where:

+
    +
  • <algorithm> is aes
  • +
  • <mode> is cbc (default)
  • +
  • <padding> is pkcs (default) or none
  • +
+

This function requires an enterprise license on a CCL distribution.

+
Immutable +decrypt_iv(data: bytes, key: bytes, iv: bytes, type: string) → bytes

Decrypt data with key using the cipher method specified by type. If the mode is CBC, the provided iv will be used. Otherwise, it will be ignored.

+

The cipher type must have the format <algorithm>[-<mode>][/pad:<padding>] where:

+
    +
  • <algorithm> is aes
  • +
  • <mode> is cbc (default)
  • +
  • <padding> is pkcs (default) or none
  • +
+

This function requires an enterprise license on a CCL distribution.

+
Immutable digest(data: bytes, type: string) → bytes

Computes a binary hash of the given data. type is the algorithm to use (md5, sha1, sha224, sha256, sha384, or sha512).

Immutable digest(data: string, type: string) → bytes

Computes a binary hash of the given data. type is the algorithm to use (md5, sha1, sha224, sha256, sha384, or sha512).

Immutable +encrypt(data: bytes, key: bytes, type: string) → bytes

Encrypt data with key using the cipher method specified by type.

+

The cipher type must have the format <algorithm>[-<mode>][/pad:<padding>] where:

+
    +
  • <algorithm> is aes
  • +
  • <mode> is cbc (default)
  • +
  • <padding> is pkcs (default) or none
  • +
+

This function requires an enterprise license on a CCL distribution.

+
Immutable +encrypt_iv(data: bytes, key: bytes, iv: bytes, type: string) → bytes

Encrypt data with key using the cipher method specified by type. If the mode is CBC, the provided iv will be used. Otherwise, it will be ignored.

+

The cipher type must have the format <algorithm>[-<mode>][/pad:<padding>] where:

+
    +
  • <algorithm> is aes
  • +
  • <mode> is cbc (default)
  • +
  • <padding> is pkcs (default) or none
  • +
+

This function requires an enterprise license on a CCL distribution.

+
Immutable gen_salt(type: string) → string

Generates a salt for input into the crypt function using the default number of rounds.

Volatile gen_salt(type: string, iter_count: int) → string

Generates a salt for input into the crypt function using iter_count number of rounds.

diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 22aaf629fe34..019991a9508a 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -67,6 +67,7 @@ ALL_TESTS = [ "//pkg/ccl/oidcccl:oidcccl_test", "//pkg/ccl/partitionccl:partitionccl_test", "//pkg/ccl/pgcryptoccl/pgcryptocipherccl:pgcryptocipherccl_test", + "//pkg/ccl/pgcryptoccl:pgcryptoccl_test", "//pkg/ccl/schemachangerccl:schemachangerccl_test", "//pkg/ccl/serverccl/adminccl:adminccl_test", "//pkg/ccl/serverccl/diagnosticsccl:diagnosticsccl_test", @@ -871,6 +872,8 @@ GO_TARGETS = [ "//pkg/ccl/partitionccl:partitionccl_test", "//pkg/ccl/pgcryptoccl/pgcryptocipherccl:pgcryptocipherccl", "//pkg/ccl/pgcryptoccl/pgcryptocipherccl:pgcryptocipherccl_test", + "//pkg/ccl/pgcryptoccl:pgcryptoccl", + "//pkg/ccl/pgcryptoccl:pgcryptoccl_test", "//pkg/ccl/schemachangerccl:schemachangerccl", "//pkg/ccl/schemachangerccl:schemachangerccl_test", "//pkg/ccl/serverccl/adminccl:adminccl_test", @@ -2041,6 +2044,7 @@ GO_TARGETS = [ "//pkg/sql/sem/asof:asof", "//pkg/sql/sem/builtins/builtinconstants:builtinconstants", "//pkg/sql/sem/builtins/builtinsregistry:builtinsregistry", + "//pkg/sql/sem/builtins/pgcrypto:pgcrypto", "//pkg/sql/sem/builtins/pgformat:pgformat", "//pkg/sql/sem/builtins/pgformat:pgformat_test", "//pkg/sql/sem/builtins:builtins", diff --git a/pkg/ccl/BUILD.bazel b/pkg/ccl/BUILD.bazel index 8664f2b167f6..b86474e903d9 100644 --- a/pkg/ccl/BUILD.bazel +++ b/pkg/ccl/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/ccl/multitenantccl", "//pkg/ccl/oidcccl", "//pkg/ccl/partitionccl", + "//pkg/ccl/pgcryptoccl", "//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 f266b2ec31d3..4a224d5cd10c 100644 --- a/pkg/ccl/ccl_init.go +++ b/pkg/ccl/ccl_init.go @@ -26,6 +26,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/ccl/multitenantccl" _ "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/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/pgcrypto_builtins b/pkg/ccl/logictestccl/testdata/logic_test/pgcrypto_builtins new file mode 100644 index 000000000000..37c40273eead --- /dev/null +++ b/pkg/ccl/logictestccl/testdata/logic_test/pgcrypto_builtins @@ -0,0 +1,136 @@ +subtest encrypt_decrypt_aes_128 + +query T +SELECT encrypt('abc', '16_byte_long_key', 'aes')::STRING +---- +\x0026cd6206cfd92140883b75c098d613 + +query T +SELECT decrypt('\x0026cd6206cfd92140883b75c098d613', '16_byte_long_key', 'aes') +---- +abc + +subtest end + +subtest encrypt_decrypt_aes_192 + +query T +SELECT encrypt('abc', '24_byte_looooooooong_key', 'aes')::STRING +---- +\x6c42e2269a65d605ecd98b2aeb8eb4e9 + +query T +SELECT decrypt('\x6c42e2269a65d605ecd98b2aeb8eb4e9', '24_byte_looooooooong_key', 'aes') +---- +abc + +subtest end + +subtest encrypt_decrypt_aes_256 + +query T +SELECT encrypt('abc', '32_byte_looooooooooooooooong_key', 'aes')::STRING +---- +\xb368f7d6adcd73633dc37696b068cfda + +query T +SELECT decrypt('\xb368f7d6adcd73633dc37696b068cfda', '32_byte_looooooooooooooooong_key', 'aes') +---- +abc + +subtest end + +subtest encrypt_decrypt_aes_multi_block_data + +query T +SELECT encrypt('abcdefghijklmnopqrstuvwxyz', 'key', 'aes')::STRING +---- +\x4649e8618af65b2b50aa73ec9cfc102c95fcbbaf04cb8a82333e493dc97060f3 + +query T +SELECT decrypt('\x4649e8618af65b2b50aa73ec9cfc102c95fcbbaf04cb8a82333e493dc97060f3', 'key', 'aes') +---- +abcdefghijklmnopqrstuvwxyz + +subtest end + +subtest encrypt_decrypt_aes_no_padding + +query T +SELECT encrypt('16byte_long_data', 'key', 'aes/pad:none')::STRING +---- +\x043db9c657e2a2cd693b4239a3d8a1cb + +query T +SELECT decrypt('\x043db9c657e2a2cd693b4239a3d8a1cb', 'key', 'aes/pad:none') +---- +16byte_long_data + +subtest end + +subtest encrypt_decrypt_iv_aes + +query T +SELECT encrypt_iv('abc', 'key', '123', 'aes')::STRING +---- +\x91b4ef63852013c8da53829da662b871 + +query T +SELECT decrypt_iv('\x91b4ef63852013c8da53829da662b871', 'key', '123', 'aes') +---- +abc + +subtest end + +subtest encrypt_error + +query error pgcode 0A000 Blowfish is insecure and not supported +SELECT encrypt('abc', 'key', 'bf') + +query error pgcode 0A000 ECB mode is insecure and not supported +SELECT encrypt('abc', 'key', 'aes-ecb') + +query error pgcode 22023 cipher method has wrong format: "aes/pad=pkcs" +SELECT encrypt('abc', 'key', 'aes/pad=pkcs') + +query error pgcode 22023 cipher method has invalid algorithm: "fakealgo" +SELECT encrypt('abc', 'key', 'fakealgo') + +query error pgcode 22023 cipher method has invalid mode: "ctr" +SELECT encrypt('abc', 'key', 'aes-ctr') + +query error pgcode 22023 cipher method has invalid padding: "zero" +SELECT encrypt('abc', 'key', 'aes/pad:zero') + +query error pgcode 22023 data has length 3, which is not a multiple of block size 16 +SELECT encrypt('abc', 'key', 'aes/pad:none') + +subtest end + +subtest decrypt_error + +query error pgcode 0A000 Blowfish is insecure and not supported +SELECT decrypt('abc', 'key', 'bf') + +query error pgcode 0A000 ECB mode is insecure and not supported +SELECT decrypt('abc', 'key', 'aes-ecb') + +query error pgcode 22023 cipher method has wrong format: "aes/pad=pkcs" +SELECT decrypt('abc', 'key', 'aes/pad=pkcs') + +query error pgcode 22023 cipher method has invalid algorithm: "fakealgo" +SELECT decrypt('abc', 'key', 'fakealgo') + +query error pgcode 22023 cipher method has invalid mode: "ctr" +SELECT decrypt('abc', 'key', 'aes-ctr') + +query error pgcode 22023 cipher method has invalid padding: "zero" +SELECT decrypt('abc', 'key', 'aes/pad:zero') + +query error pgcode 22023 data has length 3, which is not a multiple of block size 16 +SELECT decrypt('abc', 'key', 'aes') + +query error pgcode 22023 data has length 3, which is not a multiple of block size 16 +SELECT decrypt('abc', 'key', 'aes/pad:none') + +subtest end diff --git a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go index 96e5db025e1a..c1002b805cad 100644 --- a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go +++ b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go @@ -2515,6 +2515,13 @@ func TestTenantLogicCCL_partitioning_enum( runCCLLogicTest(t, "partitioning_enum") } +func TestTenantLogicCCL_pgcrypto_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runCCLLogicTest(t, "pgcrypto_builtins") +} + func TestTenantLogicCCL_redact_descriptor( t *testing.T, ) { diff --git a/pkg/ccl/logictestccl/tests/fakedist-disk/BUILD.bazel b/pkg/ccl/logictestccl/tests/fakedist-disk/BUILD.bazel index c29772179ee6..a1d3e4458c89 100644 --- a/pkg/ccl/logictestccl/tests/fakedist-disk/BUILD.bazel +++ b/pkg/ccl/logictestccl/tests/fakedist-disk/BUILD.bazel @@ -15,7 +15,7 @@ go_test( exec_properties = { "Pool": "large", }, - shard_count = 5, + shard_count = 6, 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 bec6322f598d..ed15a067a5b3 100644 --- a/pkg/ccl/logictestccl/tests/fakedist-disk/generated_test.go +++ b/pkg/ccl/logictestccl/tests/fakedist-disk/generated_test.go @@ -92,6 +92,13 @@ func TestCCLLogic_partitioning_enum( runCCLLogicTest(t, "partitioning_enum") } +func TestCCLLogic_pgcrypto_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runCCLLogicTest(t, "pgcrypto_builtins") +} + func TestCCLLogic_redact_descriptor( 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 0c31c6a344e6..81a39eea27ed 100644 --- a/pkg/ccl/logictestccl/tests/fakedist-vec-off/BUILD.bazel +++ b/pkg/ccl/logictestccl/tests/fakedist-vec-off/BUILD.bazel @@ -15,7 +15,7 @@ go_test( exec_properties = { "Pool": "large", }, - shard_count = 5, + shard_count = 6, 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 b19cc51d7498..662396114099 100644 --- a/pkg/ccl/logictestccl/tests/fakedist-vec-off/generated_test.go +++ b/pkg/ccl/logictestccl/tests/fakedist-vec-off/generated_test.go @@ -92,6 +92,13 @@ func TestCCLLogic_partitioning_enum( runCCLLogicTest(t, "partitioning_enum") } +func TestCCLLogic_pgcrypto_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runCCLLogicTest(t, "pgcrypto_builtins") +} + func TestCCLLogic_redact_descriptor( t *testing.T, ) { diff --git a/pkg/ccl/logictestccl/tests/fakedist/BUILD.bazel b/pkg/ccl/logictestccl/tests/fakedist/BUILD.bazel index 6eaa8c258a4e..afccaf838f1d 100644 --- a/pkg/ccl/logictestccl/tests/fakedist/BUILD.bazel +++ b/pkg/ccl/logictestccl/tests/fakedist/BUILD.bazel @@ -15,7 +15,7 @@ go_test( exec_properties = { "Pool": "large", }, - shard_count = 6, + shard_count = 7, 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 edd36915ffbf..4781d36ae881 100644 --- a/pkg/ccl/logictestccl/tests/fakedist/generated_test.go +++ b/pkg/ccl/logictestccl/tests/fakedist/generated_test.go @@ -99,6 +99,13 @@ func TestCCLLogic_partitioning_enum( runCCLLogicTest(t, "partitioning_enum") } +func TestCCLLogic_pgcrypto_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runCCLLogicTest(t, "pgcrypto_builtins") +} + func TestCCLLogic_redact_descriptor( 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 95fd285ad9dc..70830210bbe7 100644 --- a/pkg/ccl/logictestccl/tests/local-legacy-schema-changer/BUILD.bazel +++ b/pkg/ccl/logictestccl/tests/local-legacy-schema-changer/BUILD.bazel @@ -15,7 +15,7 @@ go_test( exec_properties = { "Pool": "large", }, - shard_count = 5, + shard_count = 6, 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 c9c128ef12b0..74fb8bc7ccb8 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 @@ -92,6 +92,13 @@ func TestCCLLogic_partitioning_enum( runCCLLogicTest(t, "partitioning_enum") } +func TestCCLLogic_pgcrypto_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runCCLLogicTest(t, "pgcrypto_builtins") +} + func TestCCLLogic_redact_descriptor( t *testing.T, ) { diff --git a/pkg/ccl/logictestccl/tests/local-mixed-22.2-23.1/BUILD.bazel b/pkg/ccl/logictestccl/tests/local-mixed-22.2-23.1/BUILD.bazel index 27609dc54134..2f058fbe96f7 100644 --- a/pkg/ccl/logictestccl/tests/local-mixed-22.2-23.1/BUILD.bazel +++ b/pkg/ccl/logictestccl/tests/local-mixed-22.2-23.1/BUILD.bazel @@ -15,7 +15,7 @@ go_test( exec_properties = { "Pool": "large", }, - shard_count = 6, + shard_count = 7, tags = [ "ccl_test", "cpu:1", diff --git a/pkg/ccl/logictestccl/tests/local-mixed-22.2-23.1/generated_test.go b/pkg/ccl/logictestccl/tests/local-mixed-22.2-23.1/generated_test.go index 94295f92a63f..071231559fe3 100644 --- a/pkg/ccl/logictestccl/tests/local-mixed-22.2-23.1/generated_test.go +++ b/pkg/ccl/logictestccl/tests/local-mixed-22.2-23.1/generated_test.go @@ -99,6 +99,13 @@ func TestCCLLogic_partitioning_enum( runCCLLogicTest(t, "partitioning_enum") } +func TestCCLLogic_pgcrypto_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runCCLLogicTest(t, "pgcrypto_builtins") +} + func TestCCLLogic_redact_descriptor( 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 c8cedae0a43b..c2be802cf341 100644 --- a/pkg/ccl/logictestccl/tests/local-vec-off/BUILD.bazel +++ b/pkg/ccl/logictestccl/tests/local-vec-off/BUILD.bazel @@ -15,7 +15,7 @@ go_test( exec_properties = { "Pool": "large", }, - shard_count = 5, + shard_count = 6, 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 ecbe0cd32845..eae4fccbd6fb 100644 --- a/pkg/ccl/logictestccl/tests/local-vec-off/generated_test.go +++ b/pkg/ccl/logictestccl/tests/local-vec-off/generated_test.go @@ -92,6 +92,13 @@ func TestCCLLogic_partitioning_enum( runCCLLogicTest(t, "partitioning_enum") } +func TestCCLLogic_pgcrypto_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runCCLLogicTest(t, "pgcrypto_builtins") +} + func TestCCLLogic_redact_descriptor( t *testing.T, ) { diff --git a/pkg/ccl/logictestccl/tests/local/BUILD.bazel b/pkg/ccl/logictestccl/tests/local/BUILD.bazel index 657b45774fac..0f8a15b77c95 100644 --- a/pkg/ccl/logictestccl/tests/local/BUILD.bazel +++ b/pkg/ccl/logictestccl/tests/local/BUILD.bazel @@ -15,7 +15,7 @@ go_test( exec_properties = { "Pool": "large", }, - shard_count = 19, + shard_count = 20, 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 c5917fc1efdf..dd66a1be3062 100644 --- a/pkg/ccl/logictestccl/tests/local/generated_test.go +++ b/pkg/ccl/logictestccl/tests/local/generated_test.go @@ -169,6 +169,13 @@ func TestCCLLogic_partitioning_index( runCCLLogicTest(t, "partitioning_index") } +func TestCCLLogic_pgcrypto_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runCCLLogicTest(t, "pgcrypto_builtins") +} + func TestCCLLogic_redact_descriptor( t *testing.T, ) { diff --git a/pkg/ccl/pgcryptoccl/BUILD.bazel b/pkg/ccl/pgcryptoccl/BUILD.bazel new file mode 100644 index 000000000000..07993ab2f685 --- /dev/null +++ b/pkg/ccl/pgcryptoccl/BUILD.bazel @@ -0,0 +1,38 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "pgcryptoccl", + srcs = ["pgcryptoccl.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/ccl/pgcryptoccl", + visibility = ["//visibility:public"], + deps = [ + "//pkg/ccl/pgcryptoccl/pgcryptocipherccl", + "//pkg/ccl/utilccl", + "//pkg/sql/sem/builtins/pgcrypto", + "//pkg/sql/sem/eval", + ], +) + +go_test( + name = "pgcryptoccl_test", + srcs = [ + "main_test.go", + "pgcryptoccl_test.go", + ], + args = ["-test.timeout=295s"], + tags = ["ccl_test"], + deps = [ + "//pkg/base", + "//pkg/ccl", + "//pkg/security/securityassets", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/randutil", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/ccl/pgcryptoccl/main_test.go b/pkg/ccl/pgcryptoccl/main_test.go new file mode 100644 index 000000000000..205ff2318720 --- /dev/null +++ b/pkg/ccl/pgcryptoccl/main_test.go @@ -0,0 +1,31 @@ +// 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 pgcryptoccl_test + +import ( + "os" + "testing" + + "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" +) + +//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go + +func TestMain(m *testing.M) { + securityassets.SetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/ccl/pgcryptoccl/pgcryptoccl.go b/pkg/ccl/pgcryptoccl/pgcryptoccl.go new file mode 100644 index 000000000000..5c5d31b1ad5a --- /dev/null +++ b/pkg/ccl/pgcryptoccl/pgcryptoccl.go @@ -0,0 +1,69 @@ +// 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 pgcryptoccl + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/ccl/pgcryptoccl/pgcryptocipherccl" + "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/pgcrypto" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" +) + +func init() { + pgcrypto.Decrypt = decrypt + pgcrypto.DecryptIV = decryptIV + pgcrypto.Encrypt = encrypt + pgcrypto.EncryptIV = encryptIV +} + +func checkEnterpriseEnabledForCipherFunctions(evalCtx *eval.Context) error { + return utilccl.CheckEnterpriseEnabled( + evalCtx.Settings, + evalCtx.ClusterID, + "pgcrypto cipher functions", + ) +} + +func decrypt( + _ context.Context, evalCtx *eval.Context, data []byte, key []byte, cipherType string, +) ([]byte, error) { + if err := checkEnterpriseEnabledForCipherFunctions(evalCtx); err != nil { + return nil, err + } + return pgcryptocipherccl.Decrypt(data, key, nil /* iv */, cipherType) +} + +func decryptIV( + _ context.Context, evalCtx *eval.Context, data []byte, key []byte, iv []byte, cipherType string, +) ([]byte, error) { + if err := checkEnterpriseEnabledForCipherFunctions(evalCtx); err != nil { + return nil, err + } + return pgcryptocipherccl.Decrypt(data, key, iv, cipherType) +} + +func encrypt( + _ context.Context, evalCtx *eval.Context, data []byte, key []byte, cipherType string, +) ([]byte, error) { + if err := checkEnterpriseEnabledForCipherFunctions(evalCtx); err != nil { + return nil, err + } + return pgcryptocipherccl.Encrypt(data, key, nil /* iv */, cipherType) +} + +func encryptIV( + _ context.Context, evalCtx *eval.Context, data []byte, key []byte, iv []byte, cipherType string, +) ([]byte, error) { + if err := checkEnterpriseEnabledForCipherFunctions(evalCtx); err != nil { + return nil, err + } + return pgcryptocipherccl.Encrypt(data, key, iv, cipherType) +} diff --git a/pkg/ccl/pgcryptoccl/pgcryptoccl_test.go b/pkg/ccl/pgcryptoccl/pgcryptoccl_test.go new file mode 100644 index 000000000000..f709f74874be --- /dev/null +++ b/pkg/ccl/pgcryptoccl/pgcryptoccl_test.go @@ -0,0 +1,60 @@ +// 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 pgcryptoccl_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/ccl" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +func TestCipherFunctionEnterpriseLicense(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + + db := tc.ServerConn(0) + defer db.Close() + + for name, query := range map[string]string{ + "decrypt": `SELECT decrypt('\xdb5f149a7caf0cd275ca18c203a212c9', 'key', 'aes')`, + "decrypt_iv": `SELECT decrypt_iv('\x91b4ef63852013c8da53829da662b871', 'key', '123', 'aes')`, + "encrypt": `SELECT encrypt('abc', 'key', 'aes')`, + "encrypt_iv": `SELECT encrypt_iv('abc', 'key', '123', 'aes')`, + } { + t.Run(name, func(t *testing.T) { + testutils.RunTrueAndFalse(t, "enterprise_license", func(t *testing.T, hasLicense bool) { + if hasLicense { + defer ccl.TestingEnableEnterprise()() + } else { + defer ccl.TestingDisableEnterprise()() + } + + rows, err := db.QueryContext(ctx, query) + + if hasLicense { + require.NoError(t, err) + require.NoError(t, rows.Close()) + } else { + require.ErrorContains(t, err, "use of pgcrypto cipher functions requires an enterprise license") + } + }) + }) + } +} diff --git a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/BUILD.bazel b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/BUILD.bazel index 08028d31d0dc..d703ca8517e6 100644 --- a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/BUILD.bazel +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "pgcryptocipherccl", srcs = [ + "cipher.go", "cipher_method.go", "doc.go", "padding.go", @@ -21,6 +22,7 @@ go_test( name = "pgcryptocipherccl_test", srcs = [ "cipher_method_test.go", + "cipher_test.go", "padding_test.go", ], args = ["-test.timeout=295s"], diff --git a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher.go b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher.go new file mode 100644 index 000000000000..5320b3b87208 --- /dev/null +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher.go @@ -0,0 +1,153 @@ +// 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 pgcryptocipherccl + +import ( + "crypto/aes" + "crypto/cipher" + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/errors" +) + +// Encrypt returns the ciphertext obtained by running the encryption +// algorithm for the specified cipher type with the provided key and +// initialization vector over the provided data. +func Encrypt(data []byte, key []byte, iv []byte, cipherType string) ([]byte, error) { + method, err := parseCipherMethod(cipherType) + if err != nil { + return nil, err + } + block, err := newCipher(method, key) + if err != nil { + return nil, err + } + blockSize := block.BlockSize() + data, err = padData(method, data, blockSize) + if err != nil { + return nil, err + } + err = validateDataLength(data, blockSize) + if err != nil { + return nil, err + } + return encrypt(method, block, iv, data) +} + +// Decrypt returns the plaintext obtained by running the decryption +// algorithm for the specified cipher type with the provided key and +// initialization vector over the provided data. +func Decrypt(data []byte, key []byte, iv []byte, cipherType string) ([]byte, error) { + method, err := parseCipherMethod(cipherType) + if err != nil { + return nil, err + } + block, err := newCipher(method, key) + if err != nil { + return nil, err + } + blockSize := block.BlockSize() + err = validateDataLength(data, blockSize) + if err != nil { + return nil, err + } + data, err = decrypt(method, block, iv, data) + if err != nil { + return nil, err + } + return unpadData(method, data) +} + +func newCipher(method cipherMethod, key []byte) (cipher.Block, error) { + switch a := method.algorithm; a { + case aesCipher: + switch l := len(key); { + case l >= 32: + key = zeroPadOrTruncate(key, 32) + case l >= 24: + key = zeroPadOrTruncate(key, 24) + default: + key = zeroPadOrTruncate(key, 16) + } + return aes.NewCipher(key) + default: + return nil, errors.Newf("cannot create new cipher for unknown algorithm: %d", a) + } +} + +func padData(method cipherMethod, data []byte, blockSize int) ([]byte, error) { + switch p := method.padding; p { + case pkcsPadding: + return pkcsPad(data, blockSize) + case noPadding: + return data, nil + default: + return nil, errors.Newf("cannot pad for unknown padding: %d", p) + } +} + +func unpadData(method cipherMethod, data []byte) ([]byte, error) { + switch p := method.padding; p { + case pkcsPadding: + return pkcsUnpad(data) + case noPadding: + return data, nil + default: + return nil, errors.Newf("cannot unpad for unknown padding: %d", p) + } +} + +func validateDataLength(data []byte, blockSize int) error { + if len(data)%blockSize != 0 { + // TODO(yang): Not sure if this is the right pgcode since Postgres + // returns pgcode.ExternalRoutineException. + return pgerror.Newf( + pgcode.InvalidParameterValue, + `data has length %d, which is not a multiple of block size %d`, + len(data), blockSize, + ) + } + return nil +} + +func encrypt(method cipherMethod, block cipher.Block, iv []byte, data []byte) ([]byte, error) { + switch m := method.mode; m { + case cbcMode: + ret := make([]byte, len(data)) + iv = zeroPadOrTruncate(iv, block.BlockSize()) + mode := cipher.NewCBCEncrypter(block, iv) + mode.CryptBlocks(ret, data) + return ret, nil + default: + return nil, errors.Newf("cannot encrypt for unknown mode: %d", m) + } +} + +func decrypt(method cipherMethod, block cipher.Block, iv []byte, data []byte) ([]byte, error) { + switch m := method.mode; m { + case cbcMode: + ret := make([]byte, len(data)) + iv = zeroPadOrTruncate(iv, block.BlockSize()) + mode := cipher.NewCBCDecrypter(block, iv) + mode.CryptBlocks(ret, data) + return ret, nil + default: + return nil, errors.Newf("cannot encrypt for unknown mode: %d", m) + } +} + +func zeroPadOrTruncate(data []byte, size int) []byte { + if len(data) >= size { + return data[:size] + } + paddedData := make([]byte, size) + copy(paddedData, data) + return paddedData +} diff --git a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_test.go b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_test.go new file mode 100644 index 000000000000..d8368fb04a88 --- /dev/null +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/cipher_test.go @@ -0,0 +1,129 @@ +// 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 pgcryptocipherccl_test + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/ccl/pgcryptoccl/pgcryptocipherccl" + "github.com/stretchr/testify/require" +) + +func TestEncrypt(t *testing.T) { + for name, tc := range cipherTestCases { + t.Run(name, func(t *testing.T) { + res, err := pgcryptocipherccl.Encrypt(tc.plaintext, tc.key, tc.iv, tc.cipherType) + require.NoError(t, err) + require.Equal(t, tc.ciphertext, res) + }) + } +} + +func TestDecrypt(t *testing.T) { + for name, tc := range cipherTestCases { + t.Run(name, func(t *testing.T) { + res, err := pgcryptocipherccl.Decrypt(tc.ciphertext, tc.key, tc.iv, tc.cipherType) + require.NoError(t, err) + require.Equal(t, tc.plaintext, res) + }) + } +} + +func FuzzEncryptDecryptAES(f *testing.F) { + for _, tc := range cipherTestCases { + f.Add(tc.plaintext, tc.key, tc.iv) + } + f.Fuzz(func(t *testing.T, plaintext []byte, key []byte, iv []byte) { + ciphertext, err := pgcryptocipherccl.Encrypt(plaintext, key, iv, "aes") + require.NoError(t, err) + decryptedCiphertext, err := pgcryptocipherccl.Decrypt(ciphertext, key, iv, "aes") + require.NoError(t, err) + require.Equal(t, plaintext, decryptedCiphertext) + }) +} + +func BenchmarkEncrypt(b *testing.B) { + for name, tc := range cipherTestCases { + b.Run(name, func(b *testing.B) { + benchmarkEncrypt(b, tc.plaintext, tc.key, tc.iv, tc.cipherType) + }) + } +} + +func BenchmarkDecrypt(b *testing.B) { + for name, tc := range cipherTestCases { + b.Run(name, func(*testing.B) { + benchmarkDecrypt(b, tc.ciphertext, tc.key, tc.iv, tc.cipherType) + }) + } +} + +func benchmarkEncrypt(b *testing.B, data []byte, key []byte, iv []byte, cipherType string) { + for n := 0; n < b.N; n++ { + _, err := pgcryptocipherccl.Encrypt(data, key, iv, cipherType) + require.NoError(b, err) + } +} + +func benchmarkDecrypt(b *testing.B, data []byte, key []byte, iv []byte, cipherType string) { + for n := 0; n < b.N; n++ { + _, err := pgcryptocipherccl.Decrypt(data, key, iv, cipherType) + require.NoError(b, err) + } +} + +// Note: The encrypted values were manually checked against the results of +// the equivalent function on Postgres 15.3. +var cipherTestCases = map[string]struct { + plaintext []byte + key []byte + iv []byte + cipherType string + ciphertext []byte +}{ + "AES-128": { + plaintext: []byte("abc"), + key: []byte("16_byte_long_key"), + cipherType: "aes", + ciphertext: []byte{0x0, 0x26, 0xcd, 0x62, 0x6, 0xcf, 0xd9, 0x21, 0x40, 0x88, 0x3b, 0x75, 0xc0, 0x98, 0xd6, 0x13}, + }, + "AES-128 with IV": { + plaintext: []byte("abc"), + key: []byte("16_byte_long_key"), + iv: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20}, + cipherType: "aes/pad:pkcs", + ciphertext: []byte{0xf8, 0x3e, 0xf0, 0x88, 0x83, 0x9b, 0x2f, 0x19, 0x1c, 0x42, 0x5, 0xa2, 0xe, 0x62, 0x69, 0xd3}, + }, + "AES-192": { + plaintext: []byte("UCVBgQEhwVxXrABksapmqglhTGNWRRnGnUUQHtNanYuyIDrQgHvQanwkfXRvTnJLsMauOfdlLBuJRJkzaGpnVRyENKGwFfauXLJD"), + key: []byte("24_byte_looooooooong_key"), + cipherType: "aes", + ciphertext: []byte{0x44, 0x68, 0x34, 0xe4, 0x3c, 0xb1, 0x76, 0x3, 0xa2, 0xb2, 0x3d, 0x25, 0x13, 0xa7, 0x84, 0xa3, 0x6b, 0xae, 0x79, 0x3, 0x72, 0x74, 0x57, 0x1c, 0x32, 0xcd, 0x29, 0x4a, 0xd, 0x6e, 0xe, 0xf9, 0x27, 0xee, 0x0, 0x76, 0xdc, 0xa4, 0x60, 0xb0, 0x9, 0x0, 0xbc, 0x11, 0x75, 0x4f, 0xee, 0x61, 0x22, 0x52, 0xa6, 0x7, 0x25, 0xa, 0x4f, 0x86, 0xd6, 0x20, 0x56, 0xb6, 0xae, 0xee, 0xc, 0xcd, 0x66, 0x4, 0x88, 0x9e, 0x7f, 0x39, 0xaf, 0x64, 0xce, 0x8b, 0x59, 0x60, 0xc1, 0x74, 0x4d, 0xb2, 0x16, 0xb0, 0x10, 0xea, 0x9c, 0x1, 0x49, 0x6c, 0x42, 0x9f, 0xd5, 0xf5, 0x81, 0x6, 0xf6, 0x4f, 0x21, 0x12, 0xf4, 0xdb, 0xc2, 0x90, 0x74, 0x2a, 0xa3, 0xd1, 0x66, 0x91, 0x5d, 0x97, 0x60, 0x64}, + }, + "AES-192 with IV": { + plaintext: []byte("UCVBgQEhwVxXrABksapmqglhTGNWRRnGnUUQHtNanYuyIDrQgHvQanwkfXRvTnJLsMauOfdlLBuJRJkzaGpnVRyENKGwFfauXLJD"), + key: []byte("24_byte_looooooooong_key"), + iv: []byte{152, 0, 68, 172}, + cipherType: "aes", + ciphertext: []byte{0xd2, 0x84, 0x24, 0x69, 0x11, 0x17, 0xec, 0x5c, 0x1c, 0xbe, 0x2d, 0x34, 0x67, 0xb7, 0xf, 0xa9, 0x94, 0xeb, 0x11, 0x97, 0xe4, 0x60, 0x72, 0xe7, 0xf1, 0x49, 0x9b, 0xc7, 0xc8, 0xc6, 0x7e, 0xad, 0x2b, 0xe2, 0x36, 0x6a, 0xd1, 0x20, 0x0, 0x4a, 0x28, 0x7a, 0x25, 0x86, 0x9a, 0x23, 0x9, 0xe9, 0xf1, 0x2c, 0x7e, 0xe, 0x3a, 0xec, 0xb9, 0x10, 0xf5, 0x35, 0xb6, 0xc2, 0xf6, 0x93, 0x16, 0x2, 0xd, 0x3b, 0x65, 0x63, 0x27, 0x29, 0xd2, 0xeb, 0xc8, 0xc0, 0xa0, 0x2e, 0x19, 0x22, 0xc4, 0x1e, 0x38, 0x73, 0xb0, 0x34, 0x69, 0xda, 0x52, 0x63, 0x9e, 0xaa, 0xa5, 0x93, 0x1, 0x37, 0xff, 0x9d, 0x4e, 0xf1, 0x7f, 0x72, 0x79, 0xe6, 0xad, 0x4b, 0x1f, 0x67, 0x4f, 0x2, 0xab, 0x17, 0x13, 0xcc}, + }, + "AES-256": { + plaintext: []byte("ghJxetyYQiLwdAtibf52bECQbA6QP0FsC0wDURcrR9DRZs7WChql2cJSunTh6rr6b5MM5YYgzWgXHvTxHaEIMiAuEXHsfcyInlxIyaHe2wS03PV6HZ1GKNbhksUx6NjKEoW5SmvmSngdlXSAOWwTYalUP6mKZm9BYHe57LQiHhiX76dKkqqVBvt16t6Hki3hRbqxdUH9JTB3sNAjoH6EjZKnH9h04M02IYncyJAhEpfDINkaerYMQ1Hbavpo0UHu"), + key: []byte("32_byte_looooooooooooooooong_key"), + cipherType: "aes/pad:none", + ciphertext: []byte{0xb1, 0x7e, 0x2f, 0xfd, 0x8d, 0xe2, 0x96, 0x5, 0xdc, 0xb1, 0x74, 0x5f, 0x59, 0x67, 0x2e, 0x30, 0x78, 0x6c, 0xdf, 0x2e, 0xd6, 0xbd, 0x23, 0xcd, 0xf2, 0x9b, 0x31, 0xd0, 0xa1, 0xa3, 0xd5, 0xd0, 0x4e, 0x25, 0x81, 0x25, 0x19, 0xdd, 0x4a, 0x67, 0xb4, 0xba, 0x71, 0x28, 0x2f, 0xa6, 0x31, 0x71, 0xf, 0xfc, 0xd6, 0x9a, 0xe3, 0xa6, 0x7e, 0xc7, 0x5d, 0xe7, 0xe5, 0xef, 0x8c, 0x90, 0xeb, 0x15, 0xfb, 0xa0, 0xd7, 0x30, 0x87, 0xf6, 0x5, 0xa9, 0x16, 0x86, 0xcc, 0xc0, 0x9f, 0x19, 0x14, 0xb8, 0x43, 0x72, 0xcb, 0x35, 0x9d, 0x18, 0xdc, 0xd2, 0xd6, 0xe4, 0x30, 0xc5, 0xc4, 0x8f, 0x4b, 0x98, 0xdd, 0x29, 0x53, 0xd9, 0x6, 0x1b, 0x48, 0xe9, 0x4a, 0x76, 0x36, 0xf5, 0x5a, 0x55, 0x5e, 0xd, 0x18, 0x57, 0xfe, 0xc2, 0x90, 0x5b, 0x82, 0x50, 0x7e, 0x14, 0xe1, 0xe0, 0xb0, 0x5e, 0x26, 0x44, 0x61, 0xe4, 0x22, 0x72, 0x7, 0x19, 0x60, 0xef, 0x6c, 0xb9, 0xfb, 0xa5, 0x9e, 0x5c, 0x7, 0xf6, 0x13, 0xc7, 0xa1, 0x97, 0x6b, 0xc, 0x1c, 0xaa, 0xaa, 0xcf, 0xd8, 0x6d, 0x39, 0x50, 0x57, 0x94, 0xce, 0x5d, 0xd, 0xf2, 0xd4, 0x2b, 0xbd, 0x71, 0xb4, 0xc6, 0x3b, 0x92, 0xb0, 0x5f, 0x3d, 0x56, 0xf5, 0x23, 0x6d, 0xaf, 0xb0, 0xbc, 0x62, 0xd5, 0xf0, 0x1e, 0x75, 0xad, 0xc2, 0x64, 0xbf, 0x66, 0xe7, 0xad, 0xab, 0xfd, 0xbf, 0xde, 0x72, 0xa4, 0x6, 0x39, 0xd2, 0xf4, 0x3, 0x5e, 0x27, 0x6, 0x98, 0x92, 0x7d, 0x2c, 0xe4, 0xac, 0x9e, 0xff, 0x41, 0xc1, 0xe7, 0xc1, 0x69, 0xbd, 0x8f, 0x5b, 0xc4, 0x1e, 0x36, 0x59, 0xa9, 0xb5, 0xbc, 0x64, 0x49, 0xfd, 0x78, 0x7f, 0xc, 0xad, 0x25, 0xae, 0x4b, 0xec, 0xb5, 0x4f, 0x97, 0x9e, 0x8c, 0xa6, 0x7b, 0x9, 0x2a, 0x76, 0xcd, 0x2d, 0x2e, 0x41}, + }, + "AES-256 with IV": { + plaintext: []byte("ghJxetyYQiLwdAtibf52bECQbA6QP0FsC0wDURcrR9DRZs7WChql2cJSunTh6rr6b5MM5YYgzWgXHvTxHaEIMiAuEXHsfcyInlxIyaHe2wS03PV6HZ1GKNbhksUx6NjKEoW5SmvmSngdlXSAOWwTYalUP6mKZm9BYHe57LQiHhiX76dKkqqVBvt16t6Hki3hRbqxdUH9JTB3sNAjoH6EjZKnH9h04M02IYncyJAhEpfDINkaerYMQ1Hbavpo0UHu"), + key: []byte("32_byte_looooooooooooooooong_key"), + iv: []byte{154, 201, 158, 144, 3, 25, 184, 208, 70, 90, 123, 45, 168, 27, 236, 25}, + cipherType: "aes", + ciphertext: []byte{0x8b, 0xb2, 0xd7, 0x52, 0xdd, 0xff, 0x46, 0xa, 0xb9, 0x67, 0x12, 0x57, 0xe0, 0xff, 0x61, 0x23, 0x7, 0x60, 0x2, 0xe2, 0x28, 0x53, 0x53, 0x7c, 0x3e, 0xa, 0xdf, 0x8, 0x6e, 0xbd, 0x54, 0x48, 0xb3, 0x82, 0xf7, 0x6a, 0xce, 0xc6, 0xc8, 0xb3, 0xda, 0x19, 0xc5, 0x81, 0x91, 0x11, 0xd8, 0xd0, 0x87, 0x7d, 0xc0, 0x29, 0xb4, 0xe4, 0x8d, 0x74, 0x93, 0x0, 0xeb, 0x34, 0xc9, 0xd2, 0xb1, 0xc9, 0x30, 0xbc, 0xd9, 0x67, 0x7b, 0xa4, 0xeb, 0x81, 0x32, 0x8c, 0xa9, 0x95, 0xb9, 0x7c, 0xfb, 0xc0, 0x5f, 0x87, 0x2c, 0x32, 0x43, 0x15, 0xb4, 0xd1, 0x9b, 0x25, 0x3c, 0xe, 0xaf, 0xd1, 0x56, 0x17, 0xac, 0x26, 0x51, 0x17, 0x36, 0xda, 0xf6, 0x13, 0x5d, 0xf8, 0x12, 0xf6, 0x1f, 0x4, 0x17, 0x5b, 0xa8, 0xf1, 0xb1, 0xda, 0xc, 0x38, 0x13, 0x10, 0xe7, 0xe0, 0xd7, 0xa8, 0x4a, 0x69, 0x83, 0x5e, 0xe7, 0xc0, 0xf7, 0x21, 0xdb, 0xe3, 0x9d, 0x84, 0xce, 0x8e, 0x50, 0x7, 0x44, 0xe4, 0xe, 0xdf, 0xde, 0x88, 0xa5, 0xee, 0xd7, 0xf4, 0x32, 0xa0, 0x60, 0x24, 0x9d, 0x8b, 0x5a, 0xdf, 0xbc, 0xca, 0x6c, 0x2c, 0x80, 0x40, 0x47, 0xdf, 0x3f, 0x9b, 0xbe, 0x86, 0x34, 0x2b, 0x79, 0xc1, 0x27, 0xcf, 0x4c, 0xda, 0x89, 0x6b, 0xd8, 0xcb, 0xf8, 0xcd, 0x57, 0x46, 0xbc, 0x54, 0x55, 0x21, 0xc3, 0x34, 0x5e, 0x4c, 0xc8, 0xc3, 0x21, 0x51, 0xce, 0x8b, 0x3e, 0xcb, 0x36, 0xdc, 0x32, 0x31, 0xdc, 0x27, 0xf8, 0x12, 0x77, 0x64, 0xa8, 0xb8, 0x6d, 0x3c, 0x42, 0x6a, 0xd9, 0x7c, 0xbf, 0x14, 0xc5, 0x58, 0xdb, 0x1d, 0x24, 0x74, 0xb5, 0x47, 0xce, 0x54, 0x50, 0x32, 0xfc, 0xb9, 0x9f, 0xd3, 0x45, 0xa2, 0x5, 0x5d, 0xa4, 0x8a, 0x57, 0x58, 0x10, 0xc7, 0x5e, 0x83, 0x9f, 0x2b, 0x64, 0xe3, 0xd4, 0x98, 0x15, 0xf7, 0x8a, 0xdf, 0xe7, 0xc7, 0x2, 0x27, 0xa4, 0xff, 0xa7, 0x98, 0x24, 0xa, 0xf0, 0x2d}, + }, +} diff --git a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/padding.go b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/padding.go index 02ed2282a45e..efd83d150699 100644 --- a/pkg/ccl/pgcryptoccl/pgcryptocipherccl/padding.go +++ b/pkg/ccl/pgcryptoccl/pgcryptocipherccl/padding.go @@ -23,10 +23,14 @@ func pkcsPad(data []byte, blockSize int) ([]byte, error) { return nil, errors.Newf("invalid block size for PKCS padding: %d", blockSize) } - paddingLen := blockSize - len(data)%blockSize - padding := bytes.Repeat([]byte{byte(paddingLen)}, paddingLen) + paddedData := make([]byte, len(data)) + copy(paddedData, data) - return append(data, padding...), nil + paddingSize := blockSize - len(data)%blockSize + padding := bytes.Repeat([]byte{byte(paddingSize)}, paddingSize) + paddedData = append(paddedData, padding...) + + return paddedData, nil } // pkcsUnpad removes the padding added by pkcsPad. diff --git a/pkg/sql/logictest/testdata/logic_test/pgcrypto_builtins b/pkg/sql/logictest/testdata/logic_test/pgcrypto_builtins index d4c2880c3c3a..f7f6d7391227 100644 --- a/pkg/sql/logictest/testdata/logic_test/pgcrypto_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pgcrypto_builtins @@ -288,3 +288,23 @@ statement error pgcode 22023 invalid salt encoding SELECT crypt('password', '$2a$06$#kv6DxN3PpZo4YboQRrIVO') subtest end + +subtest ccl_functions + +skipif config 3node-tenant-default-configs +query error pgcode XXC01 encrypt can only be used with a CCL distribution +SELECT encrypt('abc', 'key', 'aes') + +skipif config 3node-tenant-default-configs +query error pgcode XXC01 encrypt_iv can only be used with a CCL distribution +SELECT encrypt_iv('abc', 'key', '123', 'aes') + +skipif config 3node-tenant-default-configs +query error pgcode XXC01 decrypt can only be used with a CCL distribution +SELECT decrypt('\xdb5f149a7caf0cd275ca18c203a212c9', 'key', 'aes') + +skipif config 3node-tenant-default-configs +query error pgcode XXC01 decrypt_iv can only be used with a CCL distribution +SELECT decrypt_iv('\x91b4ef63852013c8da53829da662b871', 'key', '123', 'aes') + +subtest end diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index 66584bddcc70..ffe2655a7721 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -81,6 +81,7 @@ go_library( "//pkg/sql/sem/asof", "//pkg/sql/sem/builtins/builtinconstants", "//pkg/sql/sem/builtins/builtinsregistry", + "//pkg/sql/sem/builtins/pgcrypto", "//pkg/sql/sem/builtins/pgformat", "//pkg/sql/sem/cast", "//pkg/sql/sem/catconstants", diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go index 2fc94ba12763..5dff72bd7031 100644 --- a/pkg/sql/sem/builtins/fixed_oids.go +++ b/pkg/sql/sem/builtins/fixed_oids.go @@ -2453,6 +2453,10 @@ var builtinOidsArray = []string{ 2482: `bitmask_xor(a: varbit, b: string) -> varbit`, 2483: `bitmask_xor(a: string, b: varbit) -> varbit`, 2484: `oidvectortypes(vector: oidvector) -> string`, + 2485: `encrypt(data: bytes, key: bytes, type: string) -> bytes`, + 2486: `encrypt_iv(data: bytes, key: bytes, iv: bytes, type: string) -> bytes`, + 2487: `decrypt(data: bytes, key: bytes, type: string) -> bytes`, + 2488: `decrypt_iv(data: bytes, key: bytes, iv: bytes, type: string) -> bytes`, } var builtinOidsBySignature map[string]oid.Oid diff --git a/pkg/sql/sem/builtins/pgcrypto/BUILD.bazel b/pkg/sql/sem/builtins/pgcrypto/BUILD.bazel new file mode 100644 index 000000000000..f1b9992ee6bc --- /dev/null +++ b/pkg/sql/sem/builtins/pgcrypto/BUILD.bazel @@ -0,0 +1,13 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "pgcrypto", + srcs = ["pgcrypto.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/pgcrypto", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", + "//pkg/sql/sem/eval", + ], +) diff --git a/pkg/sql/sem/builtins/pgcrypto/pgcrypto.go b/pkg/sql/sem/builtins/pgcrypto/pgcrypto.go new file mode 100644 index 000000000000..54685d1303f2 --- /dev/null +++ b/pkg/sql/sem/builtins/pgcrypto/pgcrypto.go @@ -0,0 +1,47 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package pgcrypto + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" +) + +var Decrypt = func(_ context.Context, _ *eval.Context, data []byte, key []byte, cipherType string) ([]byte, error) { + return nil, pgerror.New( + pgcode.CCLRequired, + "decrypt can only be used with a CCL distribution", + ) +} + +var DecryptIV = func(_ context.Context, _ *eval.Context, data []byte, key []byte, iv []byte, cipherType string) ([]byte, error) { + return nil, pgerror.New( + pgcode.CCLRequired, + "decrypt_iv can only be used with a CCL distribution", + ) +} + +var Encrypt = func(_ context.Context, _ *eval.Context, data []byte, key []byte, cipherType string) ([]byte, error) { + return nil, pgerror.New( + pgcode.CCLRequired, + "encrypt can only be used with a CCL distribution", + ) +} + +var EncryptIV = func(_ context.Context, _ *eval.Context, data []byte, key []byte, iv []byte, cipherType string) ([]byte, error) { + return nil, pgerror.New( + pgcode.CCLRequired, + "encrypt_iv can only be used with a CCL distribution", + ) +} diff --git a/pkg/sql/sem/builtins/pgcrypto_builtins.go b/pkg/sql/sem/builtins/pgcrypto_builtins.go index 34ed29858f1b..02331b72226b 100644 --- a/pkg/sql/sem/builtins/pgcrypto_builtins.go +++ b/pkg/sql/sem/builtins/pgcrypto_builtins.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/pgcrypto" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" @@ -42,6 +43,13 @@ func init() { } } +const cipherSupportedCipherTypeInfo = "The cipher type must have the format `[-][/pad:]` where:\n" + + "* `` is `aes`\n" + + "* `` is `cbc` (default)\n" + + "* `` is `pkcs` (default) or `none`" + +const cipherRequiresEnterpriseLicenseInfo = "This function requires an enterprise license on a CCL distribution." + var pgcryptoBuiltins = map[string]builtinDefinition{ "crypt": makeBuiltin( @@ -63,6 +71,61 @@ var pgcryptoBuiltins = map[string]builtinDefinition{ }, ), + "decrypt": makeBuiltin( + tree.FunctionProperties{Category: builtinconstants.CategoryCrypto}, + tree.Overload{ + Types: tree.ParamTypes{ + {Name: "data", Typ: types.Bytes}, + {Name: "key", Typ: types.Bytes}, + {Name: "type", Typ: types.String}, + }, + ReturnType: tree.FixedReturnType(types.Bytes), + Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + data := []byte(tree.MustBeDBytes(args[0])) + key := []byte(tree.MustBeDBytes(args[1])) + cipherType := string(tree.MustBeDString(args[2])) + decryptedData, err := pgcrypto.Decrypt(ctx, evalCtx, data, key, cipherType) + if err != nil { + return nil, err + } + return tree.NewDBytes(tree.DBytes(decryptedData)), nil + }, + Info: "Decrypt `data` with `key` using the cipher method specified by `type`." + + "\n\n" + cipherSupportedCipherTypeInfo + + "\n\n" + cipherRequiresEnterpriseLicenseInfo, + Volatility: volatility.Immutable, + }, + ), + + "decrypt_iv": makeBuiltin( + tree.FunctionProperties{Category: builtinconstants.CategoryCrypto}, + tree.Overload{ + Types: tree.ParamTypes{ + {Name: "data", Typ: types.Bytes}, + {Name: "key", Typ: types.Bytes}, + {Name: "iv", Typ: types.Bytes}, + {Name: "type", Typ: types.String}, + }, + ReturnType: tree.FixedReturnType(types.Bytes), + Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + data := []byte(tree.MustBeDBytes(args[0])) + key := []byte(tree.MustBeDBytes(args[1])) + iv := []byte(tree.MustBeDBytes(args[2])) + cipherType := string(tree.MustBeDString(args[3])) + decryptedData, err := pgcrypto.DecryptIV(ctx, evalCtx, data, key, iv, cipherType) + if err != nil { + return nil, err + } + return tree.NewDBytes(tree.DBytes(decryptedData)), nil + }, + Info: "Decrypt `data` with `key` using the cipher method specified by `type`. " + + "If the mode is CBC, the provided `iv` will be used. Otherwise, it will be ignored." + + "\n\n" + cipherSupportedCipherTypeInfo + + "\n\n" + cipherRequiresEnterpriseLicenseInfo, + Volatility: volatility.Immutable, + }, + ), + "digest": makeBuiltin( tree.FunctionProperties{Category: builtinconstants.CategoryCrypto}, tree.Overload{ @@ -105,6 +168,61 @@ var pgcryptoBuiltins = map[string]builtinDefinition{ }, ), + "encrypt": makeBuiltin( + tree.FunctionProperties{Category: builtinconstants.CategoryCrypto}, + tree.Overload{ + Types: tree.ParamTypes{ + {Name: "data", Typ: types.Bytes}, + {Name: "key", Typ: types.Bytes}, + {Name: "type", Typ: types.String}, + }, + ReturnType: tree.FixedReturnType(types.Bytes), + Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + data := []byte(tree.MustBeDBytes(args[0])) + key := []byte(tree.MustBeDBytes(args[1])) + cipherType := string(tree.MustBeDString(args[2])) + encryptedData, err := pgcrypto.Encrypt(ctx, evalCtx, data, key, cipherType) + if err != nil { + return nil, err + } + return tree.NewDBytes(tree.DBytes(encryptedData)), nil + }, + Info: "Encrypt `data` with `key` using the cipher method specified by `type`." + + "\n\n" + cipherSupportedCipherTypeInfo + + "\n\n" + cipherRequiresEnterpriseLicenseInfo, + Volatility: volatility.Immutable, + }, + ), + + "encrypt_iv": makeBuiltin( + tree.FunctionProperties{Category: builtinconstants.CategoryCrypto}, + tree.Overload{ + Types: tree.ParamTypes{ + {Name: "data", Typ: types.Bytes}, + {Name: "key", Typ: types.Bytes}, + {Name: "iv", Typ: types.Bytes}, + {Name: "type", Typ: types.String}, + }, + ReturnType: tree.FixedReturnType(types.Bytes), + Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + data := []byte(tree.MustBeDBytes(args[0])) + key := []byte(tree.MustBeDBytes(args[1])) + iv := []byte(tree.MustBeDBytes(args[2])) + cipherType := string(tree.MustBeDString(args[3])) + encryptedData, err := pgcrypto.EncryptIV(ctx, evalCtx, data, key, iv, cipherType) + if err != nil { + return nil, err + } + return tree.NewDBytes(tree.DBytes(encryptedData)), nil + }, + Info: "Encrypt `data` with `key` using the cipher method specified by `type`. " + + "If the mode is CBC, the provided `iv` will be used. Otherwise, it will be ignored." + + "\n\n" + cipherSupportedCipherTypeInfo + + "\n\n" + cipherRequiresEnterpriseLicenseInfo, + Volatility: volatility.Immutable, + }, + ), + "gen_random_uuid": generateRandomUUID4Impl(), "gen_salt": makeBuiltin(