Skip to content

Commit

Permalink
externalconn,changefeedccl: support kafka in external connections
Browse files Browse the repository at this point in the history
This change adds the ability to create an external connection that
rerpresents a `kafka` sink. It also teaches changefeeds to recognize
the `external` schema URI.

When creating an external connection that represents a kafka sink, we
run validation on the passed in URI before persisting it in the system
table. This PR does not add any `WITH` options to the create statement
but in the near future we will want to allow the user to pass in certain
sink/resource specific configurations that will apply to all users of the
external connection object. For example, a kafka JSON config.

A changefeed can now be run to an `external` scheme URI that points to
an existing external connection object. Since we can only represent kafka
sinks as external connections as of now, the statement will only accept the
options that are valid for a kafka sink.

Informs: cockroachdb#84753

Release note (sql change): Users can now `CREATE EXTERNAL CONNECTION`
to represent a `kafka` sink. Subsequently, users can run
`CREATE CHANGEFEED` with an `external:///<external-connection-object-name`
URI as the sink to use the kafka resource represented by the
external connection object.
  • Loading branch information
adityamaru committed Aug 1, 2022
1 parent 12daec6 commit 3ed3fe0
Show file tree
Hide file tree
Showing 18 changed files with 490 additions and 63 deletions.
6 changes: 6 additions & 0 deletions pkg/ccl/changefeedccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,9 @@ go_library(
"scram_client.go",
"sink.go",
"sink_cloudstorage.go",
"sink_external_connection.go",
"sink_kafka.go",
"sink_kafka_connection.go",
"sink_pubsub.go",
"sink_sql.go",
"sink_webhook.go",
Expand All @@ -45,6 +47,8 @@ go_library(
"//pkg/ccl/changefeedccl/schemafeed",
"//pkg/ccl/utilccl",
"//pkg/cloud",
"//pkg/cloud/externalconn",
"//pkg/cloud/externalconn/connectionpb",
"//pkg/clusterversion",
"//pkg/docs",
"//pkg/featureflag",
Expand Down Expand Up @@ -89,6 +93,7 @@ go_library(
"//pkg/sql/sem/eval",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondatapb",
"//pkg/sql/sqlutil",
"//pkg/sql/types",
"//pkg/util/bitarray",
"//pkg/util/bufalloc",
Expand Down Expand Up @@ -151,6 +156,7 @@ go_test(
"schema_registry_test.go",
"show_changefeed_jobs_test.go",
"sink_cloudstorage_test.go",
"sink_kafka_connection_test.go",
"sink_test.go",
"sink_webhook_test.go",
"testfeed_test.go",
Expand Down
14 changes: 14 additions & 0 deletions pkg/ccl/changefeedccl/changefeedbase/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -187,6 +187,7 @@ const (
SinkSchemeNull = `null`
SinkSchemeWebhookHTTP = `webhook-http`
SinkSchemeWebhookHTTPS = `webhook-https`
SinkSchemeExternalConnection = `external`
SinkParamSASLEnabled = `sasl_enabled`
SinkParamSASLHandshake = `sasl_handshake`
SinkParamSASLUser = `sasl_user`
Expand Down Expand Up @@ -340,6 +341,19 @@ var WebhookValidOptions = makeStringSet(OptWebhookAuthHeader, OptWebhookClientTi
// PubsubValidOptions is options exclusive to pubsub sink
var PubsubValidOptions = makeStringSet()

// ExternalConnectionValidOptions is options exclusive to the external
// connection sink.
//
// TODO(adityamaru): Some of these options should be supported when creating the
// external connection rather than when setting up the changefeed. Move them once
// we support `CREATE EXTERNAL CONNECTION ... WITH <options>`.
var ExternalConnectionValidOptions = makeStringSet(
// Options valid for a kafka sink.
OptAvroSchemaPrefix,
OptConfluentSchemaRegistry,
OptKafkaSinkConfig,
)

// CaseInsensitiveOpts options which supports case Insensitive value
var CaseInsensitiveOpts = makeStringSet(OptFormat, OptEnvelope, OptCompression, OptSchemaChangeEvents,
OptSchemaChangePolicy, OptOnError, OptInitialScan)
Expand Down
17 changes: 17 additions & 0 deletions pkg/ccl/changefeedccl/sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,16 @@ type SinkWithTopics interface {
Topics() []string
}

// SinkContext contains the dependencies passed to sink implementations during
// creation.
type SinkContext struct {
targets changefeedbase.Targets
mb metricsRecorderBuilder

// Kafka specific configurations.
kafkaJSONConfig changefeedbase.SinkSpecificJSONConfig
}

func getEventSink(
ctx context.Context,
serverCfg *execinfra.ServerConfig,
Expand Down Expand Up @@ -190,6 +200,13 @@ func getSink(
return validateOptionsAndMakeSink(changefeedbase.SQLValidOptions, func() (Sink, error) {
return makeSQLSink(sinkURL{URL: u}, sqlSinkTableName, AllTargets(feedCfg), metricsBuilder)
})
case u.Scheme == changefeedbase.SinkSchemeExternalConnection:
return validateOptionsAndMakeSink(changefeedbase.ExternalConnectionValidOptions, func() (Sink, error) {
sinkCtx := SinkContext{targets: AllTargets(feedCfg), mb: metricsBuilder,
kafkaJSONConfig: opts.GetKafkaConfigJSON()}
return makeExternalConnectionSink(ctx, sinkURL{URL: u}, user, serverCfg.DB,
serverCfg.Executor, sinkCtx)
})
case u.Scheme == "":
return nil, errors.Errorf(`no scheme found for sink URL %q`, feedCfg.SinkURI)
default:
Expand Down
70 changes: 70 additions & 0 deletions pkg/ccl/changefeedccl/sink_external_connection.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
// 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 changefeedccl

import (
"context"

"github.com/cockroachdb/cockroach/pkg/cloud/externalconn"
"github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/errors"
)

func makeExternalConnectionSink(
ctx context.Context,
u sinkURL,
user username.SQLUsername,
db *kv.DB,
ie sqlutil.InternalExecutor,
sinkCtx SinkContext,
) (Sink, error) {
if u.Host == "" {
return nil, errors.Newf("host component of an external URI must refer to an "+
"existing External Connection object: %s", u.String())
}

externalConnectionName := u.Host

// TODO(adityamaru): Use the `user` in `cfg` to perform privilege checks on
// the external connection object we are about to retrieve.

// Retrieve the external connection object from the system table.
var ec *externalconn.ExternalConnection
if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
var err error
ec, err = externalconn.LoadExternalConnection(ctx, externalConnectionName,
connectionpb.TypeStorage, ie, user, txn)
return err
}); err != nil {
return nil, errors.Wrap(err, "failed to load external connection object")
}

// Construct a Sink handle for the underlying resource represented by the
// external connection object.
details := ec.ConnectionDetails()
connDetails, err := externalconn.MakeConnectionDetails(ctx, *details)
if err != nil {
return nil, err
}
connection, err := connDetails.Dial(ctx, sinkCtx, "" /* subdir */)
if err != nil {
return nil, errors.Wrap(err, "failed to Dial external connection")
}

var sink Sink
var ok bool
if sink, ok = connection.(Sink); !ok {
return nil, errors.AssertionFailedf("cannot convert Connection to Sink")
}

return sink, nil
}
88 changes: 88 additions & 0 deletions pkg/ccl/changefeedccl/sink_kafka_connection.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// 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 changefeedccl

import (
"context"
"net/url"

"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/cloud/externalconn"
"github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb"
"github.com/cockroachdb/errors"
)

func parseAndValidateKafkaSinkURI(
ctx context.Context, uri *url.URL,
) (connectionpb.ConnectionDetails, error) {
// Validate the kafka URI by creating a kafka sink.
//
// TODO(adityamaru): Add `CREATE EXTERNAL CONNECTION ... WITH` support to
// accept JSONConfig.
_, err := makeKafkaSink(ctx, sinkURL{URL: uri}, changefeedbase.Targets{}, "", nilMetricsRecorderBuilder)
if err != nil {
return connectionpb.ConnectionDetails{}, errors.Wrap(err, "invalid Kafka URI")
}

connDetails := connectionpb.ConnectionDetails{
Provider: connectionpb.ConnectionProvider_kafka,
Details: &connectionpb.ConnectionDetails_Kafka{
Kafka: &connectionpb.KafkaConnectionDetails{
URI: uri.String(),
},
},
}
return connDetails, nil
}

type kafkaConnectionDetails struct {
connectionpb.ConnectionDetails
}

// Dial implements the ConnectionDetails interface.
func (k *kafkaConnectionDetails) Dial(
ctx context.Context, connectionCtx interface{}, _ string,
) (externalconn.Connection, error) {
sinkCtx, ok := connectionCtx.(SinkContext)
if !ok {
return nil, errors.Newf("Kafka sink dialed with an incompatible context of type %T", connectionCtx)
}
uri, err := url.Parse(k.GetKafka().URI)
if err != nil {
return nil, errors.New("failed to parse kafka URI when dialing external connection")
}

// TODO(adityamaru): Currently, we're getting the kafkaJSONConfig from the
// `CREATE CHANGEFEED` statement but we might consider moving this option to a
// `CREATE EXTERNAL CONNECTION` option in the future.
return makeKafkaSink(ctx, sinkURL{URL: uri}, sinkCtx.targets, sinkCtx.kafkaJSONConfig, sinkCtx.mb)
}

// ConnectionProto implements the ConnectionDetails interface.
func (k *kafkaConnectionDetails) ConnectionProto() *connectionpb.ConnectionDetails {
return &k.ConnectionDetails
}

// ConnectionType implements the ConnectionDetails interface.
func (k *kafkaConnectionDetails) ConnectionType() connectionpb.ConnectionType {
return k.ConnectionDetails.Type()
}

var _ externalconn.ConnectionDetails = &kafkaConnectionDetails{}

func makeKafkaSinkConnectionDetails(
_ context.Context, details connectionpb.ConnectionDetails,
) externalconn.ConnectionDetails {
return &kafkaConnectionDetails{ConnectionDetails: details}
}

func init() {
externalconn.RegisterConnectionDetailsFromURIFactory(connectionpb.ConnectionProvider_kafka,
changefeedbase.SinkSchemeKafka, parseAndValidateKafkaSinkURI, makeKafkaSinkConnectionDetails)
}
Loading

0 comments on commit 3ed3fe0

Please sign in to comment.