github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/colencoding/key_encoding.go (about) 1 // Copyright 2018 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 11 package colencoding 12 13 import ( 14 "time" 15 16 "github.com/cockroachdb/apd" 17 "github.com/cockroachdb/cockroach/pkg/col/coldata" 18 "github.com/cockroachdb/cockroach/pkg/roachpb" 19 "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" 20 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 21 "github.com/cockroachdb/cockroach/pkg/sql/types" 22 "github.com/cockroachdb/cockroach/pkg/util" 23 "github.com/cockroachdb/cockroach/pkg/util/duration" 24 "github.com/cockroachdb/cockroach/pkg/util/encoding" 25 "github.com/cockroachdb/cockroach/pkg/util/log" 26 "github.com/cockroachdb/errors" 27 ) 28 29 // DecodeIndexKeyToCols decodes an index key into the idx'th position of the 30 // provided slices of colexec.Vecs. The input index key must already have its 31 // tenant id and first table id / index id prefix removed. If matches is false, 32 // the key is from a different table, and the returned remainingKey indicates a 33 // "seek prefix": the next key that might be part of the table being searched 34 // for. The input key will also be mutated if matches is false. See the analog 35 // in sqlbase/index_encoding.go. 36 func DecodeIndexKeyToCols( 37 da *sqlbase.DatumAlloc, 38 vecs []coldata.Vec, 39 idx int, 40 desc *sqlbase.ImmutableTableDescriptor, 41 index *sqlbase.IndexDescriptor, 42 indexColIdx []int, 43 types []*types.T, 44 colDirs []sqlbase.IndexDescriptor_Direction, 45 key roachpb.Key, 46 ) (remainingKey roachpb.Key, matches bool, foundNull bool, _ error) { 47 var decodedTableID sqlbase.ID 48 var decodedIndexID sqlbase.IndexID 49 var err error 50 51 origKey := key 52 53 if len(index.Interleave.Ancestors) > 0 { 54 for i, ancestor := range index.Interleave.Ancestors { 55 // Our input key had its first table id / index id chopped off, so 56 // don't try to decode those for the first ancestor. 57 if i != 0 { 58 key, decodedTableID, decodedIndexID, err = sqlbase.DecodePartialTableIDIndexID(key) 59 if err != nil { 60 return nil, false, false, err 61 } 62 if decodedTableID != ancestor.TableID || decodedIndexID != ancestor.IndexID { 63 // We don't match. Return a key with the table ID / index ID we're 64 // searching for, so the caller knows what to seek to. 65 curPos := len(origKey) - len(key) 66 key = sqlbase.EncodePartialTableIDIndexID(origKey[:curPos], ancestor.TableID, ancestor.IndexID) 67 return key, false, false, nil 68 } 69 } 70 71 length := int(ancestor.SharedPrefixLen) 72 // We don't care about whether this call to DecodeKeyVals found a null or not, because 73 // it is a interleaving ancestor. 74 var isNull bool 75 key, isNull, err = DecodeKeyValsToCols( 76 da, vecs, idx, indexColIdx[:length], types[:length], 77 colDirs[:length], nil /* unseen */, key, 78 ) 79 if err != nil { 80 return nil, false, false, err 81 } 82 indexColIdx, types, colDirs = indexColIdx[length:], types[length:], colDirs[length:] 83 foundNull = foundNull || isNull 84 85 // Consume the interleaved sentinel. 86 var ok bool 87 key, ok = encoding.DecodeIfInterleavedSentinel(key) 88 if !ok { 89 // We're expecting an interleaved sentinel but didn't find one. Append 90 // one so the caller can seek to it. 91 curPos := len(origKey) - len(key) 92 key = encoding.EncodeInterleavedSentinel(origKey[:curPos]) 93 return key, false, false, nil 94 } 95 } 96 97 key, decodedTableID, decodedIndexID, err = sqlbase.DecodePartialTableIDIndexID(key) 98 if err != nil { 99 return nil, false, false, err 100 } 101 if decodedTableID != desc.ID || decodedIndexID != index.ID { 102 // We don't match. Return a key with the table ID / index ID we're 103 // searching for, so the caller knows what to seek to. 104 curPos := len(origKey) - len(key) 105 key = sqlbase.EncodePartialTableIDIndexID(origKey[:curPos], desc.ID, index.ID) 106 return key, false, false, nil 107 } 108 } 109 110 var isNull bool 111 key, isNull, err = DecodeKeyValsToCols( 112 da, vecs, idx, indexColIdx, types, colDirs, nil /* unseen */, key, 113 ) 114 if err != nil { 115 return nil, false, false, err 116 } 117 foundNull = foundNull || isNull 118 119 // We're expecting a column family id next (a varint). If 120 // interleavedSentinel is actually next, then this key is for a child 121 // table. 122 if _, ok := encoding.DecodeIfInterleavedSentinel(key); ok { 123 curPos := len(origKey) - len(key) 124 key = encoding.EncodeNullDescending(origKey[:curPos]) 125 return key, false, false, nil 126 } 127 128 return key, true, foundNull, nil 129 } 130 131 // DecodeKeyValsToCols decodes the values that are part of the key, writing the 132 // result to the idx'th slot of the input slice of colexec.Vecs. If the 133 // directions slice is nil, the direction used will default to 134 // encoding.Ascending. 135 // If the unseen int set is non-nil, upon decoding the column with ordinal i, 136 // i will be removed from the set to facilitate tracking whether or not columns 137 // have been observed during decoding. 138 // See the analog in sqlbase/index_encoding.go. 139 // DecodeKeyValsToCols additionally returns whether a NULL was encountered when decoding. 140 func DecodeKeyValsToCols( 141 da *sqlbase.DatumAlloc, 142 vecs []coldata.Vec, 143 idx int, 144 indexColIdx []int, 145 types []*types.T, 146 directions []sqlbase.IndexDescriptor_Direction, 147 unseen *util.FastIntSet, 148 key []byte, 149 ) ([]byte, bool, error) { 150 foundNull := false 151 for j := range types { 152 enc := sqlbase.IndexDescriptor_ASC 153 if directions != nil { 154 enc = directions[j] 155 } 156 var err error 157 i := indexColIdx[j] 158 if i == -1 { 159 // Don't need the coldata - skip it. 160 key, err = sqlbase.SkipTableKey(key) 161 } else { 162 if unseen != nil { 163 unseen.Remove(i) 164 } 165 var isNull bool 166 key, isNull, err = decodeTableKeyToCol(da, vecs[i], idx, types[j], key, enc) 167 foundNull = isNull || foundNull 168 } 169 if err != nil { 170 return nil, false, err 171 } 172 } 173 return key, foundNull, nil 174 } 175 176 // decodeTableKeyToCol decodes a value encoded by EncodeTableKey, writing the result 177 // to the idx'th slot of the input colexec.Vec. 178 // See the analog, DecodeTableKey, in sqlbase/column_type_encoding.go. 179 // decodeTableKeyToCol also returns whether or not the decoded value was NULL. 180 func decodeTableKeyToCol( 181 da *sqlbase.DatumAlloc, 182 vec coldata.Vec, 183 idx int, 184 valType *types.T, 185 key []byte, 186 dir sqlbase.IndexDescriptor_Direction, 187 ) ([]byte, bool, error) { 188 if (dir != sqlbase.IndexDescriptor_ASC) && (dir != sqlbase.IndexDescriptor_DESC) { 189 return nil, false, errors.AssertionFailedf("invalid direction: %d", log.Safe(dir)) 190 } 191 var isNull bool 192 if key, isNull = encoding.DecodeIfNull(key); isNull { 193 vec.Nulls().SetNull(idx) 194 return key, true, nil 195 } 196 // We might have read a NULL value in the interleaved child table which 197 // would update the nulls vector, so we need to explicitly unset the null 198 // value here. 199 vec.Nulls().UnsetNull(idx) 200 201 var rkey []byte 202 var err error 203 switch valType.Family() { 204 case types.BoolFamily: 205 var i int64 206 if dir == sqlbase.IndexDescriptor_ASC { 207 rkey, i, err = encoding.DecodeVarintAscending(key) 208 } else { 209 rkey, i, err = encoding.DecodeVarintDescending(key) 210 } 211 vec.Bool()[idx] = i != 0 212 case types.IntFamily, types.DateFamily, types.OidFamily: 213 var i int64 214 if dir == sqlbase.IndexDescriptor_ASC { 215 rkey, i, err = encoding.DecodeVarintAscending(key) 216 } else { 217 rkey, i, err = encoding.DecodeVarintDescending(key) 218 } 219 switch valType.Width() { 220 case 16: 221 vec.Int16()[idx] = int16(i) 222 case 32: 223 vec.Int32()[idx] = int32(i) 224 case 0, 64: 225 vec.Int64()[idx] = i 226 } 227 case types.FloatFamily: 228 var f float64 229 if dir == sqlbase.IndexDescriptor_ASC { 230 rkey, f, err = encoding.DecodeFloatAscending(key) 231 } else { 232 rkey, f, err = encoding.DecodeFloatDescending(key) 233 } 234 vec.Float64()[idx] = f 235 case types.DecimalFamily: 236 var d apd.Decimal 237 if dir == sqlbase.IndexDescriptor_ASC { 238 rkey, d, err = encoding.DecodeDecimalAscending(key, nil) 239 } else { 240 rkey, d, err = encoding.DecodeDecimalDescending(key, nil) 241 } 242 vec.Decimal()[idx] = d 243 case types.BytesFamily, types.StringFamily, types.UuidFamily: 244 var r []byte 245 if dir == sqlbase.IndexDescriptor_ASC { 246 rkey, r, err = encoding.DecodeBytesAscending(key, nil) 247 } else { 248 rkey, r, err = encoding.DecodeBytesDescending(key, nil) 249 } 250 vec.Bytes().Set(idx, r) 251 case types.TimestampFamily, types.TimestampTZFamily: 252 var t time.Time 253 if dir == sqlbase.IndexDescriptor_ASC { 254 rkey, t, err = encoding.DecodeTimeAscending(key) 255 } else { 256 rkey, t, err = encoding.DecodeTimeDescending(key) 257 } 258 vec.Timestamp()[idx] = t 259 case types.IntervalFamily: 260 var d duration.Duration 261 if dir == sqlbase.IndexDescriptor_ASC { 262 rkey, d, err = encoding.DecodeDurationAscending(key) 263 } else { 264 rkey, d, err = encoding.DecodeDurationDescending(key) 265 } 266 vec.Interval()[idx] = d 267 default: 268 var d tree.Datum 269 encDir := encoding.Ascending 270 if dir == sqlbase.IndexDescriptor_DESC { 271 encDir = encoding.Descending 272 } 273 d, rkey, err = sqlbase.DecodeTableKey(da, valType, key, encDir) 274 vec.Datum().Set(idx, d) 275 } 276 return rkey, false, err 277 } 278 279 // UnmarshalColumnValueToCol decodes the value from a roachpb.Value using the 280 // type expected by the column, writing into the input Vec at the given row 281 // idx. An error is returned if the value's type does not match the column's 282 // type. 283 // See the analog, UnmarshalColumnValue, in sqlbase/column_type_encoding.go 284 func UnmarshalColumnValueToCol( 285 da *sqlbase.DatumAlloc, vec coldata.Vec, idx int, typ *types.T, value roachpb.Value, 286 ) error { 287 if value.RawBytes == nil { 288 vec.Nulls().SetNull(idx) 289 } 290 291 var err error 292 switch typ.Family() { 293 case types.BoolFamily: 294 var v bool 295 v, err = value.GetBool() 296 vec.Bool()[idx] = v 297 case types.IntFamily: 298 var v int64 299 v, err = value.GetInt() 300 switch typ.Width() { 301 case 16: 302 vec.Int16()[idx] = int16(v) 303 case 32: 304 vec.Int32()[idx] = int32(v) 305 default: 306 // Pre-2.1 BIT was using INT encoding with arbitrary sizes. 307 // We map these to 64-bit INT now. See #34161. 308 vec.Int64()[idx] = v 309 } 310 case types.FloatFamily: 311 var v float64 312 v, err = value.GetFloat() 313 vec.Float64()[idx] = v 314 case types.DecimalFamily: 315 err = value.GetDecimalInto(&vec.Decimal()[idx]) 316 case types.BytesFamily, types.StringFamily, types.UuidFamily: 317 var v []byte 318 v, err = value.GetBytes() 319 vec.Bytes().Set(idx, v) 320 case types.DateFamily, types.OidFamily: 321 var v int64 322 v, err = value.GetInt() 323 vec.Int64()[idx] = v 324 case types.TimestampFamily, types.TimestampTZFamily: 325 var v time.Time 326 v, err = value.GetTime() 327 vec.Timestamp()[idx] = v 328 case types.IntervalFamily: 329 var v duration.Duration 330 v, err = value.GetDuration() 331 vec.Interval()[idx] = v 332 // Types backed by tree.Datums. 333 default: 334 var d tree.Datum 335 d, err = sqlbase.UnmarshalColumnValue(da, typ, value) 336 if err != nil { 337 return err 338 } 339 vec.Datum().Set(idx, d) 340 } 341 return err 342 }