Skip to content

Commit 3a397eb

Browse files
craig[bot]shghasemimsbutler
committed
155778: sql: add `ALTER TABLE ... DROP STORED` to the declerative schema changer r=shghasemi a=shghasemi Previously, `ALTER TABLE ... DROP STORED` would run with the legacy schema changer. This change enables the comment be run by the declerative schema changer. Epic: none Fixes: #139603 Release note (sql change): `ALTER TABLE ... DROP STORED` will use the declerative schema changer to run. 155845: roachtest: allow local runs of azure backup roachtest r=kev-cao a=msbutler Previously the azure backup roachtest could only in the nightlies, and not by developers on their machines. This patch removes the clunky credentials auth path the test used to use. Now, the test auths to the bucket via an IAM policy that the developer needs to set up following the internal azure roachprod tutorial. Epic: none Release note: none Co-authored-by: Shadi Ghasemitaheri <[email protected]> Co-authored-by: Michael Butler <[email protected]>
3 parents 1a4ff79 + 33fbda1 + 3a94ffa commit 3a397eb

15 files changed

+371
-65
lines changed

pkg/ccl/schemachangerccl/sctestbackupccl/backup_base_generated_test.go

Lines changed: 28 additions & 0 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

pkg/cmd/roachtest/tests/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -354,7 +354,6 @@ go_library(
354354
"@com_github_prometheus_common//model",
355355
"@com_github_stretchr_testify//assert",
356356
"@com_github_stretchr_testify//require",
357-
"@in_gopkg_yaml_v3//:yaml_v3",
358357
"@org_golang_google_protobuf//proto",
359358
"@org_golang_x_exp//maps",
360359
"@org_golang_x_oauth2//clientcredentials",

pkg/cmd/roachtest/tests/backup.go

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -65,10 +65,6 @@ const (
6565
AssumeRoleGCSCredentials = "GOOGLE_CREDENTIALS_ASSUME_ROLE"
6666
AssumeRoleGCSServiceAccount = "GOOGLE_SERVICE_ACCOUNT"
6767

68-
AzureClientIDEnvVar = "AZURE_CLIENT_ID"
69-
AzureClientSecretEnvVar = "AZURE_CLIENT_SECRET"
70-
AzureTenantIDEnvVar = "AZURE_TENANT_ID"
71-
7268
// rows2TiB is the number of rows to import to load 2TB of data (when
7369
// replicated).
7470
rows2TiB = 65_104_166

pkg/cmd/roachtest/tests/backup_fixtures.go

Lines changed: 0 additions & 56 deletions
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,6 @@ import (
1010
gosql "database/sql"
1111
"fmt"
1212
"net/url"
13-
"os"
1413
"path"
1514
"time"
1615

@@ -34,21 +33,8 @@ import (
3433
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
3534
"github.com/cockroachdb/errors"
3635
"github.com/stretchr/testify/require"
37-
"gopkg.in/yaml.v3"
3836
)
3937

40-
// At the moment, Azure VMs do not have managed identities set up yet.
41-
// Therefore, in order to use implicit authentication, we need to put a
42-
// credentials file on each VM and point the
43-
// `COCKROACH_AZURE_APPLICATION_CREDENTIALS_FILE` environment variable at the
44-
// file.
45-
// Currently, the only set of credentials that have write access to the storage
46-
// buckets are the Teamcity credentials, so the Azure fixture roachtests cannot
47-
// be run locally until those managed identities are set up.
48-
// TODO (kev-cao): Once managed identities are set up, we can remove this file
49-
// and rely on the managed identity to authenticate with Azure Blob Storage.
50-
const azureCredentialsFilePath = "/home/ubuntu/azure-credentials.yaml"
51-
5238
// Maps a fixture database name to the expected number of tables in the
5339
// database, useful for verifying that the fingerprint of the fixture is as
5440
// expected.
@@ -197,9 +183,6 @@ func (bd *backupDriver) prepareCluster(ctx context.Context) {
197183
"cloudstorage.s3.client_retry_token_bucket.enabled": "false",
198184
"cloudstorage.azure.try.timeout": "0s",
199185
},
200-
install.EnvOption{
201-
fmt.Sprintf("COCKROACH_AZURE_APPLICATION_CREDENTIALS_FILE=%s", azureCredentialsFilePath),
202-
},
203186
))
204187
}
205188

@@ -653,7 +636,6 @@ func registerBackupFixtures(r registry.Registry) {
653636
Suites: bf.suites,
654637
Skip: bf.skip,
655638
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
656-
require.NoError(t, maybePutAzureCredentialsFile(ctx, c, azureCredentialsFilePath))
657639
registry := GetFixtureRegistry(ctx, t, c.Cloud())
658640

659641
handle, err := registry.Create(ctx, bf.fixture.Name, t.L())
@@ -689,44 +671,6 @@ func registerBackupFixtures(r registry.Registry) {
689671
}
690672
}
691673

692-
func maybePutAzureCredentialsFile(ctx context.Context, c cluster.Cluster, path string) error {
693-
if c.Cloud() != spec.Azure {
694-
return nil
695-
}
696-
697-
type azureCreds struct {
698-
TenantID string `yaml:"azure_tenant_id"`
699-
ClientID string `yaml:"azure_client_id"`
700-
ClientSecret string `yaml:"azure_client_secret"`
701-
}
702-
703-
azureEnvVars := []string{AzureTenantIDEnvVar, AzureClientIDEnvVar, AzureClientSecretEnvVar}
704-
azureEnvValues := make(map[string]string)
705-
for _, envVar := range azureEnvVars {
706-
val := os.Getenv(envVar)
707-
if val == "" {
708-
return errors.Newf("environment variable %s is not set", envVar)
709-
}
710-
azureEnvValues[envVar] = val
711-
}
712-
713-
creds := azureCreds{
714-
TenantID: azureEnvValues[AzureTenantIDEnvVar],
715-
ClientID: azureEnvValues[AzureClientIDEnvVar],
716-
ClientSecret: azureEnvValues[AzureClientSecretEnvVar],
717-
}
718-
719-
credsYaml, err := yaml.Marshal(creds)
720-
if err != nil {
721-
return errors.Wrapf(err, "failed to marshal Azure credentials to YAML")
722-
}
723-
724-
return errors.Wrap(
725-
c.PutString(ctx, string(credsYaml), path, 0700),
726-
"failed to put Azure credentials file in cluster",
727-
)
728-
}
729-
730674
func registerBlobFixtureGC(r registry.Registry) {
731675
r.Add(registry.TestSpec{
732676
Name: "blobfixture/gc",

pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@ go_library(
88
"alter_table_add_column.go",
99
"alter_table_add_constraint.go",
1010
"alter_table_alter_column_drop_not_null.go",
11+
"alter_table_alter_column_drop_stored.go",
1112
"alter_table_alter_column_set_default.go",
1213
"alter_table_alter_column_set_not_null.go",
1314
"alter_table_alter_column_set_on_update.go",

pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ var supportedAlterTableStatements = map[reflect.Type]supportedAlterTableCommand{
4343
reflect.TypeOf((*tree.AlterTableDropNotNull)(nil)): {fn: alterTableDropNotNull, on: true, checks: isV253Active},
4444
reflect.TypeOf((*tree.AlterTableSetOnUpdate)(nil)): {fn: alterTableSetOnUpdate, on: true, checks: isV254Active},
4545
reflect.TypeOf((*tree.AlterTableRenameColumn)(nil)): {fn: alterTableRenameColumn, on: true, checks: isV254Active},
46+
reflect.TypeOf((*tree.AlterTableDropStored)(nil)): {fn: alterTableDropStored, on: true, checks: isV261Active},
4647
}
4748

4849
func init() {
Lines changed: 69 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,69 @@
1+
// Copyright 2025 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the CockroachDB Software License
4+
// included in the /LICENSE file.
5+
6+
package scbuildstmt
7+
8+
import (
9+
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
10+
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
11+
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
12+
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
13+
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
14+
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
15+
)
16+
17+
func alterTableDropStored(
18+
b BuildCtx,
19+
tn *tree.TableName,
20+
tbl *scpb.Table,
21+
stmt tree.Statement,
22+
t *tree.AlterTableDropStored,
23+
) {
24+
alterColumnPreChecks(b, tn, tbl, t.Column)
25+
colElems := b.ResolveColumn(tbl.TableID, t.Column, ResolveParams{
26+
RequiredPrivilege: privilege.CREATE,
27+
})
28+
colElem := colElems.FilterColumn().MustGetOneElement()
29+
columnID := colElem.ColumnID
30+
// Block alters on system columns.
31+
panicIfSystemColumn(colElem, t.Column)
32+
// Ensure that the column is stored (not virtual)
33+
panicIfVirtualColumn(b, tbl.TableID, columnID, t.Column)
34+
// Retrieve the compute expression element
35+
exprElem := retrieveColumnComputeExpressionElem(b, tbl.TableID, columnID)
36+
// Ensure that the column is computed
37+
if exprElem == nil {
38+
panic(pgerror.Newf(
39+
pgcode.InvalidColumnDefinition,
40+
"column %q is not a computed column",
41+
tree.ErrString(&t.Column)))
42+
}
43+
b.Drop(exprElem)
44+
}
45+
46+
// panicIfVirtualColumn blocks operation if the column is virtual.
47+
func panicIfVirtualColumn(
48+
b BuildCtx, tableID catid.DescID, columnID catid.ColumnID, columnName tree.Name,
49+
) {
50+
colTypeEl := mustRetrieveColumnTypeElem(b, tableID, columnID)
51+
if colTypeEl.IsVirtual {
52+
panic(pgerror.Newf(
53+
pgcode.InvalidColumnDefinition,
54+
"column %q is not a stored computed column",
55+
tree.ErrString(&columnName)))
56+
}
57+
}
58+
59+
// retrieveColumnComputeExpressionElem returns the compute expression
60+
// element of the column. Returns nil if no expression exists and for
61+
// older versions that store the expression as part of the ColumnType
62+
func retrieveColumnComputeExpressionElem(
63+
b BuildCtx, tableID catid.DescID, columnID catid.ColumnID,
64+
) (expr *scpb.ColumnComputeExpression) {
65+
return b.QueryByID(tableID).FilterColumnComputeExpression().Filter(func(
66+
_ scpb.Status, _ scpb.TargetStatus, e *scpb.ColumnComputeExpression) bool {
67+
return e.ColumnID == columnID
68+
}).MustGetZeroOrOneElement()
69+
}

pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -227,3 +227,8 @@ var isV253Active = func(_ tree.NodeFormatter, _ sessiondatapb.NewSchemaChangerMo
227227
var isV254Active = func(_ tree.NodeFormatter, _ sessiondatapb.NewSchemaChangerMode, activeVersion clusterversion.ClusterVersion) bool {
228228
return activeVersion.IsActive(clusterversion.V25_4)
229229
}
230+
231+
var isV261Active = func(_ tree.NodeFormatter, _ sessiondatapb.NewSchemaChangerMode, activeVersion clusterversion.ClusterVersion) bool {
232+
// change the cluster version once V26_1 is created
233+
return activeVersion.IsActive(clusterversion.V25_4)
234+
}

pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -12,10 +12,6 @@ CREATE TABLE defaultdb.foo (
1212
);
1313
----
1414

15-
unimplemented
16-
ALTER TABLE defaultdb.foo ALTER COLUMN i DROP STORED
17-
----
18-
1915
unimplemented
2016
ALTER TABLE defaultdb.foo RENAME CONSTRAINT foobar TO baz
2117
----

pkg/sql/schemachanger/sctest_generated_test.go

Lines changed: 42 additions & 0 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

0 commit comments

Comments
 (0)