90 Star 494 Fork 152

平凯星辰(北京)科技有限公司/tidb

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
文件
克隆/下载
column.go 4.06 KB
一键复制 编辑 原始数据 按行查看 历史
// Copyright 2018 PingCAP, Inc.
//
// 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,
// See the License for the specific language governing permissions and
// limitations under the License.
package chunk
import (
"unsafe"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/json"
)
func (c *column) appendDuration(dur types.Duration) {
c.appendInt64(int64(dur.Duration))
}
func (c *column) appendMyDecimal(dec *types.MyDecimal) {
*(*types.MyDecimal)(unsafe.Pointer(&c.elemBuf[0])) = *dec
c.finishAppendFixed()
}
func (c *column) appendNameValue(name string, val uint64) {
var buf [8]byte
*(*uint64)(unsafe.Pointer(&buf[0])) = val
c.data = append(c.data, buf[:]...)
c.data = append(c.data, name...)
c.finishAppendVar()
}
func (c *column) appendJSON(j json.BinaryJSON) {
c.data = append(c.data, j.TypeCode)
c.data = append(c.data, j.Value...)
c.finishAppendVar()
}
type column struct {
length int
nullCount int
nullBitmap []byte
offsets []int32
data []byte
elemBuf []byte
}
func (c *column) isFixed() bool {
return c.elemBuf != nil
}
func (c *column) reset() {
c.length = 0
c.nullCount = 0
c.nullBitmap = c.nullBitmap[:0]
if len(c.offsets) > 0 {
// The first offset is always 0, it makes slicing the data easier, we need to keep it.
c.offsets = c.offsets[:1]
}
c.data = c.data[:0]
}
func (c *column) isNull(rowIdx int) bool {
nullByte := c.nullBitmap[rowIdx/8]
return nullByte&(1<<(uint(rowIdx)&7)) == 0
}
func (c *column) appendNullBitmap(notNull bool) {
idx := c.length >> 3
if idx >= len(c.nullBitmap) {
c.nullBitmap = append(c.nullBitmap, 0)
}
if notNull {
pos := uint(c.length) & 7
c.nullBitmap[idx] |= byte(1 << pos)
} else {
c.nullCount++
}
}
// appendMultiSameNullBitmap appends multiple same bit value to `nullBitMap`.
// notNull means not null.
// num means the number of bits that should be appended.
func (c *column) appendMultiSameNullBitmap(notNull bool, num int) {
numNewBytes := ((c.length + num + 7) >> 3) - len(c.nullBitmap)
b := byte(0)
if notNull {
b = 0xff
}
for i := 0; i < numNewBytes; i++ {
c.nullBitmap = append(c.nullBitmap, b)
}
if !notNull {
c.nullCount += num
return
}
// 1. Set all the remaining bits in the last slot of old c.numBitMap to 1.
numRemainingBits := uint(c.length % 8)
bitMask := byte(^((1 << numRemainingBits) - 1))
c.nullBitmap[c.length/8] |= bitMask
// 2. Set all the redundant bits in the last slot of new c.numBitMap to 0.
numRedundantBits := uint(len(c.nullBitmap)*8 - c.length - num)
bitMask = byte(1<<(8-numRedundantBits)) - 1
c.nullBitmap[len(c.nullBitmap)-1] &= bitMask
}
func (c *column) appendNull() {
c.appendNullBitmap(false)
if c.isFixed() {
c.data = append(c.data, c.elemBuf...)
} else {
c.offsets = append(c.offsets, c.offsets[c.length])
}
c.length++
}
func (c *column) finishAppendFixed() {
c.data = append(c.data, c.elemBuf...)
c.appendNullBitmap(true)
c.length++
}
func (c *column) appendInt64(i int64) {
*(*int64)(unsafe.Pointer(&c.elemBuf[0])) = i
c.finishAppendFixed()
}
func (c *column) appendUint64(u uint64) {
*(*uint64)(unsafe.Pointer(&c.elemBuf[0])) = u
c.finishAppendFixed()
}
func (c *column) appendFloat32(f float32) {
*(*float32)(unsafe.Pointer(&c.elemBuf[0])) = f
c.finishAppendFixed()
}
func (c *column) appendFloat64(f float64) {
*(*float64)(unsafe.Pointer(&c.elemBuf[0])) = f
c.finishAppendFixed()
}
func (c *column) finishAppendVar() {
c.appendNullBitmap(true)
c.offsets = append(c.offsets, int32(len(c.data)))
c.length++
}
func (c *column) appendString(str string) {
c.data = append(c.data, str...)
c.finishAppendVar()
}
func (c *column) appendBytes(b []byte) {
c.data = append(c.data, b...)
c.finishAppendVar()
}
func (c *column) appendTime(t types.Time) {
writeTime(c.elemBuf, t)
c.finishAppendFixed()
}
Loading...
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
Go
1
https://gitee.com/pingcap/tidb.git
git@gitee.com:pingcap/tidb.git
pingcap
tidb
tidb
v2.1.0-rc.4

搜索帮助