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 Feb 9, 2022
1 parent c7446c5 commit b732f2d
Show file tree
Hide file tree
Showing 42 changed files with 336 additions and 165 deletions.
1 change: 1 addition & 0 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -4181,6 +4181,7 @@ Response object returned by TableIndexStatsResponse.
| index_name | [string](#cockroach.server.serverpb.TableIndexStatsResponse-string) | | index_name is the name of the index. | [reserved](#support-status) |
| index_type | [string](#cockroach.server.serverpb.TableIndexStatsResponse-string) | | index_type is the type of the index i.e. primary, secondary. | [reserved](#support-status) |
| create_statement | [string](#cockroach.server.serverpb.TableIndexStatsResponse-string) | | create_statement is the SQL statement that would re-create the current index if executed. | [reserved](#support-status) |
| created_at | [google.protobuf.Timestamp](#cockroach.server.serverpb.TableIndexStatsResponse-google.protobuf.Timestamp) | | CreatedAt is an approximate timestamp at which the index was created. Note that it may not always be populated. | [reserved](#support-status) |



Expand Down
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
4 changes: 2 additions & 2 deletions pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant
Original file line number Diff line number Diff line change
Expand Up @@ -275,10 +275,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
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
Loading

0 comments on commit b732f2d

Please sign in to comment.