Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
46992: sql: Add Logical Column ID field to ColumnDescriptor r=rohany a=RichardJCai

The LogicalColumnID field mimics the ColumnID field however LogicalColumnID may be swapped
between two columns whereas ColumnID cannot. LogicalColumnID is referenced for virtual tables
(pg_catalog, information_schema) and most notably affects column ordering for SHOW COLUMNS.

This LogicalColumnID field support swapping the order of two columns - currently only used for
ALTER COLUMN TYPE when a shadow column is created and swapped with it's original column.

Does not affect existing behaviour.

Release note: None

47449: cli: add --cert-principal-map to client commands r=petermattis a=petermattis

Add support for the `--cert-principal-map` flag to the certs and client
commands. Anywhere we were accepting the `--certs-dir` flag, we now also
accept the `--cert-principal-map` flag.

Fixes #47300

Release note (cli change): Support the `--cert-principal-map` flag in
the `cert *` and "client" commands such as `sql`.

48138: keys: support splitting Ranges on tenant-id prefixed keys r=nvanbenschoten a=nvanbenschoten

Fixes #48122.
Relates to #47903.
Relates to #48123.

This PR contains a series of small commits that work towards the introduction of tenant-id prefixed keyspaces and begin the removal of some `keys.TODOSQLCodec` instances. This should be the only time we need to touch C++ throughout this work.

48160: storage,libroach: Check for MaxKeys when reading from intent history r=itsbilal a=itsbilal

We weren't checking for MaxKeys (or TargetBytes) being reached
in the case where we read from intent history in the MVCC scanner.
All other cases go through addAndAdvance(), which had these checks.

Almost certainly fixes #46652. Would be very surprised if it was
something else.

Release note (bug fix): Fixes a bug where a read operation in a transaction
would error out for exceeding the maximum count of results returned.

48162: opt: add rule to eliminate Exists when input has zero rows r=rytaft a=rytaft

This commit adds a new rule, `EliminateExistsZeroRows`, which
converts an `Exists` subquery to False when it's known
that the input produces zero rows.

Informs #47058

Release note (performance improvement): The optimizer can now
detect when an Exists subquery can be eliminated because the input
has zero rows. This leads to better plans in some cases.

Co-authored-by: richardjcai <caioftherichard@gmail.com>
Co-authored-by: Peter Mattis <petermattis@gmail.com>
Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Co-authored-by: Bilal Akhtar <bilal@cockroachlabs.com>
Co-authored-by: Rebecca Taft <becca@cockroachlabs.com>
  • Loading branch information
6 people committed Apr 29, 2020
6 parents 5b1d806 + 3afec70 + c360bd4 + 493fc81 + 3628ed9 + 67a1d4d commit 456d01b
Show file tree
Hide file tree
Showing 31 changed files with 825 additions and 411 deletions.
18 changes: 17 additions & 1 deletion c-deps/libroach/encoding.cc
Original file line number Diff line number Diff line change
Expand Up @@ -276,7 +276,23 @@ WARN_UNUSED_RESULT bool IsInt(rocksdb::Slice* buf) {
return false;
}

WARN_UNUSED_RESULT bool DecodeTablePrefix(rocksdb::Slice* buf, uint64_t* tbl) {
WARN_UNUSED_RESULT bool StripTenantPrefix(rocksdb::Slice* buf) {
if (buf->size() == 0) {
return true;
}
// kTenantPrefix is guaranteed to be a single byte.
if ((*buf)[0] != kTenantPrefix[0]) {
return true;
}
buf->remove_prefix(1);
uint64_t tid;
return DecodeUvarint64(buf, &tid);
}

WARN_UNUSED_RESULT bool DecodeTenantAndTablePrefix(rocksdb::Slice* buf, uint64_t* tbl) {
if (!StripTenantPrefix(buf)) {
return false;
}
if (!IsInt(buf) || !DecodeUvarint64(buf, tbl)) {
return false;
}
Expand Down
14 changes: 8 additions & 6 deletions c-deps/libroach/encoding.h
Original file line number Diff line number Diff line change
Expand Up @@ -118,11 +118,13 @@ rocksdb::Slice KeyPrefix(const rocksdb::Slice& src);
// if it is of type int.
WARN_UNUSED_RESULT bool IsInt(rocksdb::Slice* buf);

// DecodeTablePrefix validates that the given key has a table prefix. On
// completion, buf holds the remainder of the key (with the prefix removed) and
// tbl stores the decoded descriptor ID of the table.
//
// TODO(nvanbenschoten): support tenant ID prefix.
WARN_UNUSED_RESULT bool DecodeTablePrefix(rocksdb::Slice* buf, uint64_t* tbl);
// StripTenantPrefix validates that the given key has a tenant prefix. On
// completion, buf holds the remainder of the key (with the prefix removed).
WARN_UNUSED_RESULT bool StripTenantPrefix(rocksdb::Slice* buf);

// DecodeTenantAndTablePrefix validates that the given key has a tenant and
// table prefix. On completion, buf holds the remainder of the key (with the
// prefix removed) and tbl stores the decoded descriptor ID of the table.
WARN_UNUSED_RESULT bool DecodeTenantAndTablePrefix(rocksdb::Slice* buf, uint64_t* tbl);

} // namespace cockroach
32 changes: 32 additions & 0 deletions c-deps/libroach/encoding_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -76,3 +76,35 @@ TEST(Libroach, Encoding) {
EXPECT_EQ(*it, out);
}
}

TEST(Libroach, DecodeTenantAndTablePrefix) {
// clang-format off
std::vector<std::pair<std::string, uint64_t>> cases{
{{'\x89'}, 1LLU},
{{'\xf6', '\xff'}, 255LLU},
{{'\xf7', '\xff', '\xff'}, 65535LLU},
{{'\xf8', '\xff', '\xff', '\xff'}, 16777215LLU},
{{'\xf9', '\xff', '\xff', '\xff', '\xff'}, 4294967295LLU},
{{'\xfa', '\xff', '\xff', '\xff', '\xff', '\xff'}, 1099511627775LLU},
{{'\xfb', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff'}, 281474976710655LLU},
{{'\xfc', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff'}, 72057594037927935LLU},
{{'\xfd', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff'}, 18446744073709551615LLU},
{{'\xfe', '\x8d', '\x89'}, 1LLU},
{{'\xfe', '\x8d', '\xf6', '\xff'}, 255LLU},
{{'\xfe', '\x8d', '\xf7', '\xff', '\xff'}, 65535LLU},
{{'\xfe', '\x8d', '\xf8', '\xff', '\xff', '\xff'}, 16777215LLU},
{{'\xfe', '\x8d', '\xf9', '\xff', '\xff', '\xff', '\xff'}, 4294967295LLU},
{{'\xfe', '\x8d', '\xfa', '\xff', '\xff', '\xff', '\xff', '\xff'}, 1099511627775LLU},
{{'\xfe', '\x8d', '\xfb', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff'}, 281474976710655LLU},
{{'\xfe', '\x8d', '\xfc', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff'}, 72057594037927935LLU},
{{'\xfe', '\x8d', '\xfd', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff', '\xff'}, 18446744073709551615LLU},
};
// clang-format on

for (auto it = cases.begin(); it != cases.end(); it++) {
rocksdb::Slice slice(it->first);
uint64_t tbl = 0;
EXPECT_TRUE(DecodeTenantAndTablePrefix(&slice, &tbl));
EXPECT_EQ(it->second, tbl);
}
}
1 change: 1 addition & 0 deletions c-deps/libroach/keys.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ const rocksdb::Slice kLocalRangeIDPrefix("\x01\x69", 2);
const rocksdb::Slice kLocalRangeIDReplicatedInfix("\x72", 1);
const rocksdb::Slice kLocalRangeAppliedStateSuffix("\x72\x61\x73\x6b", 4);
const rocksdb::Slice kMeta2KeyMax("\x03\xff\xff", 3);
const rocksdb::Slice kTenantPrefix("\xfe", 1);
const rocksdb::Slice kMinKey("", 0);
const rocksdb::Slice kMaxKey("\xff\xff", 2);

Expand Down
6 changes: 6 additions & 0 deletions c-deps/libroach/mvcc.h
Original file line number Diff line number Diff line change
Expand Up @@ -407,6 +407,12 @@ template <bool reverse> class mvccScanner {
// sequence, read that value.
const bool found = getFromIntentHistory();
if (found) {
if (target_bytes_ > 0 && kvs_->NumBytes() >= target_bytes_) {
max_keys_ = kvs_->Count();
}
if (max_keys_ > 0 && kvs_->Count() == max_keys_) {
return false;
}
return advanceKey();
}
// 11. If no value in the intent history has a sequence number equal to
Expand Down
46 changes: 26 additions & 20 deletions c-deps/libroach/row_counter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -17,40 +17,46 @@ using namespace cockroach;
int RowCounter::GetRowPrefixLength(rocksdb::Slice* key) {
size_t n = key->size();

if (!IsInt(key)) {
// Strip tenant ID prefix to get a "SQL key" starting with a table ID.
rocksdb::Slice buf = rocksdb::Slice(*key);
if (!StripTenantPrefix(&buf)) {
return 0;
}
size_t sql_n = key->size();

if (!IsInt(&buf)) {
// Not a table key, so the row prefix is the entire key.
return n;
}

// The column ID length is encoded as a varint and we take advantage of the
// fact that the column ID itself will be encoded in 0-9 bytes and thus the
// length of the column ID data will fit in a single byte.
rocksdb::Slice buf = rocksdb::Slice(*key);
buf.remove_prefix(n - 1);
// The column family ID length is encoded as a varint and we take advantage of
// the fact that the column family ID itself will be encoded in 0-9 bytes and
// thus the length of the column family ID data will fit in a single byte.
buf.remove_prefix(sql_n - 1);

if (!IsInt(&buf)) {
// The last byte is not a valid column ID suffix.
// The last byte is not a valid column family ID suffix.
return 0;
}

uint64_t col_id_len;
if (!DecodeUvarint64(&buf, &col_id_len)) {
uint64_t col_fam_id_len;
if (!DecodeUvarint64(&buf, &col_fam_id_len)) {
return 0;
}

if (col_id_len > uint64_t(n - 1)) {
// The column ID length was impossible. colIDLen is the length of
// the encoded column ID suffix. We add 1 to account for the byte
// holding the length of the encoded column ID and if that total
// (colIDLen+1) is greater than the key suffix (n == len(buf))
// then we bail. Note that we don't consider this an error because
// EnsureSafeSplitKey can be called on keys that look like table
// keys but which do not have a column ID length suffix (e.g
// by SystemConfig.ComputeSplitKey).
if (col_fam_id_len > uint64_t(sql_n - 1)) {
// The column family ID length was impossible. colFamIDLen is the length of
// the encoded column family ID suffix. We add 1 to account for the byte
// holding the length of the encoded column family ID and if that total
// (colFamIDLen+1) is greater than the key suffix (sqlN == len(sqlKey)) then
// we bail. Note that we don't consider this an error because
// EnsureSafeSplitKey can be called on keys that look like table keys but
// which do not have a column family ID length suffix (e.g by
// SystemConfig.ComputeSplitKey).
return 0;
}

return n - int(col_id_len) - 1;
return n - int(col_fam_id_len) - 1;
}

// EnsureSafeSplitKey transforms the SQL table key argumnet such that it is a
Expand Down Expand Up @@ -95,7 +101,7 @@ bool RowCounter::Count(const rocksdb::Slice& key) {
prev_key.assign(decoded_key.data(), decoded_key.size());

uint64_t tbl;
if (!DecodeTablePrefix(&decoded_key, &tbl)) {
if (!DecodeTenantAndTablePrefix(&decoded_key, &tbl)) {
return false;
}

Expand Down
2 changes: 0 additions & 2 deletions c-deps/libroach/row_counter.h
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,6 @@ const int MaxReservedDescID = 49;
// RowCounter counts how many distinct rows appear in the KVs that is is shown
// via `Count`. Note: the `DataSize` field of the BulkOpSummary is *not*
// populated by this and should be set separately.
//
// TODO(nvanbenschoten): support tenant ID prefix.
struct RowCounter {
RowCounter(cockroach::roachpb::BulkOpSummary* summary) : summary(summary) {}
bool Count(const rocksdb::Slice& key);
Expand Down
3 changes: 0 additions & 3 deletions pkg/cli/cert.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,9 +203,6 @@ List certificates and keys found in the certificate directory.

// runListCerts loads and lists all certs.
func runListCerts(cmd *cobra.Command, args []string) error {
if err := security.SetCertPrincipalMap(certCtx.certPrincipalMap); err != nil {
return err
}
cm, err := security.NewCertificateManager(baseCfg.SSLCertsDir)
if err != nil {
return errors.Wrap(err, "cannot load certificates")
Expand Down
12 changes: 4 additions & 8 deletions pkg/cli/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ func initCLIDefaults() {
cliCtx.cmdTimeout = 0 // no timeout
cliCtx.clientConnHost = ""
cliCtx.clientConnPort = base.DefaultPort
cliCtx.certPrincipalMap = nil
cliCtx.sqlConnURL = ""
cliCtx.sqlConnUser = ""
cliCtx.sqlConnPasswd = ""
Expand Down Expand Up @@ -175,8 +176,6 @@ func initCLIDefaults() {

authCtx.validityPeriod = 1 * time.Hour

certCtx.certPrincipalMap = nil

initPreFlagsDefaults()

// Clear the "Changed" state of all the registered command-line flags.
Expand Down Expand Up @@ -219,6 +218,9 @@ type cliContext struct {
// clientConnPort is the port name/number to use to connect to a server.
clientConnPort string

// certPrincipalMap is the cert-principal:db-principal map.
certPrincipalMap []string

// for CLI commands that use the SQL interface, these parameters
// determine how to connect to the server.
sqlConnURL, sqlConnUser, sqlConnDBName string
Expand Down Expand Up @@ -413,9 +415,3 @@ var demoCtx struct {
insecure bool
geoLibsDir string
}

// certCtx captures the command-line parameters of the `cert` command.
// Defaults set by InitCLIDefaults() above.
var certCtx struct {
certPrincipalMap []string
}
20 changes: 14 additions & 6 deletions pkg/cli/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,7 +240,9 @@ func init() {

// Every command but start will inherit the following setting.
AddPersistentPreRunE(cockroachCmd, func(cmd *cobra.Command, _ []string) error {
extraClientFlagInit()
if err := extraClientFlagInit(); err != nil {
return err
}
return setDefaultStderrVerbosity(cmd, log.Severity_WARNING)
})

Expand Down Expand Up @@ -441,12 +443,11 @@ func init() {
f := cmd.Flags()
// All certs commands need the certificate directory.
StringFlag(f, &baseCfg.SSLCertsDir, cliflags.CertsDir, baseCfg.SSLCertsDir)
// All certs commands get the certificate principal map.
StringSlice(f, &cliCtx.certPrincipalMap,
cliflags.CertPrincipalMap, cliCtx.certPrincipalMap)
}

// The list certs command needs the certificate principal map.
StringSlice(listCertsCmd.Flags(), &certCtx.certPrincipalMap,
cliflags.CertPrincipalMap, certCtx.certPrincipalMap)

for _, cmd := range []*cobra.Command{createCACertCmd, createClientCACertCmd} {
f := cmd.Flags()
// CA certificates have a longer expiration time.
Expand Down Expand Up @@ -495,6 +496,9 @@ func init() {

// Certificate flags.
StringFlag(f, &baseCfg.SSLCertsDir, cliflags.CertsDir, baseCfg.SSLCertsDir)
// Certificate principal map.
StringSlice(f, &cliCtx.certPrincipalMap,
cliflags.CertPrincipalMap, cliCtx.certPrincipalMap)
}

// Auth commands.
Expand Down Expand Up @@ -885,7 +889,10 @@ func extraServerFlagInit(cmd *cobra.Command) error {
return nil
}

func extraClientFlagInit() {
func extraClientFlagInit() error {
if err := security.SetCertPrincipalMap(cliCtx.certPrincipalMap); err != nil {
return err
}
serverCfg.Addr = net.JoinHostPort(cliCtx.clientConnHost, cliCtx.clientConnPort)
serverCfg.AdvertiseAddr = serverCfg.Addr
serverCfg.SQLAddr = net.JoinHostPort(cliCtx.clientConnHost, cliCtx.clientConnPort)
Expand All @@ -901,6 +908,7 @@ func extraClientFlagInit() {
if sqlCtx.debugMode {
sqlCtx.echo = true
}
return nil
}

func setDefaultStderrVerbosity(cmd *cobra.Command, defaultSeverity log.Severity) error {
Expand Down
8 changes: 6 additions & 2 deletions pkg/cli/flags_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -799,7 +799,9 @@ func TestServerJoinSettings(t *testing.T) {
t.Fatalf("Parse(%#v) got unexpected error: %v", td.args, err)
}

extraClientFlagInit()
if err := extraClientFlagInit(); err != nil {
t.Fatal(err)
}

var actual []string
myHostname, _ := os.Hostname()
Expand Down Expand Up @@ -861,7 +863,9 @@ func TestClientConnSettings(t *testing.T) {
t.Fatalf("Parse(%#v) got unexpected error: %v", td.args, err)
}

extraClientFlagInit()
if err := extraClientFlagInit(); err != nil {
t.Fatal(err)
}
if td.expectedAddr != serverCfg.Addr {
t.Errorf("%d. serverCfg.Addr expected '%s', but got '%s'. td.args was '%#v'.",
i, td.expectedAddr, serverCfg.Addr, td.args)
Expand Down
33 changes: 27 additions & 6 deletions pkg/cli/interactive_tests/test_cert_advisory_validation.tcl
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,8 @@ set prompt ":/# "
eexpect $prompt

# create some cert without an IP address in there.
set certs_dir "./my-safe-directory"
set db_dir "logs/db"
set certs_dir "logs/my-safe-directory"
send "mkdir -p $certs_dir\r"
eexpect $prompt

Expand All @@ -21,7 +22,7 @@ send "$argv cert create-node localhost --certs-dir=$certs_dir --ca-key=$certs_di
eexpect $prompt

start_test "Check that the server reports a warning if attempting to advertise an IP address not in cert."
send "$argv start-single-node --certs-dir=$certs_dir --advertise-addr=127.0.0.1\r"
send "$argv start-single-node --store=$db_dir --certs-dir=$certs_dir --advertise-addr=127.0.0.1\r"
eexpect "advertise address"
eexpect "127.0.0.1"
eexpect "not in node certificate"
Expand All @@ -32,7 +33,7 @@ eexpect $prompt
end_test

start_test "Check that the server reports no warning if the avertise addr is in the cert."
send "$argv start-single-node --certs-dir=$certs_dir --advertise-addr=localhost\r"
send "$argv start-single-node --store=$db_dir --certs-dir=$certs_dir --advertise-addr=localhost\r"
expect {
"not in node certificate" {
report "unexpected warning"
Expand All @@ -51,21 +52,21 @@ send "COCKROACH_CERT_NODE_USER=foo.bar $argv cert create-node localhost --certs-
eexpect $prompt

start_test "Check that the server reports an error if the node cert does not contain a node principal."
send "$argv start-single-node --certs-dir=$certs_dir --advertise-addr=localhost\r"
send "$argv start-single-node --store=$db_dir --certs-dir=$certs_dir --advertise-addr=localhost\r"
eexpect "cannot load certificates"
expect $prompt
end_test

start_test "Check that the cert principal map can allow the use of non-standard cert principal."
send "$argv start-single-node --certs-dir=$certs_dir --cert-principal-map=foo.bar:node --advertise-addr=localhost\r"
send "$argv start-single-node --store=$db_dir --certs-dir=$certs_dir --cert-principal-map=foo.bar:node --advertise-addr=localhost\r"
eexpect "node starting"
interrupt
eexpect "interrupted"
expect $prompt
end_test

start_test "Check that the cert principal map can allow the use of a SAN principal."
send "$argv start-single-node --certs-dir=$certs_dir --cert-principal-map=localhost:node --advertise-addr=localhost\r"
send "$argv start-single-node --store=$db_dir --certs-dir=$certs_dir --cert-principal-map=localhost:node --advertise-addr=localhost\r"
eexpect "node starting"
interrupt
eexpect "interrupted"
Expand All @@ -77,3 +78,23 @@ send "$argv cert list --certs-dir=$certs_dir --cert-principal-map=foo.bar:node\r
eexpect "Certificate directory:"
expect $prompt
end_test

start_test "Check that 'cert create-client' can utilize cert principal map."
send "$argv cert create-client root.crdb.io --certs-dir=$certs_dir --ca-key=$certs_dir/ca.key --cert-principal-map=foo.bar:node\r"
eexpect $prompt
send "mv $certs_dir/client.root.crdb.io.crt $certs_dir/client.root.crt; mv $certs_dir/client.root.crdb.io.key $certs_dir/client.root.key\r"
eexpect $prompt
end_test

start_test "Check that the client commands can use cert principal map."
system "$argv start-single-node --store=$db_dir --certs-dir=$certs_dir --cert-principal-map=foo.bar:node,root.crdb.io:root --advertise-addr=localhost --background >>expect-cmd.log 2>&1"
send "$argv sql --certs-dir=$certs_dir --cert-principal-map=foo.bar:node,root.crdb.io:root -e \"select 'hello'\"\r"
eexpect "hello"
expect $prompt
send "$argv node ls --certs-dir=$certs_dir --cert-principal-map=foo.bar:node,root.crdb.io:root\r"
eexpect "1 row"
expect $prompt
send "$argv quit --certs-dir=$certs_dir --cert-principal-map=foo.bar:node,root.crdb.io:root\r"
eexpect "ok"
expect $prompt
end_test
Loading

0 comments on commit 456d01b

Please sign in to comment.