Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
59441: streamingccl: improvements to the random stream test client r=pbardea a=adityamaru

This change improves on the random stream client to allow for better
testing of the various components of the stream ingestion job.
Specifically:

- Adds support for specifying number of partitions. For simplicity,
  a partition generates KVs for a particular table span.

- Generates system KVs (descriptor and namespace) KVs, as the first two
  KVs on the partition stream. I played around with the idea of having a
separate "system" and "table data" partition, but the code and tests
became more convoluted, compared to the current approach.

- Hookup the CDC orderValidator to the random stream client's output.
  This gives us some guarantees that the data being generated is
semantically correct.

- Maintain an in-memory copy of all the streamed events, that can be
  efficiently queried. This allows us to compare the ingested KVs to the
streamed KVs and gain more confidence in our pipeline.

Infroms: #59175

Release note: None

59621: pgwire: set options based on "options" URL parameter r=rafiss a=mneverov

pgwire: set options based on "options" URL parameter

Previously, CRDB ignored "options" URL parameter. User session parameters should
have been set via URL parameters directly:
`postgres://user@host:port/database?serial_normalization=virtual_sequence`

CRDB can now parse "options" URL parameter and set corresponding session
parameters (in compliance with Postgres jdbc connection parameters):
`postgres://user@host:port/database?options=-c%20serial_normalization=virtual_sequence`

Fixes #59404

Release note (sql change): CockroachDB now recognizes "options" URL parameter.

59781: sql,metrics: do not increment ROLLBACK counter if in CommitWait r=arulajmani a=rafiss

fixes #50780 

Release note (bug fix): Previously if `RELEASE SAVEPOINT cockroach_restart`
was followed by `ROLLBACK`, the `sql.txn.rollback.count`
metric would be incremented. This was incorrect, since the txn had already
committed. Now that metric is not incremented in this case.

Co-authored-by: Aditya Maru <adityamaru@gmail.com>
Co-authored-by: Max Neverov <neverov.max@gmail.com>
Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
  • Loading branch information
4 people committed Feb 9, 2021
4 parents f780c20 + 8247919 + dd2512c + 6b60a69 commit 4086c3e
Show file tree
Hide file tree
Showing 17 changed files with 934 additions and 140 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/changefeedccl/cdctest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ go_library(
deps = [
"//pkg/jobs",
"//pkg/jobs/jobspb",
"//pkg/roachpb",
"//pkg/sql",
"//pkg/sql/parser",
"//pkg/sql/sem/tree",
Expand Down
112 changes: 93 additions & 19 deletions pkg/ccl/changefeedccl/cdctest/validator.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"sort"
"strings"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/errors"
Expand All @@ -32,15 +33,83 @@ type Validator interface {
Failures() []string
}

// StreamClientValidatorWrapper wraps a Validator and exposes additional methods
// used by stream ingestion to check for correctness.
type StreamClientValidatorWrapper interface {
GetValuesForKeyBelowTimestamp(key string, timestamp hlc.Timestamp) ([]roachpb.KeyValue, error)
GetValidator() Validator
}

type streamValidator struct {
Validator
}

var _ StreamClientValidatorWrapper = &streamValidator{}

// NewStreamClientValidatorWrapper returns a wrapped Validator, that can be used
// to validate the events emitted by the cluster to cluster streaming client.
// The wrapper currently only "wraps" an orderValidator, but can be built out
// to utilize other Validator's.
// The wrapper also allows querying the orderValidator to retrieve streamed
// events from an in-memory store.
func NewStreamClientValidatorWrapper() StreamClientValidatorWrapper {
ov := NewOrderValidator("unusedC2C")
return &streamValidator{
ov,
}
}

// GetValidator implements the StreamClientValidatorWrapper interface.
func (sv *streamValidator) GetValidator() Validator {
return sv.Validator
}

// GetValuesForKeyBelowTimestamp implements the StreamClientValidatorWrapper
// interface.
// It returns the streamed KV updates for `key` with a ts less than equal to
// `timestamp`.
func (sv *streamValidator) GetValuesForKeyBelowTimestamp(
key string, timestamp hlc.Timestamp,
) ([]roachpb.KeyValue, error) {
orderValidator, ok := sv.GetValidator().(*orderValidator)
if !ok {
return nil, errors.Newf("unknown validator %T: ", sv.GetValidator())
}
timestampValueTuples := orderValidator.keyTimestampAndValues[key]
timestampsIdx := sort.Search(len(timestampValueTuples), func(i int) bool {
return timestamp.Less(timestampValueTuples[i].ts)
})
var kv []roachpb.KeyValue
for _, tsValue := range timestampValueTuples[:timestampsIdx] {
byteRep := []byte(key)
kv = append(kv, roachpb.KeyValue{
Key: byteRep,
Value: roachpb.Value{
RawBytes: []byte(tsValue.value),
Timestamp: tsValue.ts,
},
})
}

return kv, nil
}

type timestampValue struct {
ts hlc.Timestamp
value string
}

type orderValidator struct {
topic string
partitionForKey map[string]string
keyTimestamps map[string][]hlc.Timestamp
resolved map[string]hlc.Timestamp
topic string
partitionForKey map[string]string
keyTimestampAndValues map[string][]timestampValue
resolved map[string]hlc.Timestamp

failures []string
}

var _ Validator = &orderValidator{}

// NewOrderValidator returns a Validator that checks the row and resolved
// timestamp ordering guarantees. It also asserts that keys have an affinity to
// a single partition.
Expand All @@ -52,17 +121,15 @@ type orderValidator struct {
// lower update timestamp will be emitted on that partition.
func NewOrderValidator(topic string) Validator {
return &orderValidator{
topic: topic,
partitionForKey: make(map[string]string),
keyTimestamps: make(map[string][]hlc.Timestamp),
resolved: make(map[string]hlc.Timestamp),
topic: topic,
partitionForKey: make(map[string]string),
keyTimestampAndValues: make(map[string][]timestampValue),
resolved: make(map[string]hlc.Timestamp),
}
}

// NoteRow implements the Validator interface.
func (v *orderValidator) NoteRow(
partition string, key, ignoredValue string, updated hlc.Timestamp,
) error {
func (v *orderValidator) NoteRow(partition string, key, value string, updated hlc.Timestamp) error {
if prev, ok := v.partitionForKey[key]; ok && prev != partition {
v.failures = append(v.failures, fmt.Sprintf(
`key [%s] received on two partitions: %s and %s`, key, prev, partition,
Expand All @@ -71,17 +138,20 @@ func (v *orderValidator) NoteRow(
}
v.partitionForKey[key] = partition

timestamps := v.keyTimestamps[key]
timestampsIdx := sort.Search(len(timestamps), func(i int) bool {
return updated.LessEq(timestamps[i])
timestampValueTuples := v.keyTimestampAndValues[key]
timestampsIdx := sort.Search(len(timestampValueTuples), func(i int) bool {
return updated.LessEq(timestampValueTuples[i].ts)
})
seen := timestampsIdx < len(timestamps) && timestamps[timestampsIdx] == updated
seen := timestampsIdx < len(timestampValueTuples) &&
timestampValueTuples[timestampsIdx].ts == updated

if !seen && len(timestamps) > 0 && updated.Less(timestamps[len(timestamps)-1]) {
if !seen && len(timestampValueTuples) > 0 &&
updated.Less(timestampValueTuples[len(timestampValueTuples)-1].ts) {
v.failures = append(v.failures, fmt.Sprintf(
`topic %s partition %s: saw new row timestamp %s after %s was seen`,
v.topic, partition,
updated.AsOfSystemTime(), timestamps[len(timestamps)-1].AsOfSystemTime(),
updated.AsOfSystemTime(),
timestampValueTuples[len(timestampValueTuples)-1].ts.AsOfSystemTime(),
))
}
if !seen && updated.Less(v.resolved[partition]) {
Expand All @@ -92,8 +162,12 @@ func (v *orderValidator) NoteRow(
}

if !seen {
v.keyTimestamps[key] = append(
append(timestamps[:timestampsIdx], updated), timestamps[timestampsIdx:]...)
v.keyTimestampAndValues[key] = append(
append(timestampValueTuples[:timestampsIdx], timestampValue{
ts: updated,
value: value,
}),
timestampValueTuples[timestampsIdx:]...)
}
return nil
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/ccl/streamingccl/addresses.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,11 @@ func (sa StreamAddress) URL() (*url.URL, error) {
// Each partition will emit events for a fixed span of keys.
type PartitionAddress string

// URL parses the partition address as a URL.
func (pa PartitionAddress) URL() (*url.URL, error) {
return url.Parse(string(pa))
}

// Topology is a configuration of stream partitions. These are particular to a
// stream. It specifies the number and addresses of partitions of the stream.
//
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/streamingccl/streamclient/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,12 +14,14 @@ go_library(
"//pkg/keys",
"//pkg/roachpb",
"//pkg/sql",
"//pkg/sql/catalog/catalogkeys",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/rowenc",
"//pkg/sql/sem/tree",
"//pkg/util/hlc",
"//pkg/util/randutil",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
],
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/streamingccl/streamclient/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ func (sc testStreamClient) GetTopology(

// ConsumePartition implements the Client interface.
func (sc testStreamClient) ConsumePartition(
_ context.Context, _ streamingccl.PartitionAddress, _ time.Time,
_ context.Context, pa streamingccl.PartitionAddress, _ time.Time,
) (chan streamingccl.Event, error) {
sampleKV := roachpb.KeyValue{
Key: []byte("key_1"),
Expand Down
Loading

0 comments on commit 4086c3e

Please sign in to comment.