Skip to content

Commit

Permalink
sql: add delete preserving index encoding, use for primary and second…
Browse files Browse the repository at this point in the history
…ary index

As part of the transition for Bulk Ops to use only MVCC operations, the index
backfiller must be rewritten. Part of the rewrite is to introduce a temporary
index at the beginning of the backfill encoded with a new delete-preserving
index encoding.

The delete-preserving encoding results in index values being
encoded with an additional bit that indicates whether or not the value has
been deleted, which is necessary to preserve the delete history during the
backfill.
The entire RFC is here: https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20211004_incremental_index_backfiller.md#new-index-encoding-for-deletions-vs-mvcc

This patch adds this encoding and uses the encoding for writes
to indices configured with this encoding.

Release note: None
  • Loading branch information
Rui Hu committed Nov 16, 2021
1 parent 370d1ac commit 26a054c
Show file tree
Hide file tree
Showing 27 changed files with 2,195 additions and 416 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ go_library(
"//pkg/keys",
"//pkg/kv",
"//pkg/kv/bulk",
"//pkg/kv/kvclient",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/batcheval",
"//pkg/kv/kvserver/concurrency/lock",
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/targets.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,9 @@ import (
"strings"

"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupresolver"
"github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql"
Expand Down Expand Up @@ -188,7 +188,7 @@ func getAllDescChanges(
startKey := codec.TablePrefix(keys.DescriptorTableID)
endKey := startKey.PrefixEnd()

allRevs, err := storageccl.GetAllRevisions(ctx, db, startKey, endKey, startTime, endTime)
allRevs, err := kvclient.GetAllRevisions(ctx, db, startKey, endKey, startTime, endTime)
if err != nil {
return nil, err
}
Expand Down
3 changes: 0 additions & 3 deletions pkg/ccl/storageccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,20 +6,17 @@ go_library(
"encryption.go",
"external_sst_reader.go",
"import.go",
"revision_reader.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/storageccl",
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/cloud",
"//pkg/kv",
"//pkg/kv/kvserver",
"//pkg/roachpb:with-mocks",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/storage",
"//pkg/util/hlc",
"//pkg/util/humanizeutil",
"//pkg/util/retry",
"@com_github_cockroachdb_errors//:errors",
Expand Down
12 changes: 6 additions & 6 deletions pkg/cli/testdata/doctor/test_recreate_zipdir

Large diffs are not rendered by default.

3 changes: 3 additions & 0 deletions pkg/kv/kvclient/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ go_library(
name = "kvclient",
srcs = [
"placeholder.go",
"revision_reader.go",
"scan_meta.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvclient",
Expand All @@ -12,5 +13,7 @@ go_library(
"//pkg/keys",
"//pkg/kv",
"//pkg/roachpb:with-mocks",
"//pkg/storage",
"//pkg/util/hlc",
],
)
Original file line number Diff line number Diff line change
@@ -1,12 +1,14 @@
// Copyright 2016 The Cockroach Authors.
// 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
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package storageccl
package kvclient

import (
"context"
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -459,6 +459,7 @@ go_test(
"create_stats_test.go",
"create_test.go",
"database_test.go",
"delete_preserving_index_test.go",
"dep_test.go",
"descriptor_mutation_test.go",
"distsql_physical_planner_test.go",
Expand Down Expand Up @@ -552,6 +553,7 @@ go_test(
"//pkg/jobs/jobspb",
"//pkg/keys",
"//pkg/kv",
"//pkg/kv/kvclient",
"//pkg/kv/kvclient/kvcoord:with-mocks",
"//pkg/kv/kvclient/rangecache:with-mocks",
"//pkg/kv/kvclient/rangefeed:with-mocks",
Expand Down Expand Up @@ -598,6 +600,7 @@ go_test(
"//pkg/sql/roleoption",
"//pkg/sql/row",
"//pkg/sql/rowenc",
"//pkg/sql/rowenc/rowencpb:rowencpb_go_proto",
"//pkg/sql/rowexec",
"//pkg/sql/rowinfra",
"//pkg/sql/scrub",
Expand Down
756 changes: 403 additions & 353 deletions pkg/sql/catalog/descpb/structured.pb.go

Large diffs are not rendered by default.

15 changes: 15 additions & 0 deletions pkg/sql/catalog/descpb/structured.proto
Original file line number Diff line number Diff line change
Expand Up @@ -562,6 +562,21 @@ message IndexDescriptor {
// TODO(mgartner): Update the comment to explain that columns are referenced
// by their ID once #49766 is addressed.
optional string predicate = 23 [(gogoproto.nullable) = false];

// UseDeletePreservingEncoding, if true, causes the index to be encoded with
// an additional bit that indicates whether or not the value has been deleted.
// Index key-values that are deleted in this way are not actually deleted, but
// remain in the index with a value which has the delete bit set to true.
//
// The encoding with the additional delete bit was chosen over a separate
// index encoding type because there would have to be a separate type for each
// encoding that we already have for indexes. The alternative would get harder
// to maintain if we added more index encodings in the future.
//
// This is necessary to preserve the delete history for the MVCC-compatible
// index backfiller
// docs/RFCS/20211004_incremental_index_backfiller.md#new-index-encoding-for-deletions-vs-mvcc
optional bool use_delete_preserving_encoding = 24 [(gogoproto.nullable) = false];
}

// ConstraintToUpdate represents a constraint to be added to the table and
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/table_elements.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,6 +165,7 @@ type Index interface {

NumCompositeColumns() int
GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID
UseDeletePreservingEncoding() bool
}

// Column is an interface around the column descriptor types.
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/catalog/tabledesc/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -306,6 +306,17 @@ func (w index) GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID
return w.desc.CompositeColumnIDs[compositeColumnOrdinal]
}

// UseDeletePreservingEncoding returns true if the index is to be encoded with
// an additional bit that indicates whether or not the value has been deleted.
// Index key-values that are deleted in this way are not actually deleted,
// but remain in the index with a value which has the delete bit set to true.
// This is necessary to preserve the delete history for the MVCC-compatible
// index backfiller
// docs/RFCS/20211004_incremental_index_backfiller.md#new-index-encoding-for-deletions-vs-mvcc
func (w index) UseDeletePreservingEncoding() bool {
return w.desc.UseDeletePreservingEncoding
}

// partitioning is the backing struct for a catalog.Partitioning interface.
type partitioning struct {
desc *descpb.PartitioningDescriptor
Expand Down
21 changes: 11 additions & 10 deletions pkg/sql/catalog/tabledesc/validate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,16 +154,17 @@ var validationMap = []struct {
"ForeignKey": {status: thisFieldReferencesNoObjects},
"ReferencedBy": {status: thisFieldReferencesNoObjects},

"Interleave": {status: iSolemnlySwearThisFieldIsValidated},
"InterleavedBy": {status: iSolemnlySwearThisFieldIsValidated},
"Partitioning": {status: iSolemnlySwearThisFieldIsValidated},
"Type": {status: thisFieldReferencesNoObjects},
"CreatedExplicitly": {status: thisFieldReferencesNoObjects},
"EncodingType": {status: thisFieldReferencesNoObjects},
"Sharded": {status: iSolemnlySwearThisFieldIsValidated},
"Disabled": {status: thisFieldReferencesNoObjects},
"GeoConfig": {status: thisFieldReferencesNoObjects},
"Predicate": {status: iSolemnlySwearThisFieldIsValidated},
"Interleave": {status: iSolemnlySwearThisFieldIsValidated},
"InterleavedBy": {status: iSolemnlySwearThisFieldIsValidated},
"Partitioning": {status: iSolemnlySwearThisFieldIsValidated},
"Type": {status: thisFieldReferencesNoObjects},
"CreatedExplicitly": {status: thisFieldReferencesNoObjects},
"EncodingType": {status: thisFieldReferencesNoObjects},
"Sharded": {status: iSolemnlySwearThisFieldIsValidated},
"Disabled": {status: thisFieldReferencesNoObjects},
"GeoConfig": {status: thisFieldReferencesNoObjects},
"Predicate": {status: iSolemnlySwearThisFieldIsValidated},
"UseDeletePreservingEncoding": {status: thisFieldReferencesNoObjects},
},
},
{
Expand Down
Loading

0 comments on commit 26a054c

Please sign in to comment.