Skip to content

Commit

Permalink
sql,externalconn: introduce CREATE EXTERNAL CONNECTION
Browse files Browse the repository at this point in the history
This change introduces the `CREATE EXTERNAL CONNECTION` syntax
to CockroachDB. This statement can be used to create an External
Connection object that represents an external resource.

Majority of this change is introducing the required interfaces
to persist an External Connection object in the `system.external_connections`
table. We only register `nodelocal` as a supported External Connection
to allow for end to end testing of functionality. A user can now execute:

`CREATE EXTERNAL CONNECTION foo AS 'nodelocal://1/foo';`

All the other providers will be registered as part of #84228.
Furthermore, none of the permission model outlined in the
RFC #84209 has been implemented, to minimize the scope of this change.

Release note (sql change): introduce `CREATE EXTERNAL CONNECTION`
syntax that can be used to create an External Connection representing
a resource that resides outside of CockroachDB. The only supported resource
at the moment is a `nodelocal` URI that can be represented as an External
Connection object using:

`CREATE EXTERNAL CONNECTION foo AS 'nodelocal://1/foo'`;

Fixes: #84225
  • Loading branch information
adityamaru committed Jul 14, 2022
1 parent b21322b commit 066577b
Show file tree
Hide file tree
Showing 33 changed files with 1,172 additions and 42 deletions.
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/create_stmt.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -5,3 +5,4 @@ create_stmt ::=
| create_schedule_for_backup_stmt
| create_changefeed_stmt
| create_extension_stmt
| create_external_connection_stmt
8 changes: 8 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -178,6 +178,7 @@ create_stmt ::=
| create_schedule_for_backup_stmt
| create_changefeed_stmt
| create_extension_stmt
| create_external_connection_stmt

delete_stmt ::=
opt_with_clause 'DELETE' 'FROM' table_expr_opt_alias_idx opt_where_clause opt_sort_clause opt_limit_clause returning_clause
Expand Down Expand Up @@ -564,6 +565,9 @@ create_extension_stmt ::=
'CREATE' 'EXTENSION' 'IF' 'NOT' 'EXISTS' name
| 'CREATE' 'EXTENSION' name

create_external_connection_stmt ::=
'CREATE' 'EXTERNAL' 'CONNECTION' connection_label_spec 'AS' string_or_placeholder

opt_with_clause ::=
with_clause
|
Expand Down Expand Up @@ -1686,6 +1690,10 @@ target_list ::=
changefeed_target_expr ::=
insert_target

connection_label_spec ::=
string_or_placeholder
| 'IF' 'NOT' 'EXISTS' string_or_placeholder

with_clause ::=
'WITH' cte_list
| 'WITH' 'RECURSIVE' cte_list
Expand Down
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ ALL_TESTS = [
"//pkg/ccl/changefeedccl/schemafeed:schemafeed_test",
"//pkg/ccl/changefeedccl:changefeedccl_test",
"//pkg/ccl/cliccl:cliccl_test",
"//pkg/ccl/cloudccl/externalconnccl:externalconnccl_test",
"//pkg/ccl/importerccl:importerccl_test",
"//pkg/ccl/jobsccl/jobsprotectedtsccl:jobsprotectedtsccl_test",
"//pkg/ccl/kvccl/kvfollowerreadsccl:kvfollowerreadsccl_test",
Expand Down
29 changes: 29 additions & 0 deletions pkg/ccl/cloudccl/externalconnccl/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "externalconnccl_test",
srcs = [
"datadriven_test.go",
"main_test.go",
],
data = glob(["testdata/**"]),
deps = [
"//pkg/base",
"//pkg/ccl/kvccl/kvtenantccl",
"//pkg/cloud/externalconn/externalconntestutils",
"//pkg/cloud/externalconn/impl",
"//pkg/jobs",
"//pkg/roachpb",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/leaktest",
"//pkg/util/randutil",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_stretchr_testify//require",
],
)
97 changes: 97 additions & 0 deletions pkg/ccl/cloudccl/externalconnccl/datadriven_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
// Copyright 2022 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 externalconnccl_test

import (
"context"
gosql "database/sql"
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/cloud/externalconn/externalconntestutils"
_ "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/impl" // register all the concrete External Connection implementations
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/datadriven"
"github.com/stretchr/testify/require"
)

func TestDataDriven(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
datadriven.Walk(t, testutils.TestDataPath(t), func(t *testing.T, path string) {
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speeds up test
},
},
})
defer tc.Stopper().Stop(ctx)

externalConnTestCluster := externalconntestutils.NewHandle(t, tc)
defer externalConnTestCluster.Cleanup()

externalConnTestCluster.InitializeTenant(ctx, roachpb.SystemTenantID)
datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string {
tenantID := roachpb.SystemTenantID
if d.HasArg("tenant") {
var id uint64
d.ScanArgs(t, "tenant", &id)
tenantID = roachpb.MakeTenantID(id)
}

tenant, found := externalConnTestCluster.LookupTenant(tenantID)
if d.Cmd != "initialize" {
require.Truef(t, found, "tenant %s not found (was it initialized?)", tenantID)
}

switch d.Cmd {
case "initialize":
externalConnTestCluster.InitializeTenant(ctx, tenantID)

case "exec-sql":
if err := tenant.ExecWithErr(d.Input); err != nil {
return fmt.Sprint(err.Error())
}

case "query-sql":
var rows *gosql.Rows
var err error
if rows, err = tenant.QueryWithErr(d.Input); err != nil {
return fmt.Sprint(err.Error())
}
output, err := sqlutils.RowsToDataDrivenOutput(rows)
require.NoError(t, err)
return output

case "inspect-system-table":
rows := tenant.Query(`
SELECT connection_name, connection_type, crdb_internal.pb_to_json('cockroach.cloud.externalconn.connectionpb.ConnectionDetails', connection_details)
FROM system.external_connections;
`)
output, err := sqlutils.RowsToDataDrivenOutput(rows)
require.NoError(t, err)
return output

default:
t.Fatalf("unknown command: %s", d.Cmd)
}
return ""
})
})
}
34 changes: 34 additions & 0 deletions pkg/ccl/cloudccl/externalconnccl/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// Copyright 2022 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 externalconnccl_test

import (
"os"
"testing"

_ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl"
"github.com/cockroachdb/cockroach/pkg/security/securityassets"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
)

func TestMain(m *testing.M) {
securityassets.SetLoader(securitytest.EmbeddedAssets)
randutil.SeedForTests()
serverutils.InitTestServerFactory(server.TestServerFactory)
serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)
os.Exit(m.Run())
}

//go:generate ../../util/leaktest/add-leaktest.sh *_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
subtest basic-nodelocal

exec-sql
CREATE EXTERNAL CONNECTION foo AS 'nodelocal://1/foo/bar';
----

inspect-system-table
----
foo STORAGE {"nodelocal": {"cfg": {"nodeId": 1, "path": "/foo/bar"}}, "provider": "nodelocal"}

# Try to create another External Connection with the same name.
exec-sql
CREATE EXTERNAL CONNECTION foo AS 'nodelocal://1/foo';
----
pq: external connection with connection name 'foo' already exists

# Create another External Connection with a unique name.
exec-sql
CREATE EXTERNAL CONNECTION bar123 AS 'nodelocal://1/baz';
----

inspect-system-table
----
bar123 STORAGE {"nodelocal": {"cfg": {"nodeId": 1, "path": "/baz"}}, "provider": "nodelocal"}
foo STORAGE {"nodelocal": {"cfg": {"nodeId": 1, "path": "/foo/bar"}}, "provider": "nodelocal"}

subtest end
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
subtest basic-nodelocal

initialize tenant=10
----

exec-sql
CREATE EXTERNAL CONNECTION foo AS 'nodelocal://1/foo/bar';
----

inspect-system-table
----
foo STORAGE {"nodelocal": {"cfg": {"nodeId": 1, "path": "/foo/bar"}}, "provider": "nodelocal"}

# Try to create another External Connection with the same name.
exec-sql
CREATE EXTERNAL CONNECTION foo AS 'nodelocal://1/foo';
----
pq: external connection with connection name 'foo' already exists

# Create another External Connection with a unique name.
exec-sql
CREATE EXTERNAL CONNECTION bar123 AS 'nodelocal://1/baz';
----

inspect-system-table
----
bar123 STORAGE {"nodelocal": {"cfg": {"nodeId": 1, "path": "/baz"}}, "provider": "nodelocal"}
foo STORAGE {"nodelocal": {"cfg": {"nodeId": 1, "path": "/foo/bar"}}, "provider": "nodelocal"}

subtest end
25 changes: 25 additions & 0 deletions pkg/cloud/externalconn/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "externalconn",
srcs = [
"connection.go",
"connection_record.go",
"impl_registry.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/cloud/externalconn",
visibility = ["//visibility:public"],
deps = [
"//pkg/cloud/externalconn/connectionpb",
"//pkg/kv",
"//pkg/security/username",
"//pkg/sql/catalog/colinfo",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/sql/sqlutil",
"//pkg/util/protoutil",
"@com_github_cockroachdb_errors//:errors",
],
)
32 changes: 32 additions & 0 deletions pkg/cloud/externalconn/connection.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
// Copyright 2022 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 externalconn

import (
"context"
"net/url"

"github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb"
)

// ConnectionDetails is the interface to the external resource represented by an
// External Connection object.
type ConnectionDetails interface {
// ConnectionProto prepares the ConnectionDetails for serialization.
ConnectionProto() *connectionpb.ConnectionDetails
// ConnectionType returns the type of the connection.
ConnectionType() connectionpb.ConnectionType
}

// ConnectionDetailsFromURIFactory is the factory method that takes in an
// endpoint URI for an external resource, and returns a ConnectionDetails
// interface to interact with it.
type ConnectionDetailsFromURIFactory func(ctx context.Context, uri *url.URL) (ConnectionDetails, error)
Loading

0 comments on commit 066577b

Please sign in to comment.