Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
81388: authors: add chinemeremchigbo to authors r=lassenordahl a=ChinemeremChigbo

authors: add chinemeremchigbo to authors

Release note: None
Release justification: non-production code change

81460: acceptance: run `python`, `psql` containers as current uid r=knz a=rickystewart

`postgres`'s permission checking for certificates has gotten more
rigorous since [this commit](https://git.postgresql.org/gitweb/?p=postgresql.git;a=commitdiff;h=a59c79564bdc209a5bc7b02d706f0d7352eb82fa).
This has broken a couple `acceptance` tests which do not pin to any
specific `postgres` version (see #81313, #81437).

Here we attempt to solve the problem "once and for all" by ensuring that
these containers run with a UID that is equal to the one that created
the certificates.

Release note: None

81464: ui: Add tests for the TimeScaleDropdown component r=jocrl a=jocrl

This commit splits out tests in `timescale.spec.tsx` and adds additional tests
for the TimeScaleDropdown component, including testing that the custom time
picker is initialized to the currently selected time. This also adds
TimeScaleDropdown stories.

Release note: None

81486: sql/backfill: fix bug adding new columns to new index with volatile default r=ajwerner a=ajwerner

In #58990 we added support for the index backfiller to evaluate expressions. This unblocked the usage of virtual computed columns in secondary expressions, so wasn't a totally bogus change, but probably was trying to do too much without understanding all of the implications. The original motivation for that change was to unblock the then nascent declarative schema changer prototype which wanted to only implement #47989 as a column change protocol. In order to do that, it needed to evaluate default expressions when adding new columns to a new primary index. For whatever reason, probably lack of understanding, I made it such that all default expressions for columns which were in the adding state were evaluated. This is wrong in cases where the column has already been backfilled into the current primary index; it's wrong because volatile expressions will evaluate to different values causing the newly backfilled secondary index to be bogus and corrupt. 

This change addresses the failure of the above change by being more thoughtful about whether it's sane to evaluate a default expression when backfilling into an index. Note that it's still insane to backfill a volatile expression for a new column into the key of a new primary index. As of writing, there is no way to do this. 

Backports will address #81448.

Release note (bug fix): In 21.1 a bug was introduced whereby default values
were recomputed when populating data in new secondary indexes for columns
which were added in the same transaction as the index. This arises, for example
in cases like `ALTER TABLE t ADD COLUMN f FLOAT8 UNIQUE DEFAULT (random())`.
If the default expression is not volatile, then the recomputation is harmless.
If, however, the default expression is volatile, the data in the secondary
index will not match the data in the primary index: a corrupt index will have
been created. This bug has now been fixed.

81504: acceptance/roachtest: update ORM versions under test r=ecwall a=rafiss

See DXTPT-35

Release note: None

81523: sql: fix typo in warning on using DateStyle/IntervalStyle r=rafiss a=otan

Release note (sql change): Fixed a small typo when using DateStyle and
IntervalStyle.

Co-authored-by: Chinemerem <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Josephine Lee <[email protected]>
Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: Oliver Tan <[email protected]>
  • Loading branch information
7 people committed May 19, 2022
7 parents 52868f8 + f64c8ae + 3ef77d5 + 4feee3f + d531343 + 6c5241c + 3cd070a commit bb3f07b
Show file tree
Hide file tree
Showing 30 changed files with 1,222 additions and 409 deletions.
1 change: 1 addition & 0 deletions AUTHORS
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,7 @@ Charlie Vieth <[email protected]>
Chen Chen <[email protected]> <[email protected]>
chengwei <[email protected]>
Chengxiong Ruan <[email protected]> <[email protected]>
Chinemerem Chigbo <[email protected]> <[email protected]>
Chris Seto <[email protected]> <@cockroachlabs.com>
Christian Meunier <[email protected]>
Christopher Fitzner <[email protected]> <[email protected]>
Expand Down
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -226,6 +226,7 @@ ALL_TESTS = [
"//pkg/spanconfig/spanconfigstore:spanconfigstore_test",
"//pkg/spanconfig/spanconfigtestutils:spanconfigtestutils_test",
"//pkg/spanconfig:spanconfig_test",
"//pkg/sql/backfill:backfill_test",
"//pkg/sql/catalog/catalogkeys:catalogkeys_test",
"//pkg/sql/catalog/catformat:catformat_test",
"//pkg/sql/catalog/catpb:catpb_test",
Expand Down
1 change: 1 addition & 0 deletions pkg/acceptance/compose/gss/docker-compose-python.yml
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ services:
- ${COCKROACH_BINARY:-../../../../cockroach-linux-2.6.32-gnu-amd64}:/cockroach/cockroach
python:
build: ./python
user: "${UID}:${GID}"
depends_on:
- cockroach
environment:
Expand Down
1 change: 1 addition & 0 deletions pkg/acceptance/compose/gss/docker-compose.yml
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ services:
- ${COCKROACH_BINARY:-../../../../cockroach-linux-2.6.32-gnu-amd64}:/cockroach/cockroach
psql:
build: ./psql
user: "${UID}:${GID}"
depends_on:
- cockroach
environment:
Expand Down
11 changes: 9 additions & 2 deletions pkg/acceptance/compose/gss/psql/Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -7,12 +7,19 @@ RUN GO111MODULE=off go get -d -t -tags gss_compose
RUN GO111MODULE=off go test -v -c -tags gss_compose -o gss.test

# Copy the test binary to an image with psql and krb installed.
FROM postgres:11.15
FROM postgres:11

RUN apt-get update && \
DEBIAN_FRONTEND=noninteractive apt-get install --yes --no-install-recommends \
ca-certificates \
curl \
krb5-user

COPY --from=builder /workspace/gss.test .

ENTRYPOINT ["/start.sh"]
RUN curl -fsSL "https://github.com/benesch/autouseradd/releases/download/1.3.0/autouseradd-1.3.0-amd64.tar.gz" -o autouseradd.tar.gz \
&& echo "442dae58b727a79f81368127fac141d7f95501ffa05f8c48943d27c4e807deb7 autouseradd.tar.gz" | sha256sum -c - \
&& tar xzf autouseradd.tar.gz --strip-components 1 \
&& rm autouseradd.tar.gz

ENTRYPOINT ["autouseradd", "--user", "roach", "--no-create-home", "/start.sh"]
2 changes: 1 addition & 1 deletion pkg/acceptance/compose/gss/psql/start.sh
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ echo "Preparing SQL user ahead of test"
env \
PGSSLKEY=/certs/client.root.key \
PGSSLCERT=/certs/client.root.crt \
psql -c "ALTER USER root WITH PASSWORD rootpw"
psql -U root -c "ALTER USER root WITH PASSWORD rootpw"

echo "Running test"
./gss.test
8 changes: 7 additions & 1 deletion pkg/acceptance/compose/gss/python/Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -5,13 +5,19 @@ RUN wget --quiet -O - https://www.postgresql.org/media/keys/ACCC4CF8.asc | apt-k
echo "deb http://apt.postgresql.org/pub/repos/apt/ bullseye-pgdg main" | tee /etc/apt/sources.list.d/pgdg.list && \
apt-get update && \
DEBIAN_FRONTEND=noninteractive apt-get install --yes --no-install-recommends \
curl \
krb5-user \
postgresql-client-11

RUN curl -fsSL "https://github.com/benesch/autouseradd/releases/download/1.3.0/autouseradd-1.3.0-amd64.tar.gz" -o autouseradd.tar.gz \
&& echo "442dae58b727a79f81368127fac141d7f95501ffa05f8c48943d27c4e807deb7 autouseradd.tar.gz" | sha256sum -c - \
&& tar xzf autouseradd.tar.gz --strip-components 1 \
&& rm autouseradd.tar.gz

RUN mkdir /code
WORKDIR /code
COPY requirements.txt /code/
RUN pip install -r requirements.txt
COPY . /code/

ENTRYPOINT ["/start.sh"]
ENTRYPOINT ["autouseradd", "--user", "roach", "--no-create-home", "/start.sh"]
11 changes: 11 additions & 0 deletions pkg/acceptance/compose_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"os"
"os/exec"
"path/filepath"
"strconv"
"testing"

"github.com/cockroachdb/cockroach/pkg/acceptance/cluster"
Expand Down Expand Up @@ -61,6 +62,16 @@ func testCompose(t *testing.T, path string, exitCodeFrom string) {
} else {
path = filepath.Join(composeDir, path)
}
uid := os.Getuid()
err := os.Setenv("UID", strconv.Itoa(uid))
if err != nil {
t.Fatalf(err.Error())
}
gid := os.Getgid()
err = os.Setenv("GID", strconv.Itoa(gid))
if err != nil {
t.Fatalf(err.Error())
}
cmd := exec.Command(
"docker-compose",
"--no-ansi",
Expand Down
2 changes: 1 addition & 1 deletion pkg/acceptance/testdata/csharp/acceptance.csproj
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,6 @@
<TargetFramework>netcoreapp2.0</TargetFramework>
</PropertyGroup>
<ItemGroup>
<PackageReference Include="npgsql" Version="3.2.5" />
<PackageReference Include="npgsql" Version="6.0.4" />
</ItemGroup>
</Project>
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/gorm.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import (
)

var gormReleaseTag = regexp.MustCompile(`^v(?P<major>\d+)\.(?P<minor>\d+)\.(?P<point>\d+)$`)
var gormSupportedTag = "v1.23.1"
var gormSupportedTag = "v1.23.5"

func registerGORM(r registry.Registry) {
runGORM := func(ctx context.Context, t test.Test, c cluster.Cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/hibernate.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
)

var hibernateReleaseTagRegex = regexp.MustCompile(`^(?P<major>\d+)\.(?P<minor>\d+)\.(?P<point>\d+)$`)
var supportedHibernateTag = "5.4.30"
var supportedHibernateTag = "5.6.9"

type hibernateOptions struct {
testName string
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/knex.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import (
"github.com/stretchr/testify/require"
)

const supportedKnexTag = "0.95.14"
const supportedKnexTag = "2.0.0"

// This test runs one of knex's test suite against a single cockroach
// node.
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/ruby_pg.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import (
var rubyPGTestFailureRegex = regexp.MustCompile(`^rspec ./.*# .*`)
var testFailureFilenameRegexp = regexp.MustCompile("^rspec .*.rb.*([0-9]|]) # ")
var testSummaryRegexp = regexp.MustCompile("^([0-9]+) examples, [0-9]+ failures")
var rubyPGVersion = "v1.2.3"
var rubyPGVersion = "v1.3.5"

// This test runs Ruby PG's full test suite against a single cockroach node.
func registerRubyPG(r registry.Registry) {
Expand Down
10 changes: 8 additions & 2 deletions pkg/cmd/roachtest/tests/rust_postgres.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
)

const rustPostgresSupportedTag = "postgres-v0.19.3"

func registerRustPostgres(r registry.Registry) {
runRustPostgres := func(ctx context.Context, t test.Test, c cluster.Cluster) {
if c.IsLocal() {
Expand Down Expand Up @@ -58,10 +60,14 @@ func registerRustPostgres(r registry.Registry) {
t.Fatal(err)
}

if err := c.RunE(
if err := repeatGitCloneE(
ctx,
t,
c,
"https://github.com/sfackler/rust-postgres.git",
"/mnt/data1/rust-postgres",
rustPostgresSupportedTag,
node,
"cd /mnt/data1 && git clone https://github.com/sfackler/rust-postgres.git",
); err != nil {
t.Fatal(err)
}
Expand Down
14 changes: 13 additions & 1 deletion pkg/sql/backfill/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,9 +1,10 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "backfill",
srcs = [
"backfill.go",
"index_backfiller_cols.go",
"mvcc_index_merger.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/backfill",
Expand Down Expand Up @@ -40,3 +41,14 @@ go_library(
"@com_github_cockroachdb_logtags//:logtags",
],
)

go_test(
name = "backfill_test",
srcs = ["index_backfiller_cols_test.go"],
embed = [":backfill"],
deps = [
"//pkg/sql/catalog",
"//pkg/sql/catalog/descpb",
"@com_github_stretchr_testify//require",
],
)
90 changes: 20 additions & 70 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
Expand Down Expand Up @@ -414,27 +413,14 @@ type muBoundAccount struct {

// IndexBackfiller is capable of backfilling all the added index.
type IndexBackfiller struct {
indexBackfillerCols

added []catalog.Index
// colIdxMap maps ColumnIDs to indices into desc.Columns and desc.Mutations.
colIdxMap catalog.TableColMap

types []*types.T
rowVals tree.Datums
evalCtx *eval.Context

// cols are all of the writable (PUBLIC and DELETE_AND_WRITE_ONLY) columns in
// the descriptor.
cols []catalog.Column

// addedCols are the columns in DELETE_AND_WRITE_ONLY being added as part of
// this index which are not computed.
addedCols []catalog.Column

// computedCols are the columns in this index which are computed and do
// not have concrete values in the source index. This is virtual computed
// columns and stored computed columns which are non-public.
computedCols []catalog.Column

// Map of columns which need to be evaluated to their expressions.
colExprs map[descpb.ColumnID]tree.TypedExpr

Expand All @@ -447,10 +433,6 @@ type IndexBackfiller struct {
// backfilled.
indexesToEncode []catalog.Index

// valNeededForCol contains the indexes (into cols) of all columns that we
// need to fetch values for.
valNeededForCol util.FastIntSet

alloc tree.DatumAlloc

// mon is a memory monitor linked with the IndexBackfiller on creation.
Expand Down Expand Up @@ -478,11 +460,14 @@ func (ib *IndexBackfiller) InitForLocalUse(
desc catalog.TableDescriptor,
mon *mon.BytesMonitor,
) error {
// Initialize ib.cols and ib.colIdxMap.
ib.initCols(desc)

// Initialize ib.added.
ib.valNeededForCol = ib.initIndexes(desc)
ib.initIndexes(desc)

// Initialize ib.cols and ib.colIdxMap.
if err := ib.initCols(desc); err != nil {
return err
}

predicates, colExprs, referencedColumns, err := constructExprs(
ctx, desc, ib.added, ib.cols, ib.addedCols, ib.computedCols, evalCtx, semaCtx,
Expand Down Expand Up @@ -614,11 +599,14 @@ func (ib *IndexBackfiller) InitForDistributedUse(
desc catalog.TableDescriptor,
mon *mon.BytesMonitor,
) error {
// Initialize ib.cols and ib.colIdxMap.
ib.initCols(desc)

// Initialize ib.added.
ib.valNeededForCol = ib.initIndexes(desc)
ib.initIndexes(desc)

// Initialize ib.indexBackfillerCols.
if err := ib.initCols(desc); err != nil {
return err
}

evalCtx := flowCtx.NewEvalCtx()
var predicates map[descpb.IndexID]tree.TypedExpr
Expand Down Expand Up @@ -689,35 +677,17 @@ func (ib *IndexBackfiller) ShrinkBoundAccount(ctx context.Context, shrinkBy int6

// initCols is a helper to populate column metadata of an IndexBackfiller. It
// populates the cols and colIdxMap fields.
func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) {
ib.cols = make([]catalog.Column, 0, len(desc.DeletableColumns()))
for _, column := range desc.DeletableColumns() {
if column.Public() {
if column.IsComputed() && column.IsVirtual() {
ib.computedCols = append(ib.computedCols, column)
}
} else if column.Adding() && column.WriteAndDeleteOnly() {
// If there are ongoing mutations, add columns that are being added and in
// the DELETE_AND_WRITE_ONLY state.
if column.IsComputed() {
ib.computedCols = append(ib.computedCols, column)
} else {
ib.addedCols = append(ib.addedCols, column)
}
} else {
continue
}
// Create a map of each column's ID to its ordinal.
ib.colIdxMap.Set(column.GetID(), len(ib.cols))
ib.cols = append(ib.cols, column)
}
func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) (err error) {
ib.indexBackfillerCols, err = makeIndexBackfillColumns(
desc.DeletableColumns(), desc.GetPrimaryIndex(), ib.added,
)
return err
}

// initIndexes is a helper to populate index metadata of an IndexBackfiller. It
// populates the added field. It returns a set of column ordinals that must be
// fetched in order to backfill the added indexes.
func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) util.FastIntSet {
var valNeededForCol util.FastIntSet
func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) {
mutations := desc.AllMutations()
mutationID := mutations[0].MutationID()

Expand All @@ -729,29 +699,9 @@ func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) util.FastIn
}
if IndexMutationFilter(m) {
idx := m.AsIndex()
colIDs := idx.CollectKeyColumnIDs()
if idx.GetEncodingType() == descpb.PrimaryIndexEncoding {
for _, col := range ib.cols {
if !col.IsVirtual() {
colIDs.Add(col.GetID())
}
}
} else {
colIDs.UnionWith(idx.CollectSecondaryStoredColumnIDs())
colIDs.UnionWith(idx.CollectKeySuffixColumnIDs())
}

ib.added = append(ib.added, idx)
for i := range ib.cols {
id := ib.cols[i].GetID()
if colIDs.Contains(id) && i < len(desc.PublicColumns()) && !ib.cols[i].IsVirtual() {
valNeededForCol.Add(i)
}
}
}
}

return valNeededForCol
}

// init completes the initialization of an IndexBackfiller.
Expand Down
Loading

0 comments on commit bb3f07b

Please sign in to comment.