Skip to content

Commit

Permalink
Merge #31407
Browse files Browse the repository at this point in the history
31407: server(ccl)/ui: identify zone configs by zone name, not ID r=vilterp a=vilterp

Previously, the `DataDistribution` endpoint was returning a map of zone configs by zone config ID. However, zone config ID is not a unique identier within `crdb_internal.zones`, since subzones attached to partitions share the same ID (that of the top-level zone config), so map entries were being overwritten and the endpoint was returning an incomplete list.

Zone names should be unique; this change uses those instead and updates the UI accordingly.

This change also introduces a `serverccl` package for the unit test to live in, since testing this code path requires creating partitions, a CCL feature.

Fixes #27718

Release note: None

Co-authored-by: Pete Vilter <[email protected]>
craig[bot] and Pete Vilter committed Oct 16, 2018
2 parents c9497fd + cb1e95c commit fc334d3
Showing 7 changed files with 421 additions and 284 deletions.
77 changes: 77 additions & 0 deletions pkg/ccl/serverccl/admin_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
// Copyright 2018 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 serverccl

import (
"context"
"reflect"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
)

var adminPrefix = "/_admin/v1/"

// TestAdminAPIDataDistributionPartitioning partitions a table and verifies
// that we see all zone configs (#27718).
func TestAdminAPIDataDistributionPartitioning(t *testing.T) {
defer leaktest.AfterTest(t)()

testCluster := serverutils.StartTestCluster(t, 3, base.TestClusterArgs{})
defer testCluster.Stopper().Stop(context.Background())

firstServer := testCluster.Server(0)
sqlDB := sqlutils.MakeSQLRunner(testCluster.ServerConn(0))

sqlDB.Exec(t, `CREATE DATABASE roachblog`)
sqlDB.Exec(t, `USE roachblog`)
sqlDB.Exec(t, `CREATE TABLE posts (id INT PRIMARY KEY, title text, body text)`)
sqlDB.Exec(t, `CREATE TABLE comments (
id INT,
post_id INT REFERENCES posts,
user_region STRING,
body text,
PRIMARY KEY (user_region, id)
) PARTITION BY LIST (user_region) (
PARTITION us VALUES IN ('US'),
PARTITION eu VALUES IN ('EU'),
PARTITION DEFAULT VALUES IN (default)
)`)

// Create a zone config for each partition.
// Would use locality constraints except this test cluster hasn't been started up with localities.
sqlDB.Exec(t, `ALTER PARTITION us OF TABLE comments CONFIGURE ZONE USING gc.ttlseconds = 9001`)
sqlDB.Exec(t, `ALTER PARTITION eu OF TABLE comments CONFIGURE ZONE USING gc.ttlseconds = 9002`)

// Assert that we get all roachblog zone configs back.
expectedZoneConfigNames := map[string]struct{}{
"roachblog.comments.eu": {},
"roachblog.comments.us": {},
}

var resp serverpb.DataDistributionResponse
if err := serverutils.GetJSONProto(firstServer, adminPrefix+"data_distribution", &resp); err != nil {
t.Fatal(err)
}

actualZoneConfigNames := map[string]struct{}{}
for name := range resp.ZoneConfigs {
if strings.HasPrefix(name, "roachblog.") {
actualZoneConfigNames[name] = struct{}{}
}
}
if !reflect.DeepEqual(actualZoneConfigNames, expectedZoneConfigNames) {
t.Fatalf("expected zone config names %v; got %v", expectedZoneConfigNames, actualZoneConfigNames)
}
}
10 changes: 10 additions & 0 deletions pkg/ccl/serverccl/doc.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,10 @@
// Copyright 2018 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 serverccl houses tests that verify CCL behavior of a running CockroachDB server.
package serverccl
34 changes: 34 additions & 0 deletions pkg/ccl/serverccl/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// Copyright 2018 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 serverccl

import (
"os"
"testing"

_ "github.com/cockroachdb/cockroach/pkg/ccl"
"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"
)

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())
}

//go:generate ../../util/leaktest/add-leaktest.sh *_test.go
24 changes: 13 additions & 11 deletions pkg/server/admin.go
Original file line number Diff line number Diff line change
@@ -1414,7 +1414,7 @@ func (s *adminServer) DataDistribution(
) (*serverpb.DataDistributionResponse, error) {
resp := &serverpb.DataDistributionResponse{
DatabaseInfo: make(map[string]serverpb.DataDistributionResponse_DatabaseInfo),
ZoneConfigs: make(map[int64]serverpb.DataDistributionResponse_ZoneConfig),
ZoneConfigs: make(map[string]serverpb.DataDistributionResponse_ZoneConfig),
}

// Get ids and names for databases and tables.
@@ -1535,8 +1535,11 @@ func (s *adminServer) DataDistribution(

// Get zone configs.
// TODO(vilterp): this can be done in parallel with getting table/db names and replica counts.
zoneConfigsQuery := `SELECT zone_id, cli_specifier, config_sql, config_protobuf
FROM crdb_internal.zones WHERE cli_specifier IS NOT NULL`
zoneConfigsQuery := `
SELECT zone_name, config_sql, config_protobuf
FROM crdb_internal.zones
WHERE zone_name IS NOT NULL
`
rows2, _ /* cols */, err := s.server.internalExecutor.QueryWithUser(
ctx, "admin-replica-matrix", nil /* txn */, userName, zoneConfigsQuery,
)
@@ -1545,19 +1548,18 @@ func (s *adminServer) DataDistribution(
}

for _, row := range rows2 {
zcID := int64(tree.MustBeDInt(row[0]))
zcCliSpecifier := string(tree.MustBeDString(row[1]))
zcSQL := tree.MustBeDString(row[2])
zcBytes := tree.MustBeDBytes(row[3])
zoneName := string(tree.MustBeDString(row[0]))
zcSQL := tree.MustBeDString(row[1])
zcBytes := tree.MustBeDBytes(row[2])
var zcProto config.ZoneConfig
if err := protoutil.Unmarshal([]byte(zcBytes), &zcProto); err != nil {
return nil, s.serverError(err)
}

resp.ZoneConfigs[zcID] = serverpb.DataDistributionResponse_ZoneConfig{
CliSpecifier: zcCliSpecifier,
Config: zcProto,
ConfigSQL: string(zcSQL),
resp.ZoneConfigs[zoneName] = serverpb.DataDistributionResponse_ZoneConfig{
ZoneName: zoneName,
Config: zcProto,
ConfigSQL: string(zcSQL),
}
}

546 changes: 279 additions & 267 deletions pkg/server/serverpb/admin.pb.go

Large diffs are not rendered by default.

10 changes: 6 additions & 4 deletions pkg/server/serverpb/admin.proto
Original file line number Diff line number Diff line change
@@ -536,9 +536,9 @@ message DataDistributionRequest {

message DataDistributionResponse {
message ZoneConfig {
// cli_specifier is ".default" for the default zone config.
// zone_name is ".default" for the default zone config.
// Otherwise, it's of the form [database.]table[@index[.partition]]
string cli_specifier = 1;
string zone_name = 1;

config.ZoneConfig config = 2 [(gogoproto.nullable) = false];

@@ -563,8 +563,10 @@ message DataDistributionResponse {
// By database name.
map<string, DatabaseInfo> database_info = 1 [(gogoproto.nullable) = false];

// By zone config id.
map<int64, ZoneConfig> zone_configs = 2 [(gogoproto.nullable) = false];
reserved 2;

// By zone name.
map<string, ZoneConfig> zone_configs = 3 [(gogoproto.nullable) = false];
}

// MetricMetadataRequest requests metadata for all metrics.
Original file line number Diff line number Diff line change
@@ -55,7 +55,7 @@ class DataDistribution extends React.Component<DataDistributionProps> {
<div className="zone-config-list">
<ul>
{this.props.sortedZoneConfigs.map((zoneConfig) => (
<li key={zoneConfig.cli_specifier} className="zone-config">
<li key={zoneConfig.zone_name} className="zone-config">
<pre className="zone-config__raw-sql">
{zoneConfig.config_sql}
</pre>
@@ -192,7 +192,7 @@ const sortedZoneConfigs = createSelector(
if (!dataDistributionState.data) {
return null;
}
return _.sortBy(dataDistributionState.data.zone_configs, (zc) => zc.cli_specifier);
return _.sortBy(dataDistributionState.data.zone_configs, (zc) => zc.zone_name);
},
);

0 comments on commit fc334d3

Please sign in to comment.