-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
Copy pathconstants.go
277 lines (246 loc) · 12.6 KB
/
constants.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
// Copyright 2015 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.
//
// Author: Peter Mattis ([email protected])
package keys
import (
"math"
"github.com/cockroachdb/cockroach/roachpb"
"github.com/cockroachdb/cockroach/util/encoding"
)
// These constants are single bytes for performance. They allow single-byte
// comparisons which are considerably faster than bytes.HasPrefix.
const (
localPrefixByte = '\x01'
localMaxByte = '\x02'
meta1PrefixByte = localMaxByte
meta2PrefixByte = '\x03'
metaMaxByte = '\x04'
systemPrefixByte = metaMaxByte
systemMaxByte = '\x05'
)
// Constants for system-reserved keys in the KV map.
//
// Note: preserve group-wise ordering when adding new constants.
var (
// localPrefix is the prefix for keys which hold data local to a
// RocksDB instance, such as store and range-specific metadata which
// must not pollute the user key space, but must be collocate with
// the store and/or ranges which they refer to. Storing this
// information in the normal system keyspace would place the data on
// an arbitrary set of stores, with no guarantee of collocation.
// Local data includes store metadata, range metadata, abort
// cache values, transaction records, range-spanning binary tree
// node pointers, and message queues.
//
// The local key prefix has been deliberately chosen to sort before
// the SystemPrefix, because these local keys are not addressable
// via the meta range addressing indexes.
//
// Some local data are not replicated, such as the store's 'ident'
// record. Most local data are replicated, such as abort cache
// entries and transaction rows, but are not addressable as normal
// MVCC values as part of transactions. Finally, some local data are
// stored as MVCC values and are addressable as part of distributed
// transactions, such as range metadata, range-spanning binary tree
// node pointers, and message queues.
localPrefix = roachpb.Key{localPrefixByte}
// LocalMax is the end of the local key range. It is itself a global
// key.
LocalMax = roachpb.Key{localMaxByte}
// localSuffixLength specifies the length in bytes of all local
// key suffixes.
localSuffixLength = 4
// There are three types of local key data enumerated below:
// store-local, range-local by ID, and range-local by key.
// localStorePrefix is the prefix identifying per-store data.
localStorePrefix = makeKey(localPrefix, roachpb.Key("s"))
// localStoreIdentSuffix stores an immutable identifier for this
// store, created when the store is first bootstrapped.
localStoreIdentSuffix = []byte("iden")
// localStoreGossipSuffix stores gossip bootstrap metadata for this
// store, updated any time new gossip hosts are encountered.
localStoreGossipSuffix = []byte("goss")
// LocalRangeIDPrefix is the prefix identifying per-range data
// indexed by Range ID. The Range ID is appended to this prefix,
// encoded using EncodeUvarint. The specific sort of per-range
// metadata is identified by one of the suffixes listed below, along
// with potentially additional encoded key info, for instance in the
// case of abort cache entry.
//
// NOTE: LocalRangeIDPrefix must be kept in sync with the value
// in storage/engine/rocksdb/db.cc.
LocalRangeIDPrefix = roachpb.RKey(makeKey(localPrefix, roachpb.Key("i")))
// localRangeIDReplicatedInfix is the post-Range ID specifier for all Raft
// replicated per-range data. By appending this after the Range ID, these
// keys will be sorted directly before the local unreplicated keys for the
// same Range ID, so they can be manipulated either together or individually
// in a single scan.
localRangeIDReplicatedInfix = []byte("r")
// LocalAbortCacheSuffix is the suffix for abort cache entries. The
// abort cache protects a transaction from re-reading its own intents
// after it's been aborted.
LocalAbortCacheSuffix = []byte("abc-")
// localRangeFrozenStatusSuffix is the suffix for a frozen status.
LocalRangeFrozenStatusSuffix = []byte("fzn-")
// localRangeLastGCSuffix is the suffix for the last GC.
LocalRangeLastGCSuffix = []byte("lgc-")
// LocalRaftAppliedIndexSuffix is the suffix for the raft applied index.
LocalRaftAppliedIndexSuffix = []byte("rfta")
// localRaftTombstoneSuffix is the suffix for the raft tombstone.
LocalRaftTombstoneSuffix = []byte("rftb")
// localRaftTruncatedStateSuffix is the suffix for the RaftTruncatedState.
LocalRaftTruncatedStateSuffix = []byte("rftt")
// localRangeLeaseSuffix is the suffix for a range lease.
LocalRangeLeaseSuffix = []byte("rll-")
// LocalLeaseAppliedIndexSuffix is the suffix for the applied lease index.
LocalLeaseAppliedIndexSuffix = []byte("rlla")
// localRangeStatsSuffix is the suffix for range statistics.
LocalRangeStatsSuffix = []byte("stat")
// LocalTxnSpanGCThresholdSuffix is the suffix for the last txn span GC's
// threshold.
LocalTxnSpanGCThresholdSuffix = []byte("tst-")
// localRangeIDUnreplicatedInfix is the post-Range ID specifier for all
// per-range data that is not fully Raft replicated. By appending this
// after the Range ID, these keys will be sorted directly after the local
// replicated keys for the same Range ID, so they can be manipulated either
// together or individually in a single scan.
localRangeIDUnreplicatedInfix = []byte("u")
// localRaftHardStateSuffix is the Suffix for the raft HardState.
LocalRaftHardStateSuffix = []byte("rfth")
// localRaftLastIndexSuffix is the suffix for raft's last index.
LocalRaftLastIndexSuffix = []byte("rfti")
// LocalRaftLogSuffix is the suffix for the raft log.
LocalRaftLogSuffix = []byte("rftl")
// localRangeLastReplicaGCTimestampSuffix is the suffix for a range's
// last replica GC timestamp (for GC of old replicas).
LocalRangeLastReplicaGCTimestampSuffix = []byte("rlrt")
// localRangeLastVerificationTimestampSuffixDeprecated is the suffix for a range's
// last verification timestamp (for checking integrity of on-disk data).
// Note: DEPRECATED.
LocalRangeLastVerificationTimestampSuffixDeprecated = []byte("rlvt")
// LocalRangeReplicaDestroyedErrorSuffix is the suffix for a range's replica
// destroyed error (for marking replicas as dead).
LocalRangeReplicaDestroyedErrorSuffix = []byte("rrde")
// LocalRangePrefix is the prefix identifying per-range data indexed
// by range key (either start key, or some key in the range). The
// key is appended to this prefix, encoded using EncodeBytes. The
// specific sort of per-range metadata is identified by one of the
// suffixes listed below, along with potentially additional encoded
// key info, such as the txn ID in the case of a transaction record.
//
// NOTE: LocalRangePrefix must be kept in sync with the value in
// storage/engine/rocksdb/db.cc.
LocalRangePrefix = roachpb.Key(makeKey(localPrefix, roachpb.RKey("k")))
LocalRangeMax = LocalRangePrefix.PrefixEnd()
// LocalRangeDescriptorSuffix is the suffix for keys storing
// range descriptors. The value is a struct of type RangeDescriptor.
LocalRangeDescriptorSuffix = roachpb.RKey("rdsc")
// localTransactionSuffix specifies the key suffix for
// transaction records. The additional detail is the transaction id.
// NOTE: if this value changes, it must be updated in C++
// (storage/engine/rocksdb/db.cc).
localTransactionSuffix = roachpb.RKey("txn-")
// Meta1Prefix is the first level of key addressing. It is selected such that
// all range addressing records sort before any system tables which they
// might describe. The value is a RangeDescriptor struct.
Meta1Prefix = roachpb.Key{meta1PrefixByte}
// Meta2Prefix is the second level of key addressing. The value is a
// RangeDescriptor struct.
Meta2Prefix = roachpb.Key{meta2PrefixByte}
// Meta1KeyMax is the end of the range of the first level of key addressing.
// The value is a RangeDescriptor struct.
Meta1KeyMax = roachpb.Key(makeKey(Meta1Prefix, roachpb.RKeyMax))
// Meta2KeyMax is the end of the range of the second level of key addressing.
// The value is a RangeDescriptor struct.
Meta2KeyMax = roachpb.Key(makeKey(Meta2Prefix, roachpb.RKeyMax))
// MetaMin is the start of the range of addressing keys.
MetaMin = Meta1Prefix
// MetaMax is the end of the range of addressing keys.
MetaMax = roachpb.Key{metaMaxByte}
// SystemPrefix indicates the beginning of the key range for
// global, system data which are replicated across the cluster.
SystemPrefix = roachpb.Key{systemPrefixByte}
SystemMax = roachpb.Key{systemMaxByte}
// NodeLivenessPrefix specifies the key prefix for the node liveness
// table. Note that this should sort before the rest of the system
// keyspace in order to limit the number of ranges which must use
// expiration-based range leases instead of the more efficient
// node-liveness epoch-based range leases (see
// https://github.com/cockroachdb/cockroach/blob/develop/docs/RFCS/range_leases.md)
NodeLivenessPrefix = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("\x00liveness-")))
// NodeLivenessKeyMax is the maximum value for any node liveness key.
NodeLivenessKeyMax = NodeLivenessPrefix.PrefixEnd()
// DescIDGenerator is the global descriptor ID generator sequence used for
// table and namespace IDs.
DescIDGenerator = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("desc-idgen")))
// NodeIDGenerator is the global node ID generator sequence.
NodeIDGenerator = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("node-idgen")))
// RangeIDGenerator is the global range ID generator sequence.
RangeIDGenerator = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("range-idgen")))
// StoreIDGenerator is the global store ID generator sequence.
StoreIDGenerator = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("store-idgen")))
// StatusPrefix specifies the key prefix to store all status details.
StatusPrefix = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("status-")))
// StatusNodePrefix stores all status info for nodes.
StatusNodePrefix = roachpb.Key(makeKey(StatusPrefix, roachpb.RKey("node-")))
// TimeseriesPrefix is the key prefix for all timeseries data.
TimeseriesPrefix = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("tsd")))
// UpdateCheckPrefix is the key prefix for all update check times.
UpdateCheckPrefix = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("update-")))
UpdateCheckCluster = roachpb.Key(makeKey(UpdateCheckPrefix, roachpb.RKey("cluster")))
// TableDataMin is the start of the range of table data keys.
TableDataMin = roachpb.Key(encoding.EncodeVarintAscending(nil, math.MinInt64))
// TableDataMin is the end of the range of table data keys.
TableDataMax = roachpb.Key(encoding.EncodeVarintAscending(nil, math.MaxInt64))
// SystemConfigTableDataMax is the end key of system config structured data.
SystemConfigTableDataMax = roachpb.Key(MakeTablePrefix(MaxSystemConfigDescID + 1))
// UserTableDataMin is the start key of user structured data.
UserTableDataMin = roachpb.Key(MakeTablePrefix(MaxReservedDescID + 1))
// MaxKey is the infinity marker which is larger than any other key.
MaxKey = roachpb.KeyMax
// MinKey is a minimum key value which sorts before all other keys.
MinKey = roachpb.KeyMin
)
// Various IDs used by the structured data layer.
// NOTE: these must not change during the lifetime of a cluster.
const (
// MaxSystemConfigDescID is the maximum system descriptor ID that will be
// gossiped as part of the SystemConfig. Be careful adding new descriptors to
// this ID range.
MaxSystemConfigDescID = 10
// MaxReservedDescID is the maximum value of reserved descriptor
// IDs. Reserved IDs are used by namespaces and tables used internally by
// cockroach.
MaxReservedDescID = 49
// VirtualDescriptorID is the ID used by all virtual descriptors.
VirtualDescriptorID = math.MaxUint32
// RootNamespaceID is the ID of the root namespace.
RootNamespaceID = 0
// SystemDatabaseID and following are the database/table IDs for objects
// in the system span.
// NOTE: IDs must be <= MaxSystemConfigDescID.
SystemDatabaseID = 1
NamespaceTableID = 2
DescriptorTableID = 3
UsersTableID = 4
ZonesTableID = 5
// Reserved IDs for other system tables. If you're adding a new system table,
// it probably belongs here.
// NOTE: IDs must be <= MaxReservedDescID.
LeaseTableID = 11
EventLogTableID = 12
RangeEventTableID = 13
UITableID = 14
)