Skip to content

Commit

Permalink
sql,ccl: allow builtins to be registered from anywhere
Browse files Browse the repository at this point in the history
Previously, builtin functions needed to be created from
inside the builtins package. This lead to a giant package
with all the implementations, and also made it difficult
to register non-CCL builtins conditionally on the build
being non-oss. This commit makes builtin registration capable
of happening in any init() code by making everything that
needs to iterate over all builtins provide a hook instead
of forcing a particular build order.
This commit only leverages that to move sql telemetry logic
out of the builtins package. The following commit uses it
to create a CCL builtin function.

Release note: None

Release justification: Refactor necessary for CCL functions.
  • Loading branch information
HonoreDB committed Sep 7, 2022
1 parent 5ee5d85 commit d4a41be
Show file tree
Hide file tree
Showing 35 changed files with 234 additions and 82 deletions.
12 changes: 11 additions & 1 deletion pkg/ccl/utilccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "utilccl",
srcs = ["license_check.go"],
srcs = [
"builtins.go",
"license_check.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/utilccl",
visibility = ["//visibility:public"],
deps = [
Expand All @@ -14,6 +17,8 @@ go_library(
"//pkg/settings/cluster",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/sem/builtins/builtinsregistry",
"//pkg/sql/sem/tree",
"//pkg/util/envutil",
"//pkg/util/log",
"//pkg/util/metric",
Expand All @@ -29,6 +34,7 @@ go_test(
name = "utilccl_test",
size = "small",
srcs = [
"builtins_test.go",
"license_check_test.go",
"license_test.go",
],
Expand All @@ -39,11 +45,15 @@ go_test(
"//pkg/ccl/utilccl/licenseccl",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql/sem/eval",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/testutils",
"//pkg/util/envutil",
"//pkg/util/stop",
"//pkg/util/timeutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
],
)
Expand Down
24 changes: 24 additions & 0 deletions pkg/ccl/utilccl/builtins.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
// Copyright 2022 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 utilccl

import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)

// RegisterCCLBuiltin adds a builtin defined in CCL code to the global builtins registry.
func RegisterCCLBuiltin(name string, description string, overload tree.Overload) {
props := tree.FunctionProperties{
Class: tree.NormalClass,
Category: `CCL-only internal function`,
}

builtinsregistry.Register(name, &props, []tree.Overload{overload})
}
59 changes: 59 additions & 0 deletions pkg/ccl/utilccl/builtins_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
// Copyright 2017 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 utilccl

import (
"testing"

"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

func unexportedFunction(s string) string {
return "behold: " + s
}

func TestRegister(t *testing.T) {
RegisterCCLBuiltin("test_builtin_from_unexported_function", "", unexportedFunction)
RegisterCCLBuiltin("test_builtin_with_error", "", func(s string) (string, error) {
if s == "dog" {
return "pet the dog", nil
}
return s, errors.New("please provide dog")
})
resolver := func(fn string) (*tree.FunctionDefinition, error) {
name := tree.UnresolvedName{NumParts: 1, Parts: [4]string{fn}}
return name.ResolveFunction(nil)
}
wrapper, err := resolver("test_builtin_from_unexported_function")
require.NoError(t, err)
require.Equal(t, "(: string) -> string", wrapper.Definition[0].(*tree.Overload).Signature(true))

args := tree.Datums{tree.NewDString("dog")}
ctx := eval.MakeTestingEvalContext(nil)
o, err := wrapper.Definition[0].(*tree.Overload).Fn.(eval.FnOverload)(&ctx, args)
require.NoError(t, err)
require.Equal(t, o.ResolvedType(), types.String)
require.Equal(t, "'behold: dog'", o.String())

petter, err := resolver("test_builtin_with_error")
require.NoError(t, err)
require.Equal(t, "(: string) -> string", petter.Definition[0].(*tree.Overload).Signature(true))

_, err = petter.Definition[0].(*tree.Overload).Fn.(eval.FnOverload)(&ctx, args)
require.NoError(t, err)

args[0] = tree.NewDString("no dog")
_, err = petter.Definition[0].(*tree.Overload).Fn.(eval.FnOverload)(&ctx, args)
require.Error(t, err)

}
2 changes: 1 addition & 1 deletion pkg/cli/doctor.go
Original file line number Diff line number Diff line change
Expand Up @@ -349,7 +349,7 @@ func fromZipDir(
retErr error,
) {
// To make parsing user functions code happy.
_ = builtins.AllBuiltinNames
_ = builtins.AllBuiltinNames()

descTable = make(doctor.DescriptorTable, 0)
if err := slurp(zipDirPath, "system.descriptor.txt", func(row string) error {
Expand Down
6 changes: 3 additions & 3 deletions pkg/cmd/docgen/funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,17 +44,17 @@ func init() {
}

if err := os.WriteFile(
filepath.Join(outDir, "functions.md"), generateFunctions(builtins.AllBuiltinNames, true), 0644,
filepath.Join(outDir, "functions.md"), generateFunctions(builtins.AllBuiltinNames(), true), 0644,
); err != nil {
return err
}
if err := os.WriteFile(
filepath.Join(outDir, "aggregates.md"), generateFunctions(builtins.AllAggregateBuiltinNames, false), 0644,
filepath.Join(outDir, "aggregates.md"), generateFunctions(builtins.AllAggregateBuiltinNames(), false), 0644,
); err != nil {
return err
}
if err := os.WriteFile(
filepath.Join(outDir, "window_functions.md"), generateFunctions(builtins.AllWindowBuiltinNames, false), 0644,
filepath.Join(outDir, "window_functions.md"), generateFunctions(builtins.AllWindowBuiltinNames(), false), 0644,
); err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/builtin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ func TestFuncNull(t *testing.T) {
}
}

for _, name := range builtins.AllBuiltinNames {
for _, name := range builtins.AllBuiltinNames() {
switch strings.ToLower(name) {
case "crdb_internal.force_panic", "crdb_internal.force_log_fatal", "pg_sleep":
continue
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/schemaexpr/check_constraint_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ func TestCheckConstraintBuilder_Build(t *testing.T) {
semaCtx := tree.MakeSemaContext()

// Trick to get the init() for the builtins package to run.
_ = builtins.AllBuiltinNames
_ = builtins.AllBuiltinNames()

database := tree.Name("foo")
table := tree.Name("bar")
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/catalog/schemaexpr/expr_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ func TestValidateExpr(t *testing.T) {
semaCtx := tree.MakeSemaContext()

// Trick to get the init() for the builtins package to run.
_ = builtins.AllBuiltinNames
_ = builtins.AllBuiltinNames()

database := tree.Name("foo")
table := tree.Name("bar")
Expand Down Expand Up @@ -111,7 +111,7 @@ func TestValidateExpr(t *testing.T) {

func TestExtractColumnIDs(t *testing.T) {
// Trick to get the init() for the builtins package to run.
_ = builtins.AllBuiltinNames
_ = builtins.AllBuiltinNames()

table := tree.Name("foo")
desc := testTableDesc(
Expand Down Expand Up @@ -154,7 +154,7 @@ func TestExtractColumnIDs(t *testing.T) {

func TestValidColumnReferences(t *testing.T) {
// Trick to get the init() for the builtins package to run.
_ = builtins.AllBuiltinNames
_ = builtins.AllBuiltinNames()

table := tree.Name("foo")
desc := testTableDesc(
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/schemaexpr/partial_index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ func TestIndexPredicateValidator_Validate(t *testing.T) {
semaCtx := tree.MakeSemaContext()

// Trick to get the init() for the builtins package to run.
_ = builtins.AllBuiltinNames
_ = builtins.AllBuiltinNames()

database := tree.Name("foo")
table := tree.Name("bar")
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/colexec/builtin_funcs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ func TestBasicBuiltinFunctions(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
// Trick to get the init() for the builtins package to run.
_ = builtins.AllBuiltinNames
_ = builtins.AllBuiltinNames()
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
evalCtx := eval.MakeTestingEvalContext(st)
Expand Down Expand Up @@ -179,7 +179,7 @@ func benchmarkBuiltinFunctions(b *testing.B, useSelectionVector bool, hasNulls b
}

func BenchmarkBuiltinFunctions(b *testing.B) {
_ = builtins.AllBuiltinNames
_ = builtins.AllBuiltinNames()
for _, useSel := range []bool{true, false} {
for _, hasNulls := range []bool{true, false} {
b.Run(fmt.Sprintf("useSel=%t,hasNulls=%t", useSel, hasNulls), func(b *testing.B) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -2427,7 +2427,7 @@ CREATE TABLE crdb_internal.builtin_functions (
details STRING NOT NULL
)`,
populate: func(ctx context.Context, _ *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
for _, name := range builtins.AllBuiltinNames {
for _, name := range builtins.AllBuiltinNames() {
props, overloads := builtinsregistry.GetBuiltinProperties(name)
for _, f := range overloads {
if err := addRow(
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/memo/typing_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ func TestTypingComparisonAssumptions(t *testing.T) {
// 2. The return type of overloads is fixed.
// 3. The return type for min/max aggregates is same as type of argument.
func TestTypingAggregateAssumptions(t *testing.T) {
for _, name := range builtins.AllAggregateBuiltinNames {
for _, name := range builtins.AllAggregateBuiltinNames() {
if name == builtins.AnyNotNull ||
name == "percentile_disc" ||
name == "percentile_cont" {
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/pg_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -2298,7 +2298,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-proc.html`,
err := forEachDatabaseDesc(ctx, p, dbContext, false, /* requiresPrivileges */
func(db catalog.DatabaseDescriptor) error {
nspOid := h.NamespaceOid(db.GetID(), pgCatalogName)
for _, name := range builtins.AllBuiltinNames {
for _, name := range builtins.AllBuiltinNames() {
// parser.Builtins contains duplicate uppercase and lowercase keys.
// Only return the lowercase ones for compatibility with postgres.
var first rune
Expand Down Expand Up @@ -4295,7 +4295,7 @@ https://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html`,
h := makeOidHasher()
return forEachDatabaseDesc(ctx, p, dbContext, false, /* requiresPrivileges */
func(db catalog.DatabaseDescriptor) error {
for _, name := range builtins.AllAggregateBuiltinNames {
for _, name := range builtins.AllAggregateBuiltinNames() {
if name == builtins.AnyNotNull {
// any_not_null is treated as a special case.
continue
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/sem/builtins/aggregate_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ import (
"github.com/twpayne/go-geom"
)

func initAggregateBuiltins() {
func init() {
// Add all aggregates to the builtins map after a few sanity checks.
for k, v := range aggregates {

Expand Down
Loading

0 comments on commit d4a41be

Please sign in to comment.