diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index ee7e10c19b30..7641ee5b8f2e 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -464,6 +464,7 @@ go_test( "//pkg/spanconfig", "//pkg/sql", "//pkg/sql/catalog/descpb", + "//pkg/sql/clusterunique", "//pkg/sql/execinfrapb", "//pkg/sql/idxusage", "//pkg/sql/roleoption", diff --git a/pkg/server/status_test.go b/pkg/server/status_test.go index 7e16497e4278..84006153dc6e 100644 --- a/pkg/server/status_test.go +++ b/pkg/server/status_test.go @@ -45,6 +45,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -3733,6 +3734,9 @@ func TestTransactionContentionEvents(t *testing.T) { } for _, event := range resp.Events { + require.NotEqual(t, event.WaitingStmtFingerprintID, 0) + require.NotEqual(t, event.WaitingStmtID.String(), clusterunique.ID{}.String()) + require.Equal(t, tc.canViewContendingKey, len(event.BlockingEvent.Key) > 0, "expected to %s, but the contending key has length of %d", expectationStr, diff --git a/pkg/sql/clusterunique/id.go b/pkg/sql/clusterunique/id.go index 5fa675603db0..8e3a45b4a063 100644 --- a/pkg/sql/clusterunique/id.go +++ b/pkg/sql/clusterunique/id.go @@ -11,6 +11,8 @@ package clusterunique import ( + "encoding/json" + "github.com/biogo/store/llrb" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -75,6 +77,27 @@ func (id *ID) Unmarshal(data []byte) error { return nil } +// MarshalJSON returns the JSON encoding of u. +func (id ID) MarshalJSON() ([]byte, error) { + return json.Marshal(id.String()) +} + +// UnmarshalJSON unmarshal the JSON encoded data into u. +func (id *ID) UnmarshalJSON(data []byte) error { + var uint128String string + if err := json.Unmarshal(data, &uint128String); err != nil { + return err + } + + uint128, err := uint128.FromString(uint128String) + if err != nil { + return err + } + + id.Uint128 = uint128 + return nil +} + // ID implements llrb.Comparable. // While these IDs don't really have a global ordering, it is convenient to // be able to use them as keys in our `cache.OrderedCache`: their ordering diff --git a/pkg/sql/contentionpb/BUILD.bazel b/pkg/sql/contentionpb/BUILD.bazel index c0ebaf97fba1..52389c66ed97 100644 --- a/pkg/sql/contentionpb/BUILD.bazel +++ b/pkg/sql/contentionpb/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/contentionpb", visibility = ["//visibility:public"], deps = [ + "//pkg/sql/clusterunique", "//pkg/util", "//pkg/util/encoding", "//pkg/util/uuid", @@ -38,6 +39,7 @@ go_proto_library( deps = [ "//pkg/roachpb", # keep "//pkg/sql/catalog/descpb", # keep + "//pkg/sql/clusterunique", # keep "//pkg/util/uuid", # keep "@com_github_gogo_protobuf//gogoproto", ], @@ -49,8 +51,9 @@ go_test( args = ["-test.timeout=295s"], embed = [":contentionpb"], deps = [ + "//pkg/sql/clusterunique", "//pkg/util", - "//pkg/util/timeutil", + "//pkg/util/uint128", "//pkg/util/uuid", "@com_github_stretchr_testify//require", ], diff --git a/pkg/sql/contentionpb/contention.go b/pkg/sql/contentionpb/contention.go index a927811a7f8a..7de0c0682895 100644 --- a/pkg/sql/contentionpb/contention.go +++ b/pkg/sql/contentionpb/contention.go @@ -14,6 +14,7 @@ import ( "fmt" "strings" + "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -78,15 +79,23 @@ func (e *ExtendedContentionEvent) Valid() bool { } // Hash returns a hash that's unique to ExtendedContentionEvent using -// blocking txn's txnID, waiting txn's txnID and the event collection timestamp. +// blocking txn's txnID, waiting txn's txnID and the event waiting stmt id. func (e *ExtendedContentionEvent) Hash() uint64 { hash := util.MakeFNV64() hashUUID(e.BlockingEvent.TxnMeta.ID, &hash) hashUUID(e.WaitingTxnID, &hash) - hash.Add(uint64(e.CollectionTs.UnixMilli())) + hashClusterUniqueID(e.WaitingStmtID, &hash) return hash.Sum() } +// hashClusterUniqueID adds the hash of the clusterunique.ID into the fnv. +// A clusterunique.ID is an uint128. To hash we treat it as two uint64 integers, +// since uint128 has a lo and hi uint64. +func hashClusterUniqueID(id clusterunique.ID, hash *util.FNV64) { + hash.Add(id.Lo) + hash.Add(id.Hi) +} + // hashUUID adds the hash of the uuid into the fnv. // An uuid is a 16 byte array. To hash UUID, we treat it as two uint64 integers, // since uint64 is 8-byte. This is why we decode the byte array twice and add diff --git a/pkg/sql/contentionpb/contention.proto b/pkg/sql/contentionpb/contention.proto index e0d1d62410d3..36c860d24841 100644 --- a/pkg/sql/contentionpb/contention.proto +++ b/pkg/sql/contentionpb/contention.proto @@ -170,4 +170,13 @@ message ExtendedContentionEvent { (gogoproto.nullable) = false, (gogoproto.stdtime) = true ]; + + uint64 waiting_stmt_fingerprint_id = 6 [ + (gogoproto.customname) = "WaitingStmtFingerprintID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StmtFingerprintID" + ]; + + bytes waiting_stmt_id = 7 [(gogoproto.customname) = "WaitingStmtID", + (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/sql/clusterunique.ID", + (gogoproto.nullable) = false]; } diff --git a/pkg/sql/contentionpb/contention_test.go b/pkg/sql/contentionpb/contention_test.go index 765fd6dbe027..a00cf616dcdc 100644 --- a/pkg/sql/contentionpb/contention_test.go +++ b/pkg/sql/contentionpb/contention_test.go @@ -12,10 +12,10 @@ package contentionpb import ( "testing" - "time" + "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/uint128" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/stretchr/testify/require" ) @@ -24,7 +24,7 @@ func TestExtendedContentionEventHash(t *testing.T) { event1 := ExtendedContentionEvent{} event1.BlockingEvent.TxnMeta.ID = uuid.FastMakeV4() event1.WaitingTxnID = uuid.FastMakeV4() - event1.CollectionTs = timeutil.Now() + event1.WaitingStmtID = clusterunique.ID{Uint128: uint128.Uint128{Lo: 12, Hi: 987}} eventWithDifferentBlockingTxnID := event1 eventWithDifferentBlockingTxnID.BlockingEvent.TxnMeta.ID = uuid.FastMakeV4() @@ -35,9 +35,11 @@ func TestExtendedContentionEventHash(t *testing.T) { eventWithDifferentWaitingTxnID.WaitingTxnID = uuid.FastMakeV4() require.NotEqual(t, eventWithDifferentWaitingTxnID.Hash(), event1.Hash()) - eventWithDifferentCollectionTs := event1 - eventWithDifferentCollectionTs.CollectionTs = event1.CollectionTs.Add(time.Second) - require.NotEqual(t, eventWithDifferentCollectionTs.Hash(), event1.Hash()) + eventWithDifferentStmtId := event1 + stmtId := event1.WaitingStmtID + stmtId.Hi = 764 + eventWithDifferentStmtId.WaitingStmtID = stmtId + require.NotEqual(t, eventWithDifferentStmtId.Hash(), event1.Hash()) } func TestHashingUUID(t *testing.T) { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 7ef2aae5aaaa..f7e256c05c62 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -6394,7 +6394,10 @@ CREATE TABLE crdb_internal.transaction_contention_events ( waiting_txn_fingerprint_id BYTES NOT NULL, contention_duration INTERVAL NOT NULL, - contending_key BYTES NOT NULL + contending_key BYTES NOT NULL, + + waiting_stmt_id string NOT NULL, + waiting_stmt_fingerprint_id BYTES NOT NULL );`, generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { // Check permission first before making RPC fanout. @@ -6466,15 +6469,22 @@ CREATE TABLE crdb_internal.transaction_contention_events ( tree.DBytes(resp.Events[i].BlockingEvent.Key)) } + waitingStmtFingerprintID := tree.NewDBytes( + tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(resp.Events[i].WaitingStmtFingerprintID)))) + + waitingStmtId := tree.NewDString(hex.EncodeToString(resp.Events[i].WaitingStmtID.GetBytes())) + row = row[:0] row = append(row, collectionTs, // collection_ts tree.NewDUuid(tree.DUuid{UUID: resp.Events[i].BlockingEvent.TxnMeta.ID}), // blocking_txn_id blockingFingerprintID, // blocking_fingerprint_id tree.NewDUuid(tree.DUuid{UUID: resp.Events[i].WaitingTxnID}), // waiting_txn_id - waitingFingerprintID, // waiting_fingerprint_id - contentionDuration, // contention_duration - contendingKey, // contending_key + waitingFingerprintID, // waiting_fingerprint_id + contentionDuration, // contention_duration + contendingKey, // contending_key, + waitingStmtId, // waiting_stmt_id + waitingStmtFingerprintID, // waiting_stmt_fingerprint_id ) if err = pusher.pushRow(row...); err != nil { diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index 2183eaa0c055..603c84fc5c55 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -978,15 +978,21 @@ func TestTxnContentionEventsTable(t *testing.T) { testutils.SucceedsWithin(t, func() error { rows, errVerify := conn.QueryContext(ctx, `SELECT blocking_txn_id, - waiting_txn_id + waiting_txn_id, + waiting_stmt_id, + encode( + waiting_txn_fingerprint_id, 'hex' + ) AS waiting_txn_fingerprint_id FROM crdb_internal.transaction_contention_events tce inner join ( - select + select + fingerprint_id, transaction_fingerprint_id, metadata->'query' as query from crdb_internal.statement_statistics t where metadata->>'query' like 'UPDATE t SET %') stats - on stats.transaction_fingerprint_id = tce.waiting_txn_fingerprint_id`) + on stats.transaction_fingerprint_id = tce.waiting_txn_fingerprint_id + and stats.fingerprint_id = tce.waiting_stmt_fingerprint_id`) if errVerify != nil { return errVerify } @@ -994,12 +1000,24 @@ func TestTxnContentionEventsTable(t *testing.T) { for rows.Next() { rowCount++ - var blocking, waiting string - errVerify = rows.Scan(&blocking, &waiting) + var blockingTxnId, waitingTxnId, waitingStmtId, waitingStmtFingerprint string + errVerify = rows.Scan(&blockingTxnId, &waitingTxnId, &waitingStmtId, &waitingStmtFingerprint) if errVerify != nil { return errVerify } + const defaultIdString = "0x0000000000000000" + if blockingTxnId == defaultIdString { + return fmt.Errorf("transaction_contention_events had default txn blocking id %s, waiting txn id %s", blockingTxnId, waitingTxnId) + } + + if waitingTxnId == defaultIdString { + return fmt.Errorf("transaction_contention_events had default waiting txn id %s, blocking txn id %s", waitingTxnId, blockingTxnId) + } + + if waitingStmtId == defaultIdString { + return fmt.Errorf("transaction_contention_events had default waiting stmt id %s, blocking txn id %s, waiting txn id %s", waitingStmtId, blockingTxnId, waitingTxnId) + } } if rowCount < 1 { diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index 93e5089861b8..e9cfcdf12c95 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -229,8 +229,10 @@ func (ex *connExecutor) recordStatementSummary( if queryLevelStatsOk { for _, ev := range queryLevelStats.ContentionEvents { contentionEvent := contentionpb.ExtendedContentionEvent{ - BlockingEvent: ev, - WaitingTxnID: planner.txn.ID(), + BlockingEvent: ev, + WaitingTxnID: planner.txn.ID(), + WaitingStmtFingerprintID: stmtFingerprintID, + WaitingStmtID: stmt.QueryID, } ex.server.cfg.ContentionRegistry.AddContentionEvent(contentionEvent) diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index 49b497eb52a3..1c5df2411ca3 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -431,7 +431,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 4294967257 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "total_reads", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "last_read", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967257, "name": "index_usage_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967258 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967258, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["descriptor_id"], "name": "index_spans_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["index_id", "start_key", "end_key"], "version": 3}], "name": "index_spans", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967259 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "column_direction", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "implicit", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967259, "name": "index_columns", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967260 {"table": {"columns": [{"id": 1, "name": "collection_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "blocking_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "blocking_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "waiting_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 5, "name": "waiting_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "contention_duration", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "contending_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967260, "name": "transaction_contention_events", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967260 {"table": {"columns": [{"id": 1, "name": "collection_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "blocking_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "blocking_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "waiting_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 5, "name": "waiting_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "contention_duration", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "contending_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 8, "name": "waiting_stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "waiting_stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967260, "name": "transaction_contention_events", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967261 {"table": {"columns": [{"id": 1, "name": "source_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "target_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967261, "name": "gossip_network", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967262 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "epoch", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "expiration", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "draining", "type": {"oid": 16}}, {"id": 5, "name": "decommissioning", "type": {"oid": 16}}, {"id": 6, "name": "membership", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "updated_at", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967262, "name": "gossip_liveness", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967263 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "category", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "description", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "value", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967263, "name": "gossip_alerts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}}