Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
59490: add licensing info to README r=bdarnell a=gemma-shay

Added a section on licensing info.
Links to the new Licensing doc (#9338) will not work yet, so will wait to merge until they are up.
Closes cockroachdb/docs#7441

60513: sql: ensure REGIONAL BY ROW statements roundtrip r=ajstorm a=otan

Resolves #59362

See individual commits for details.

60552: sql: add descriptor validation on write r=postamar a=postamar

Previously, we didn't systematically validate descriptors when they were
written. Furthermore, there existed no common method to validate
descriptors across all descriptor subtypes

This commit adds three methods to the catalog.Descriptor interface:
  1. ValidateSelf ( context.Context ) error
  2. Validate ( context.Context, catalog.DescGetter ) error
  3. ValidateTxnCommit ( context.Context, catalog.DescGetter) error

Each performs a subset the checks performed by the next. ValidateSelf
contains all checks which can be performed in isolation, Validate also
performs all those involving DescGetters (i.e. cross-reference checks)
and ValidateTxnCommit also performs checks which should only be done at
commit-time. An example of the latter is checking that a table has
a primary key: dropping the PK is allowed within a transaction as long
as a new PK is subsequently provided before committing.

This commit adds new validation calls when writing descriptors:
  1. ValidateSelf is called prior to Collection adding a descriptor Put
     to a kv.Batch. At this point, we want descriptors to be at least
     internally-consistent, both to catch validation errors early and
     because it's not possible to do cross-reference checking at this
     point (changes on FKs for instance involve multiple descriptors).
  2. ValidateTxnCommit is called on the descs.Collection's uncommitted
     descriptors when the corresponding txn is about to commit, just
     prior to the two-version-invariant check.

These validations may be disabled using a new cluster setting:
    sql.catalog.descs.validate_on_write.enabled
Setting this to false makes it possible to corrupt the descriptor state
using the crdb_internal.unsafe_* functions.

Release note: None

60616: builtins: add builtin to retrieve the payload(s) for a span. r=knz,irfansharif a=angelapwen

Part of addressing #55733 

The `payloads_for_span` builtin retrieves all payloads for
a given span ID, given that the span is part of an active trace.
The payloads are returned in JSONB format. If the span is not
found, or if the span does not have any payloads, the builtin
returns an empty JSON object.

With the appropriate usage of this builtin and the
`crdb_internal.trace_id` builtin as shown in the `contention_event`
logic test, all payloads for the current trace may be surfaced.

Release note (sql change): add `payloads_for_span` builtin that
takes in a span ID and returns its paylods in JSONB format. If
the span is not found, or if the span does not have any payloads,
the builtin returns an empty JSON object.

60692: sql: add tests for inverted indexes on virtual columns r=mgartner a=mgartner

No code changes were needed to support inverted indexes on virtual
columns.

Release note: None

Co-authored-by: Gemma Shay <[email protected]>
Co-authored-by: Oliver Tan <[email protected]>
Co-authored-by: Marius Posta <[email protected]>
Co-authored-by: angelapwen <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
  • Loading branch information
6 people committed Feb 18, 2021
6 parents 6f34689 + c99c17c + 9a6b6be + bd0e202 + 96b24ae + d6c76a8 commit d67d6cc
Show file tree
Hide file tree
Showing 60 changed files with 1,129 additions and 423 deletions.
8 changes: 8 additions & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,14 @@ Guide](https://www.cockroachlabs.com/docs/stable/architecture/overview.html).
For the original design motivation, see our [design
doc](https://github.com/cockroachdb/cockroach/blob/master/docs/design.md).

## Licensing

Current CockroachDB code is released under a combination of two licenses, the [Business Source License (BSL)](https://www.cockroachlabs.com/docs/stable/licensing.html#bsl) and the [Cockroach Community License (CCL)](https://www.cockroachlabs.com/docs/stable/licensing.html#ccl-free).

When contributing to a CockroachDB feature, you can find the relevant license in the comments at the top of each file.

For more information, see the [Licensing FAQs](https://www.cockroachlabs.com/docs/stable/licensing-faqs.html).

## Comparison with Other Databases

To see how key features of CockroachDB stack up against other databases,
Expand Down
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -2638,6 +2638,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)</p>
</span></td></tr>
<tr><td><a name="crdb_internal.num_inverted_index_entries"></a><code>crdb_internal.num_inverted_index_entries(val: jsonb, version: <a href="int.html">int</a>) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.payloads_for_span"></a><code>crdb_internal.payloads_for_span(span ID: <a href="int.html">int</a>) &rarr; jsonb</code></td><td><span class="funcdesc"><p>Returns the payload(s) of the span whose ID is passed in the argument.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.pretty_key"></a><code>crdb_internal.pretty_key(raw_key: <a href="bytes.html">bytes</a>, skip_fields: <a href="int.html">int</a>) &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.range_stats"></a><code>crdb_internal.range_stats(key: <a href="bytes.html">bytes</a>) &rarr; jsonb</code></td><td><span class="funcdesc"><p>This function is used to retrieve range statistics information as a JSON object.</p>
Expand Down
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ ALL_TESTS = [
"//pkg/ccl/kvccl/kvfollowerreadsccl:kvfollowerreadsccl_test",
"//pkg/ccl/kvccl/kvtenantccl:kvtenantccl_test",
"//pkg/ccl/logictestccl:logictestccl_test",
"//pkg/ccl/multiregionccl:multiregionccl_test",
"//pkg/ccl/oidcccl:oidcccl_test",
"//pkg/ccl/partitionccl:partitionccl_test",
"//pkg/ccl/serverccl:serverccl_test",
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ go_library(
"//pkg/ccl/gssapiccl",
"//pkg/ccl/importccl",
"//pkg/ccl/kvccl",
"//pkg/ccl/multiregionccl",
"//pkg/ccl/oidcccl",
"//pkg/ccl/partitionccl",
"//pkg/ccl/storageccl",
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -426,7 +426,7 @@ func WriteDescriptors(
}

for _, db := range databases {
if err := db.Validate(); err != nil {
if err := db.Validate(ctx, dg); err != nil {
return errors.Wrapf(err,
"validate database %d", errors.Safe(db.GetID()))
}
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/ccl_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
_ "github.com/cockroachdb/cockroach/pkg/ccl/gssapiccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/importccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/oidcccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/avro_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ func parseTableDesc(createTableStmt string) (catalog.TableDescriptor, error) {
if err != nil {
return nil, err
}
return mutDesc, mutDesc.ValidateTable(ctx)
return mutDesc, mutDesc.ValidateSelf(ctx)
}

func parseValues(tableDesc catalog.TableDescriptor, values string) ([]rowenc.EncDatumRow, error) {
Expand Down
81 changes: 60 additions & 21 deletions pkg/ccl/logictestccl/testdata/logic_test/alter_table_locality
Original file line number Diff line number Diff line change
Expand Up @@ -1749,27 +1749,66 @@ DATABASE alter_locality_test ALTER DATABASE alter_locality_test CONFIGURE ZONE
voter_constraints = '[+region=ca-central-1]',
lease_preferences = '[[+region=ca-central-1]]'

# TODO(#60606): this is flaky because REGIONAL BY ROW CREATE TABLE statements
# do not round trip.

#statement ok
#CREATE TABLE regional_by_row_to_regional_by_row_as (
# pk INT PRIMARY KEY,
# i INT,
# cr crdb_internal_region NOT NULL DEFAULT 'ap-southeast-2',
# INDEX(i),
# FAMILY (pk, i)
#) LOCALITY REGIONAL BY ROW;
#INSERT INTO regional_by_row_to_regional_by_row_as (pk, i) VALUES (1, 1);
#ALTER TABLE regional_by_row_to_regional_by_row_as SET LOCALITY REGIONAL BY ROW AS "cr"

#query TT
#SHOW CREATE TABLE regional_by_row_to_regional_by_row_as
#----

#query TT
#SHOW ZONE CONFIGURATION FOR TABLE regional_by_row_to_regional_by_row_as
#----
statement ok
CREATE TABLE regional_by_row_to_regional_by_row_as (
pk INT PRIMARY KEY,
i INT,
cr crdb_internal_region NOT NULL DEFAULT 'ap-southeast-2',
INDEX(i),
FAMILY (pk, i)
) LOCALITY REGIONAL BY ROW;
INSERT INTO regional_by_row_to_regional_by_row_as (pk, i) VALUES (1, 1);
ALTER TABLE regional_by_row_to_regional_by_row_as SET LOCALITY REGIONAL BY ROW AS "cr"

query TT
SHOW CREATE TABLE regional_by_row_to_regional_by_row_as
----
regional_by_row_to_regional_by_row_as CREATE TABLE public.regional_by_row_to_regional_by_row_as (
pk INT8 NOT NULL,
i INT8 NULL,
cr public.crdb_internal_region NOT NULL DEFAULT 'ap-southeast-2':::public.crdb_internal_region,
crdb_region public.crdb_internal_region NOT VISIBLE NOT NULL DEFAULT default_to_database_primary_region(gateway_region())::public.crdb_internal_region,
CONSTRAINT "primary" PRIMARY KEY (pk ASC),
INDEX regional_by_row_to_regional_by_row_as_i_idx (i ASC),
FAMILY fam_0_pk_i_cr_crdb_region (pk, i, cr, crdb_region)
) LOCALITY REGIONAL BY ROW AS cr;
ALTER PARTITION "ap-southeast-2" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@primary CONFIGURE ZONE USING
num_voters = 3,
voter_constraints = '[+region=ap-southeast-2]',
lease_preferences = '[[+region=ap-southeast-2]]';
ALTER PARTITION "ap-southeast-2" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@regional_by_row_to_regional_by_row_as_i_idx CONFIGURE ZONE USING
num_voters = 3,
voter_constraints = '[+region=ap-southeast-2]',
lease_preferences = '[[+region=ap-southeast-2]]';
ALTER PARTITION "ca-central-1" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@primary CONFIGURE ZONE USING
num_voters = 3,
voter_constraints = '[+region=ca-central-1]',
lease_preferences = '[[+region=ca-central-1]]';
ALTER PARTITION "ca-central-1" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@regional_by_row_to_regional_by_row_as_i_idx CONFIGURE ZONE USING
num_voters = 3,
voter_constraints = '[+region=ca-central-1]',
lease_preferences = '[[+region=ca-central-1]]';
ALTER PARTITION "us-east-1" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@primary CONFIGURE ZONE USING
num_voters = 3,
voter_constraints = '[+region=us-east-1]',
lease_preferences = '[[+region=us-east-1]]';
ALTER PARTITION "us-east-1" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@regional_by_row_to_regional_by_row_as_i_idx CONFIGURE ZONE USING
num_voters = 3,
voter_constraints = '[+region=us-east-1]',
lease_preferences = '[[+region=us-east-1]]'

query TT
SHOW ZONE CONFIGURATION FOR TABLE regional_by_row_to_regional_by_row_as
----
DATABASE alter_locality_test ALTER DATABASE alter_locality_test CONFIGURE ZONE USING
range_min_bytes = 134217728,
range_max_bytes = 536870912,
gc.ttlseconds = 90000,
num_replicas = 5,
num_voters = 3,
constraints = '{+region=ap-southeast-2: 1, +region=ca-central-1: 1, +region=us-east-1: 1}',
voter_constraints = '[+region=ca-central-1]',
lease_preferences = '[[+region=ca-central-1]]'

# Altering from REGIONAL BY ROW AS

Expand Down
55 changes: 44 additions & 11 deletions pkg/ccl/logictestccl/testdata/logic_test/regional_by_row
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,6 @@ CREATE TABLE regional_by_row_table (
PARTITION BY LIST (pk) (PARTITION one VALUES IN ((1)))
LOCALITY REGIONAL BY ROW

statement error cannot specify crdb_region column in REGIONAL BY ROW table as the column is implicitly created by the system
CREATE TABLE regional_by_row_table (
pk int,
a int,
crdb_region crdb_internal_region
)
LOCALITY REGIONAL BY ROW

statement error REGIONAL BY ROW on a TABLE containing PARTITION BY is not supported
CREATE TABLE regional_by_row_table (
pk int
Expand Down Expand Up @@ -72,6 +64,46 @@ CREATE TABLE regional_by_row_table (
)
LOCALITY REGIONAL BY ROW

statement error cannot use column crdb_region which has type INT8 in REGIONAL BY ROW\nDETAIL: Column crdb_internal_region must be of type crdb_internal_region
CREATE TABLE regional_by_row_table (
pk int,
a int,
crdb_region int
)
LOCALITY REGIONAL BY ROW

statement ok
CREATE TABLE regional_by_row_table_explicit_crdb_region_column (
pk int PRIMARY KEY,
a int,
crdb_region crdb_internal_region,
FAMILY (pk, a, crdb_region)
)
LOCALITY REGIONAL BY ROW

query T
SELECT create_statement FROM [SHOW CREATE TABLE regional_by_row_table_explicit_crdb_region_column]
----
CREATE TABLE public.regional_by_row_table_explicit_crdb_region_column (
pk INT8 NOT NULL,
a INT8 NULL,
crdb_region public.crdb_internal_region NOT NULL,
CONSTRAINT "primary" PRIMARY KEY (pk ASC),
FAMILY fam_0_pk_a_crdb_region (pk, a, crdb_region)
) LOCALITY REGIONAL BY ROW;
ALTER PARTITION "ap-southeast-2" OF INDEX multi_region_test_db.public.regional_by_row_table_explicit_crdb_region_column@primary CONFIGURE ZONE USING
num_voters = 5,
voter_constraints = '{+region=ap-southeast-2: 2}',
lease_preferences = '[[+region=ap-southeast-2]]';
ALTER PARTITION "ca-central-1" OF INDEX multi_region_test_db.public.regional_by_row_table_explicit_crdb_region_column@primary CONFIGURE ZONE USING
num_voters = 5,
voter_constraints = '{+region=ca-central-1: 2}',
lease_preferences = '[[+region=ca-central-1]]';
ALTER PARTITION "us-east-1" OF INDEX multi_region_test_db.public.regional_by_row_table_explicit_crdb_region_column@primary CONFIGURE ZONE USING
num_voters = 5,
voter_constraints = '{+region=us-east-1: 2}',
lease_preferences = '[[+region=us-east-1]]'

statement ok
CREATE TABLE regional_by_row_table (
pk int PRIMARY KEY,
Expand Down Expand Up @@ -169,8 +201,9 @@ regional_by_row_table_j_idx j false
query TTTTIT colnames
SHOW TABLES
----
schema_name table_name type owner estimated_row_count locality
public regional_by_row_table table root 0 REGIONAL BY ROW
schema_name table_name type owner estimated_row_count locality
public regional_by_row_table table root 0 REGIONAL BY ROW
public regional_by_row_table_explicit_crdb_region_column table root 0 REGIONAL BY ROW

query TI
INSERT INTO regional_by_row_table (pk, pk2, a, b, j) VALUES
Expand Down Expand Up @@ -843,7 +876,7 @@ ALTER PARTITION "us-east-1" OF INDEX multi_region_test_db.public.regional_by_row
lease_preferences = '[[+region=us-east-1]]'

# Tests for REGIONAL BY TABLE AS
statement error cannot use column crdb_region_col which has type INT8 in REGIONAL BY ROW AS\nDETAIL:\s+REGIONAL BY ROW AS must reference a column of type crdb_internal_region.
statement error cannot use column crdb_region_col which has type INT8 in REGIONAL BY ROW\nDETAIL: REGIONAL BY ROW AS must reference a column of type crdb_internal_region
CREATE TABLE regional_by_row_table_as (
pk int PRIMARY KEY,
crdb_region_col int
Expand Down
42 changes: 42 additions & 0 deletions pkg/ccl/multiregionccl/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "multiregionccl",
srcs = ["multiregion.go"],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl",
visibility = ["//visibility:public"],
)

go_test(
name = "multiregionccl_test",
srcs = [
"main_test.go",
"regional_by_row_test.go",
"show_test.go",
],
deps = [
"//pkg/base",
"//pkg/ccl/partitionccl",
"//pkg/ccl/utilccl",
"//pkg/jobs",
"//pkg/keys",
"//pkg/roachpb",
"//pkg/security",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/sql",
"//pkg/sql/catalog/catalogkv",
"//pkg/sql/execinfra",
"//pkg/sql/sqltestutils",
"//pkg/sql/tests",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/randutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
],
)
33 changes: 33 additions & 0 deletions pkg/ccl/multiregionccl/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
// Copyright 2021 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 multiregionccl_test

import (
"os"
"testing"

"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/security"
"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) {
defer utilccl.TestingEnableEnterprise()()
security.SetAssetLoader(securitytest.EmbeddedAssets)
randutil.SeedForTests()
serverutils.InitTestServerFactory(server.TestServerFactory)
serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)
os.Exit(m.Run())
}
9 changes: 9 additions & 0 deletions pkg/ccl/multiregionccl/multiregion.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,9 @@
// Copyright 2021 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 multiregionccl
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package partitionccl_test
package multiregionccl_test

import (
"context"
Expand All @@ -32,7 +32,7 @@ import (
"github.com/stretchr/testify/require"
)

// REGIONAL BY ROW tests are defined in partitionccl as REGIONAL BY ROW
// REGIONAL BY ROW tests are defined in multiregionccl as REGIONAL BY ROW
// requires CCL to operate.

// TestAlterTableLocalityRegionalByRowError tests an alteration involving
Expand Down
Loading

0 comments on commit d67d6cc

Please sign in to comment.