Skip to content

Commit

Permalink
sql,server: provide an approximate creation time for indexes
Browse files Browse the repository at this point in the history
This change plumbs a timestamp through to the index descriptor when it is
created. The timestamp used is based on the transaction start time. The
timestamp is then plumbed into the `crdb_internal.table_indexes` table and
exposed as a new NULL-able TIMESTAMP column, `created_at`.

Then, lastly, the timestamp is plumbed through to the status server via the
`TableIndexStatsRequest`.

Release note (sql change): The database now records the approximate time when
an index was created it. This information is exposed via a new NULL-able
TIMESTAMP column, `created_at`, on `crdb_internal.table_indexes`.
  • Loading branch information
ajwerner committed Jan 31, 2022
1 parent a5158c4 commit 80018e9
Show file tree
Hide file tree
Showing 38 changed files with 315 additions and 163 deletions.
2 changes: 2 additions & 0 deletions pkg/ccl/importccl/import_table_creation.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)
Expand Down Expand Up @@ -168,6 +169,7 @@ func MakeSimpleTableDescriptor(
Regions: makeImportRegionOperator(""),
SessionDataStack: sessiondata.NewStack(&sessiondata.SessionData{}),
ClientNoticeSender: &faketreeeval.DummyClientNoticeSender{},
TxnTimestamp: timeutil.Unix(0, walltime),
Settings: st,
}
affected := make(map[descpb.ID]*tabledesc.Mutable)
Expand Down
74 changes: 58 additions & 16 deletions pkg/ccl/serverccl/statusccl/tenant_status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -508,7 +508,7 @@ WHERE
}

func testTableIndexStats(ctx context.Context, t *testing.T, testHelper *tenantTestHelper) {
getTableIndexStats := func(helper *tenantTestHelper, db string) *serverpb.TableIndexStatsResponse {
getTableIndexStats := func(t *testing.T, helper *tenantTestHelper, db string) *serverpb.TableIndexStatsResponse {
// Get index usage stats using function call.
cluster := helper.testCluster()
status := cluster.tenantStatusSrv(randomServer)
Expand All @@ -520,6 +520,8 @@ func testTableIndexStats(ctx context.Context, t *testing.T, testHelper *tenantTe

cluster := testHelper.testCluster()

timePreCreate := timeutil.Now()

// Create table on a database.
cluster.tenantConn(0).Exec(t, `
CREATE DATABASE test_db1;
Expand Down Expand Up @@ -555,21 +557,61 @@ FROM pg_catalog.pg_indexes
WHERE tablename = 'test' AND indexname = $1`

// Get index usage stats and assert expected results.
resp := getTableIndexStats(testHelper, "test_db1")
require.Equal(t, uint64(1), resp.Statistics[0].Statistics.Stats.TotalReadCount)
require.True(t, resp.Statistics[0].Statistics.Stats.LastRead.After(timePreRead))
indexName := resp.Statistics[0].IndexName
createStmt := cluster.tenantConn(0).QueryStr(t, getCreateStmtQuery, indexName)[0][0]
print(createStmt)
require.Equal(t, resp.Statistics[0].CreateStatement, createStmt)

resp = getTableIndexStats(testHelper, "test_db2")
require.Equal(t, uint64(0), resp.Statistics[0].Statistics.Stats.TotalReadCount)
require.Equal(t, resp.Statistics[0].Statistics.Stats.LastRead, time.Time{})
indexName = resp.Statistics[0].IndexName
cluster.tenantConn(0).Exec(t, `SET DATABASE=test_db2`)
createStmt = cluster.tenantConn(0).QueryStr(t, getCreateStmtQuery, indexName)[0][0]
require.Equal(t, resp.Statistics[0].CreateStatement, createStmt)
requireAfter := func(t *testing.T, a, b *time.Time) {
t.Helper()
require.NotNil(t, a)
require.NotNil(t, b)
require.Truef(t, a.After(*b), "%v is not after %v", a, b)
}
requireBetween := func(t *testing.T, before time.Time, ts *time.Time, after time.Time) {
t.Helper()
requireAfter(t, ts, &before)
requireAfter(t, &after, ts)
}

t.Run("validate read index", func(t *testing.T) {
resp := getTableIndexStats(t, testHelper, "test_db1")
require.Equal(t, uint64(1), resp.Statistics[0].Statistics.Stats.TotalReadCount)
requireAfter(t, &resp.Statistics[0].Statistics.Stats.LastRead, &timePreRead)
indexName := resp.Statistics[0].IndexName
createStmt := cluster.tenantConn(0).QueryStr(t, getCreateStmtQuery, indexName)[0][0]
print(createStmt)
require.Equal(t, resp.Statistics[0].CreateStatement, createStmt)
requireBetween(t, timePreCreate, resp.Statistics[0].CreatedAt, timePreRead)
})

t.Run("validate unread index", func(t *testing.T) {
resp := getTableIndexStats(t, testHelper, "test_db2")
require.Equal(t, uint64(0), resp.Statistics[0].Statistics.Stats.TotalReadCount)
require.Equal(t, resp.Statistics[0].Statistics.Stats.LastRead, time.Time{})
indexName := resp.Statistics[0].IndexName
cluster.tenantConn(0).Exec(t, `SET DATABASE=test_db2`)
createStmt := cluster.tenantConn(0).QueryStr(t, getCreateStmtQuery, indexName)[0][0]
require.Equal(t, resp.Statistics[0].CreateStatement, createStmt)
requireBetween(t, timePreCreate, resp.Statistics[0].CreatedAt, timePreRead)
})

// Test that a subsequent index creation has an appropriate timestamp.
t.Run("validate CreatedAt for new index", func(t *testing.T) {
timeBeforeCreateNewIndex := timeutil.Now()
cluster.tenantConn(0).Exec(t, `
SET DATABASE=test_db2;
CREATE INDEX idx2 ON test (b, a)`)
timeAfterCreateNewIndex := timeutil.Now()

resp := getTableIndexStats(t, testHelper, "test_db2")
var stat serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics
var found bool
for _, idx := range resp.Statistics {
if found = idx.IndexName == "idx2"; found {
stat = *idx
break
}
}
require.True(t, found)
requireBetween(t,
timeBeforeCreateNewIndex, stat.CreatedAt, timeAfterCreateNewIndex)
})
}

func ensureExpectedStmtFingerprintExistsInRPCResponse(
Expand Down
11 changes: 9 additions & 2 deletions pkg/server/index_usage_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -230,7 +230,8 @@ func getTableIndexUsageStats(
ti.index_type,
total_reads,
last_read,
indexdef
indexdef,
ti.created_at
FROM crdb_internal.index_usage_statistics AS us
JOIN crdb_internal.table_indexes AS ti ON us.index_id = ti.index_id
AND us.table_id = ti.descriptor_id
Expand All @@ -241,7 +242,7 @@ func getTableIndexUsageStats(
tableID,
)

const expectedNumDatums = 6
const expectedNumDatums = 7

it, err := ie.QueryIteratorEx(ctx, "index-usage-stats", nil,
sessiondata.InternalExecutorOverride{
Expand Down Expand Up @@ -279,6 +280,11 @@ func getTableIndexUsageStats(
lastRead = tree.MustBeDTimestampTZ(row[4]).Time
}
createStmt := tree.MustBeDString(row[5])
var createdAt *time.Time
if row[6] != tree.DNull {
ts := tree.MustBeDTimestamp(row[6])
createdAt = &ts.Time
}

if err != nil {
return nil, err
Expand All @@ -298,6 +304,7 @@ func getTableIndexUsageStats(
IndexName: string(indexName),
IndexType: string(indexType),
CreateStatement: string(createStmt),
CreatedAt: createdAt,
}

idxUsageStats = append(idxUsageStats, idxStatsRow)
Expand Down
4 changes: 4 additions & 0 deletions pkg/server/serverpb/status.proto
Original file line number Diff line number Diff line change
Expand Up @@ -1392,6 +1392,10 @@ message TableIndexStatsResponse {
// create_statement is the SQL statement that would re-create the
// current index if executed.
string create_statement = 4;
// CreatedAt is an approximate timestamp at which the index was created.
// Note that it may not always be populated.
google.protobuf.Timestamp created_at = 5 [(gogoproto.stdtime) = true];

}

repeated ExtendedCollectedIndexUsageStatistics statistics = 1;
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package sql

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -168,6 +169,7 @@ func (p *planner) AlterPrimaryKey(
EncodingType: descpb.PrimaryIndexEncoding,
Type: descpb.IndexDescriptor_FORWARD,
Version: descpb.LatestNonPrimaryIndexDescriptorVersion,
CreatedAtNanos: p.EvalContext().GetTxnTimestamp(time.Microsecond).UnixNano(),
}

// If the new index is requested to be sharded, set up the index descriptor
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
gojson "encoding/json"
"fmt"
"sort"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
Expand Down Expand Up @@ -269,6 +270,7 @@ func (n *alterTableNode) startExec(params runParams) error {
Name: string(d.Name),
Unique: true,
StoreColumnNames: d.Storing.ToStrings(),
CreatedAtNanos: params.EvalContext().GetTxnTimestamp(time.Microsecond).UnixNano(),
}
if err := idx.FillColumns(d.Columns); err != nil {
return err
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/catalog/descpb/structured.proto
Original file line number Diff line number Diff line change
Expand Up @@ -512,6 +512,10 @@ message IndexDescriptor {
// 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];

// CreatedAtNanos, if non-zero, represents an approximate time at which the
// index was created.
optional int64 created_at_nanos = 25 [(gogoproto.nullable) = false];
}

// ConstraintToUpdate represents a constraint to be added to the table and
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/catalog/table_elements.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,8 @@
package catalog

import (
"time"

"github.com/cockroachdb/cockroach/pkg/geo/geoindex"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -186,6 +188,11 @@ type Index interface {
NumCompositeColumns() int
GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID
UseDeletePreservingEncoding() bool

// CreatedAt is an approximate timestamp at which the index was created.
// It is derived from the statement time at which the relevant statement
// was issued.
CreatedAt() time.Time
}

// Column is an interface around the column descriptor types.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/tabledesc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ go_library(
"//pkg/util/iterutil",
"//pkg/util/log",
"//pkg/util/protoutil",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@com_github_lib_pq//oid",
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/catalog/tabledesc/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package tabledesc
import (
"fmt"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/geo/geoindex"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand All @@ -21,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
)

var _ catalog.Index = (*index)(nil)
Expand Down Expand Up @@ -327,6 +329,13 @@ func (w index) UseDeletePreservingEncoding() bool {
return w.desc.UseDeletePreservingEncoding
}

func (w index) CreatedAt() time.Time {
if w.desc.CreatedAtNanos == 0 {
return time.Time{}
}
return timeutil.Unix(0, w.desc.CreatedAtNanos)
}

// partitioning is the backing struct for a catalog.Partitioning interface.
type partitioning struct {
desc *descpb.PartitioningDescriptor
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/tabledesc/validate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,7 @@ var validationMap = []struct {
"GeoConfig": {status: thisFieldReferencesNoObjects},
"Predicate": {status: iSolemnlySwearThisFieldIsValidated},
"UseDeletePreservingEncoding": {status: thisFieldReferencesNoObjects},
"CreatedAtNanos": {status: thisFieldReferencesNoObjects},
},
},
{
Expand Down
15 changes: 12 additions & 3 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -2587,8 +2587,6 @@ CREATE TABLE crdb_internal.table_columns (
}

// crdbInternalTableIndexesTable exposes the index descriptors.
//
// TODO(tbg): prefix with kv_.
var crdbInternalTableIndexesTable = virtualSchemaTable{
comment: "indexes accessible by current user in current database (KV scan)",
schema: `
Expand All @@ -2600,7 +2598,8 @@ CREATE TABLE crdb_internal.table_indexes (
index_type STRING NOT NULL,
is_unique BOOL NOT NULL,
is_inverted BOOL NOT NULL,
is_sharded BOOL NOT NULL
is_sharded BOOL NOT NULL,
created_at TIMESTAMP
)
`,
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
Expand All @@ -2622,6 +2621,15 @@ CREATE TABLE crdb_internal.table_indexes (
if idx.Primary() {
idxType = primary
}
createdAt := tree.DNull
if ts := idx.CreatedAt(); !ts.IsZero() {
tsDatum, err := tree.MakeDTimestamp(ts, time.Nanosecond)
if err != nil {
log.Warningf(ctx, "failed to construct timestamp for index: %v", err)
} else {
createdAt = tsDatum
}
}
row = append(row,
tableID,
tableName,
Expand All @@ -2631,6 +2639,7 @@ CREATE TABLE crdb_internal.table_indexes (
tree.MakeDBool(tree.DBool(idx.IsUnique())),
tree.MakeDBool(idx.GetType() == descpb.IndexDescriptor_INVERTED),
tree.MakeDBool(tree.DBool(idx.IsSharded())),
createdAt,
)
return pusher.pushRow(row...)
})
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/create_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package sql

import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/docs"
"github.com/cockroachdb/cockroach/pkg/geo/geoindex"
Expand Down Expand Up @@ -188,6 +189,7 @@ func makeIndexDescriptor(
Unique: n.Unique,
StoreColumnNames: n.Storing.ToStrings(),
CreatedExplicitly: true,
CreatedAtNanos: params.EvalContext().GetTxnTimestamp(time.Microsecond).UnixNano(),
}

if n.Inverted {
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/create_sequence.go
Original file line number Diff line number Diff line change
Expand Up @@ -275,6 +275,7 @@ func NewSequenceTableDesc(
KeyColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC},
EncodingType: descpb.PrimaryIndexEncoding,
Version: descpb.LatestPrimaryIndexDescriptorVersion,
CreatedAtNanos: creationTime.WallTime,
})
desc.Families = []descpb.ColumnFamilyDescriptor{
{
Expand Down
12 changes: 12 additions & 0 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"go/constant"
"strconv"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/docs"
Expand Down Expand Up @@ -2117,6 +2118,17 @@ func NewTableDesc(
return nil, err
}

// Note that due to historical reasons, the automatic creation of the primary
// index occurs in AllocateIDs. That call does not have access to the current
// timestamp to set the created_at timestamp.
if desc.IsPhysicalTable() && !catalog.IsSystemDescriptor(&desc) {
ts := evalCtx.GetTxnTimestamp(time.Microsecond).UnixNano()
_ = catalog.ForEachNonDropIndex(&desc, func(idx catalog.Index) error {
idx.IndexDesc().CreatedAtNanos = ts
return nil
})
}

// Record the types of indexes that the table has.
if err := catalog.ForEachNonDropIndex(&desc, func(idx catalog.Index) error {
if idx.IsSharded() {
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
Expand Up @@ -255,10 +255,10 @@ SELECT * FROM crdb_internal.table_columns WHERE descriptor_name = ''
----
descriptor_id descriptor_name column_id column_name column_type nullable default_expr hidden

query ITITTBBB colnames
query ITITTBBBT colnames
SELECT * FROM crdb_internal.table_indexes WHERE descriptor_name = ''
----
descriptor_id descriptor_name index_id index_name index_type is_unique is_inverted is_sharded
descriptor_id descriptor_name index_id index_name index_type is_unique is_inverted is_sharded created_at

query ITITTITTB colnames
SELECT * FROM crdb_internal.index_columns WHERE descriptor_name = ''
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/create_statements
Original file line number Diff line number Diff line change
Expand Up @@ -1435,7 +1435,8 @@ CREATE TABLE crdb_internal.table_indexes (
index_type STRING NOT NULL,
is_unique BOOL NOT NULL,
is_inverted BOOL NOT NULL,
is_sharded BOOL NOT NULL
is_sharded BOOL NOT NULL,
created_at TIMESTAMP NULL
) CREATE TABLE crdb_internal.table_indexes (
descriptor_id INT8 NULL,
descriptor_name STRING NOT NULL,
Expand All @@ -1444,7 +1445,8 @@ CREATE TABLE crdb_internal.table_indexes (
index_type STRING NOT NULL,
is_unique BOOL NOT NULL,
is_inverted BOOL NOT NULL,
is_sharded BOOL NOT NULL
is_sharded BOOL NOT NULL,
created_at TIMESTAMP NULL
) {} {}
CREATE TABLE crdb_internal.table_row_statistics (
table_id INT8 NOT NULL,
Expand Down
Loading

0 comments on commit 80018e9

Please sign in to comment.