Skip to content

Commit

Permalink
Merge #36919 #36934
Browse files Browse the repository at this point in the history
36919: exec: add null ranges and optimize nulls in the vect. merge joiner r=georgeutsin a=georgeutsin

Added a new member function to coldata.Nulls to allow setting a
range of null values, given a start and end index. This is useful
for example in the merge joiner when we are building the left
groups, since it is faster to write nulls using a bitwise operation
compared to setting each row to null one by one.

This required refactoring Nulls to use a slice of uint64 instead
of int64 for the null bitmap under the hood, since right shifting
a mask (commonly -1) doesn't work as expected for a signed number,
as the sign digit does not get shifted.

Some benchmarks:
```
name                                      old time/op    new time/op    delta
MergeJoiner/rows=1024-8                     40.0µs ± 1%    39.8µs ± 1%  -0.57%  (p=0.023 n=10+10)
MergeJoiner/rows=4096-8                      152µs ± 0%     150µs ± 1%  -1.12%  (p=0.000 n=10+10)
MergeJoiner/rows=16384-8                     591µs ± 3%     580µs ± 1%  -1.88%  (p=0.000 n=9+10)
MergeJoiner/rows=1048576-8                  37.1ms ± 3%    35.4ms ± 2%  -4.48%  (p=0.000 n=9+10)
MergeJoiner/oneSideRepeat-rows=1024-8       41.3µs ± 3%    40.2µs ± 0%  -2.49%  (p=0.000 n=10+10)
MergeJoiner/oneSideRepeat-rows=4096-8        154µs ± 1%     150µs ± 1%  -2.21%  (p=0.000 n=9+10)
MergeJoiner/oneSideRepeat-rows=16384-8       592µs ± 1%     587µs ± 1%  -0.76%  (p=0.008 n=9+10)
MergeJoiner/oneSideRepeat-rows=1048576-8    37.6ms ± 7%    37.4ms ± 3%    ~     (p=0.853 n=10+10)
MergeJoiner/bothSidesRepeat-rows=1024-8     43.0µs ± 0%    42.7µs ± 0%  -0.87%  (p=0.000 n=8+9)
MergeJoiner/bothSidesRepeat-rows=4096-8      189µs ± 1%     186µs ± 1%  -1.59%  (p=0.000 n=10+9)
MergeJoiner/bothSidesRepeat-rows=16384-8    1.42ms ± 2%    1.39ms ± 0%  -2.03%  (p=0.000 n=10+10)
MergeJoiner/bothSidesRepeat-rows=32768-8    4.86ms ± 2%    4.80ms ± 1%  -1.26%  (p=0.001 n=10+10)

name                                      old speed      new speed      delta
MergeJoiner/rows=1024-8                   1.64GB/s ± 1%  1.65GB/s ± 1%  +0.58%  (p=0.023 n=10+10)
MergeJoiner/rows=4096-8                   1.73GB/s ± 0%  1.75GB/s ± 1%  +1.14%  (p=0.000 n=10+10)
MergeJoiner/rows=16384-8                  1.77GB/s ± 3%  1.81GB/s ± 1%  +1.90%  (p=0.000 n=9+10)
MergeJoiner/rows=1048576-8                1.81GB/s ± 3%  1.89GB/s ± 2%  +4.68%  (p=0.000 n=9+10)
MergeJoiner/oneSideRepeat-rows=1024-8     1.59GB/s ± 3%  1.63GB/s ± 0%  +2.53%  (p=0.000 n=10+10)
MergeJoiner/oneSideRepeat-rows=4096-8     1.71GB/s ± 1%  1.75GB/s ± 1%  +2.25%  (p=0.000 n=9+10)
MergeJoiner/oneSideRepeat-rows=16384-8    1.77GB/s ± 1%  1.79GB/s ± 1%  +0.76%  (p=0.008 n=9+10)
MergeJoiner/oneSideRepeat-rows=1048576-8  1.79GB/s ± 6%  1.80GB/s ± 3%    ~     (p=0.853 n=10+10)
MergeJoiner/bothSidesRepeat-rows=1024-8   1.52GB/s ± 0%  1.54GB/s ± 0%  +0.88%  (p=0.000 n=8+9)
MergeJoiner/bothSidesRepeat-rows=4096-8   1.38GB/s ± 1%  1.41GB/s ± 1%  +1.61%  (p=0.000 n=10+9)
MergeJoiner/bothSidesRepeat-rows=16384-8   738MB/s ± 2%   753MB/s ± 0%  +2.07%  (p=0.000 n=10+10)
MergeJoiner/bothSidesRepeat-rows=32768-8   431MB/s ± 2%   437MB/s ± 1%  +1.26%  (p=0.001 n=10+10)

name                                      old alloc/op   new alloc/op   delta
MergeJoiner/rows=1024-8                      7.40B ±32%     6.20B ±45%    ~     (p=0.370 n=10+10)
MergeJoiner/rows=4096-8                      27.0B ± 0%     27.0B ± 0%    ~     (all equal)
MergeJoiner/rows=16384-8                      104B ±31%       90B ± 0%    ~     (p=0.211 n=10+10)
MergeJoiner/rows=1048576-8                  5.45kB ± 0%    5.45kB ± 0%    ~     (all equal)
MergeJoiner/oneSideRepeat-rows=1024-8        9.00B ± 0%     9.00B ± 0%    ~     (all equal)
MergeJoiner/oneSideRepeat-rows=4096-8        27.0B ± 0%     27.0B ± 0%    ~     (all equal)
MergeJoiner/oneSideRepeat-rows=16384-8       90.0B ± 0%     90.0B ± 0%    ~     (all equal)
MergeJoiner/oneSideRepeat-rows=1048576-8    6.90kB ±32%    5.45kB ± 0%    ~     (p=0.087 n=10+10)
MergeJoiner/bothSidesRepeat-rows=1024-8      9.00B ± 0%     9.00B ± 0%    ~     (all equal)
MergeJoiner/bothSidesRepeat-rows=4096-8      27.0B ± 0%     27.0B ± 0%    ~     (all equal)
MergeJoiner/bothSidesRepeat-rows=16384-8      272B ± 0%      272B ± 0%    ~     (all equal)
MergeJoiner/bothSidesRepeat-rows=32768-8      908B ± 0%      908B ± 0%    ~     (all equal)

name                                      old allocs/op  new allocs/op  delta
MergeJoiner/rows=1024-8                       0.00           0.00         ~     (all equal)
MergeJoiner/rows=4096-8                       0.00           0.00         ~     (all equal)
MergeJoiner/rows=16384-8                      0.00           0.00         ~     (all equal)
MergeJoiner/rows=1048576-8                    1.00 ± 0%      1.00 ± 0%    ~     (all equal)
MergeJoiner/oneSideRepeat-rows=1024-8         0.00           0.00         ~     (all equal)
MergeJoiner/oneSideRepeat-rows=4096-8         0.00           0.00         ~     (all equal)
MergeJoiner/oneSideRepeat-rows=16384-8        0.00           0.00         ~     (all equal)
MergeJoiner/oneSideRepeat-rows=1048576-8      1.40 ±43%      1.00 ± 0%    ~     (p=0.087 n=10+10)
MergeJoiner/bothSidesRepeat-rows=1024-8       0.00           0.00         ~     (all equal)
MergeJoiner/bothSidesRepeat-rows=4096-8       0.00           0.00         ~     (all equal)
MergeJoiner/bothSidesRepeat-rows=16384-8      0.00           0.00         ~     (all equal)
MergeJoiner/bothSidesRepeat-rows=32768-8      0.00           0.00         ~     (all equal)
```

Release note: None

36934: testutils: add a helper for removing setup code from -memprofile r=tbg a=danhhz

AllocProfileDiff writes two alloc profiles, one before running closure and
one after. This is similar in spirit to passing the -memprofile flag to a
test or benchmark, but make it possible to subtract out setup code, which
-memprofile does not.

Example usage:
    setupCode()
    AllocProfileDiff(t, "mem.before", "mem.after", func() {
      interestingCode()
    })

The resulting profiles are then diffed via:
    go tool pprof -base mem.before mem.after

I've wanted this a number of times when working on bulkio stuff, but it
was recently so useful that I decided to clean it up and check it in.

The result of -memprofile on BenchmarkImportWorkload/AddSSTable:

    (pprof) top10
    Showing nodes accounting for 1921.59MB, 74.06% of 2594.70MB total
    Dropped 496 nodes (cum <= 12.97MB)
    Showing top 10 nodes out of 119
          flat  flat%   sum%        cum   cum%
      463.28MB 17.85% 17.85%   463.28MB 17.85%  github.com/cockroachdb/cockroach/pkg/ccl/workloadccl/format.ToSSTable.func2
      266.46MB 10.27% 28.12%   415.37MB 16.01%  github.com/cockroachdb/cockroach/vendor/github.com/golang/leveldb/table.(*Reader).readBlock
      256.60MB  9.89% 38.01%   289.10MB 11.14%  github.com/cockroachdb/cockroach/pkg/sql.GenerateInsertRow
      241.35MB  9.30% 47.32%   241.35MB  9.30%  github.com/cockroachdb/cockroach/pkg/storage.defaultSubmitProposalLocked
      239.81MB  9.24% 56.56%   239.81MB  9.24%  github.com/cockroachdb/cockroach/pkg/storage/storagepb.(*ReplicatedEvalResult_AddSSTable).Unmarshal
      148.91MB  5.74% 62.30%   148.91MB  5.74%  github.com/cockroachdb/cockroach/vendor/github.com/golang/snappy.Decode
       79.94MB  3.08% 65.38%    79.94MB  3.08%  github.com/cockroachdb/cockroach/pkg/storage/engine.gobytes
       78.72MB  3.03% 68.41%    78.72MB  3.03%  github.com/cockroachdb/cockroach/pkg/ccl/importccl.(*rowConverter).sendBatch
       77.51MB  2.99% 71.40%    84.61MB  3.26%  github.com/cockroachdb/cockroach/pkg/workload/tpcc.(*tpcc).tpccOrderLineInitialRowBatch
       69.02MB  2.66% 74.06%    69.02MB  2.66%  github.com/cockroachdb/cockroach/pkg/roachpb.(*Value).SetTuple

The result of using this and diffing the profiles:

    (pprof) top10
    Showing nodes accounting for 299.54MB, 99.02% of 302.50MB total
    Dropped 9 nodes (cum <= 1.51MB)
    Showing top 10 nodes out of 66
          flat  flat%   sum%        cum   cum%
          95MB 31.40% 31.40%   139.67MB 46.17%  github.com/cockroachdb/cockroach/vendor/github.com/golang/leveldb/table.(*Reader).readBlock
       79.94MB 26.43% 57.83%    79.94MB 26.43%  github.com/cockroachdb/cockroach/pkg/storage.defaultSubmitProposalLocked
       79.94MB 26.43% 84.26%    79.94MB 26.43%  github.com/cockroachdb/cockroach/pkg/storage/storagepb.(*ReplicatedEvalResult_AddSSTable).Unmarshal
       44.67MB 14.77% 99.02%    44.67MB 14.77%  github.com/cockroachdb/cockroach/vendor/github.com/golang/snappy.Decode
             0     0% 99.02%     2.45MB  0.81%  github.com/cockroachdb/cockroach/pkg/ccl/importccl_test.BenchmarkImportWorkload.func1.1
             0     0% 99.02%     2.45MB  0.81%  github.com/cockroachdb/cockroach/pkg/ccl/importccl_test.benchmarkAddSSTable
             0     0% 99.02%    63.22MB 20.90%  github.com/cockroachdb/cockroach/pkg/ccl/importccl_test.benchmarkAddSSTable.func1
             0     0% 99.02%    79.94MB 26.43%  github.com/cockroachdb/cockroach/pkg/internal/client.(*CrossRangeTxnWrapperSender).Send
             0     0% 99.02%    79.94MB 26.43%  github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).AddSSTable
             0     0% 99.02%    79.94MB 26.43%  github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).Run

Release note: None

Co-authored-by: George Utsin <[email protected]>
Co-authored-by: Daniel Harrison <[email protected]>
  • Loading branch information
3 people committed Apr 22, 2019
3 parents aa799d3 + a35a60c + 73d6bba commit 47168df
Show file tree
Hide file tree
Showing 5 changed files with 202 additions and 25 deletions.
52 changes: 45 additions & 7 deletions pkg/sql/exec/coldata/vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,6 +127,10 @@ type Nulls interface {
// SetNull64 takes in a uint64 and sets the ith value of the column to null.
SetNull64(i uint64)

// SetNullRange takes a start uint64 and an end uint64 index, and sets all the values
// in [start, end) to null.
SetNullRange(start uint64, end uint64)

// UnsetNulls sets the column to have 0 null values.
UnsetNulls()
// SetNulls sets the column to have only null values.
Expand All @@ -137,16 +141,19 @@ var _ Vec = &memColumn{}

// zeroedNulls is a zeroed out slice representing a bitmap of size BatchSize.
// This is copied to efficiently clear a nulls slice.
var zeroedNulls [(BatchSize-1)>>6 + 1]int64
var zeroedNulls [(BatchSize-1)>>6 + 1]uint64

// filledNulls is a slice representing a bitmap of size BatchSize with every
// single bit set.
var filledNulls [(BatchSize-1)>>6 + 1]int64
var filledNulls [(BatchSize-1)>>6 + 1]uint64

// onesMask is a max uint64, where every bit is set to 1.
const onesMask = ^uint64(0)

func init() {
// Initializes filledNulls to the desired slice.
for i := range filledNulls {
filledNulls[i] = ^0
filledNulls[i] = onesMask
}
}

Expand All @@ -155,18 +162,18 @@ func init() {
type memColumn struct {
col column

nulls []int64
nulls []uint64
// hasNulls represents whether or not the memColumn has any null values set.
hasNulls bool
}

// NewMemColumn returns a new memColumn, initialized with a length.
func NewMemColumn(t types.T, n int) Vec {
var nulls []int64
var nulls []uint64
if n > 0 {
nulls = make([]int64, (n-1)>>6+1)
nulls = make([]uint64, (n-1)>>6+1)
} else {
nulls = make([]int64, 0)
nulls = make([]uint64, 0)
}

switch t {
Expand Down Expand Up @@ -238,6 +245,37 @@ func (m *memColumn) SetNull64(i uint64) {
m.nulls[intIdx] |= 1 << (i % 64)
}

func (m *memColumn) SetNullRange(start uint64, end uint64) {
if start >= end {
return
}

m.hasNulls = true
sIdx := start >> 6
eIdx := end >> 6

// Case where mask only spans one uint64.
if sIdx == eIdx {
mask := onesMask << (start % 64)
mask = mask & (onesMask >> (64 - (end % 64)))
m.nulls[sIdx] |= mask
return
}

// Case where mask spans at least two uint64s.
if sIdx < eIdx {
mask := onesMask << (start % 64)
m.nulls[sIdx] |= mask

mask = onesMask >> (64 - (end % 64))
m.nulls[eIdx] |= mask

for i := sIdx + 1; i < eIdx; i++ {
m.nulls[i] |= onesMask
}
}
}

func (m *memColumn) Bool() []bool {
return m.col.([]bool)
}
Expand Down
73 changes: 73 additions & 0 deletions pkg/sql/exec/coldata/vec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,3 +60,76 @@ func TestMemColumnSlice(t *testing.T) {
}
}
}

func TestNullRanges(t *testing.T) {
tcs := []struct {
start uint64
end uint64
}{
{
start: 1,
end: 1,
},
{
start: 50,
end: 0,
},
{
start: 0,
end: 50,
},
{
start: 0,
end: 64,
},
{
start: 25,
end: 50,
},
{
start: 0,
end: 80,
},
{
start: 20,
end: 80,
},
{
start: 0,
end: 387,
},
{
start: 385,
end: 387,
},
{
start: 0,
end: 1023,
},
{
start: 1022,
end: 1023,
}, {
start: 1023,
end: 1023,
},
}

c := NewMemColumn(types.Int64, BatchSize)
for _, tc := range tcs {
c.UnsetNulls()
c.SetNullRange(tc.start, tc.end)

for i := uint64(0); i < BatchSize; i++ {
if i >= tc.start && i < tc.end {
if !c.NullAt64(i) {
t.Fatalf("expected null at %d, start: %d end: %d", i, tc.start, tc.end)
}
} else {
if c.NullAt64(i) {
t.Fatalf("expected non-null at %d, start: %d end: %d", i, tc.start, tc.end)
}
}
}
}
}
12 changes: 6 additions & 6 deletions pkg/sql/exec/coldata/vec_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func (m *memColumn) Append(vec Vec, colType types.T, toLength uint64, fromLength
}

if fromLength > 0 {
m.nulls = append(m.nulls, make([]int64, (fromLength-1)>>6+1)...)
m.nulls = append(m.nulls, make([]uint64, (fromLength-1)>>6+1)...)

if vec.HasNulls() {
for i := uint16(0); i < fromLength; i++ {
Expand Down Expand Up @@ -106,7 +106,7 @@ func (m *memColumn) AppendWithSel(
}

if batchSize > 0 {
m.nulls = append(m.nulls, make([]int64, (batchSize-1)>>6+1)...)
m.nulls = append(m.nulls, make([]uint64, (batchSize-1)>>6+1)...)
for i := uint16(0); i < batchSize; i++ {
if vec.NullAt(sel[i]) {
m.SetNull64(toLength + uint64(i))
Expand Down Expand Up @@ -254,7 +254,7 @@ func (m *memColumn) Slice(colType types.T, start uint64, end uint64) Vec {
// {{range .}}
case _TYPES_T:
col := m._TemplateType()
var nulls []int64
var nulls []uint64
if m.hasNulls {
mod := start % 64
startIdx := start >> 6
Expand All @@ -267,7 +267,7 @@ func (m *memColumn) Slice(colType types.T, start uint64, end uint64) Vec {
// If start is not a multiple of 64, we need to shift over the bitmap
// to have the first index correspond. Allocate new null bitmap as we
// want to keep the original bitmap safe for reuse.
nulls = make([]int64, len(nulls))
nulls = make([]uint64, len(nulls))
for i, j := startIdx, 0; i < endIdx-1; i, j = i+1, j+1 {
// Bring the first null to the beginning.
nulls[j] = m.nulls[i] >> mod
Expand Down Expand Up @@ -310,7 +310,7 @@ func (m *memColumn) ExtendNulls(vec Vec, destStartIdx uint64, srcStartIdx uint16
if uint64(cap(m.nulls)) < outputLen/64 {
// (batchSize-1)>>6+1 is the number of Int64s needed to encode the additional elements/nulls in the Vec.
// This is equivalent to ceil(batchSize/64).
m.nulls = append(m.nulls, make([]int64, (toAppend-1)>>6+1)...)
m.nulls = append(m.nulls, make([]uint64, (toAppend-1)>>6+1)...)
}
if vec.HasNulls() {
for i := uint16(0); i < toAppend; i++ {
Expand All @@ -330,7 +330,7 @@ func (m *memColumn) ExtendNullsWithSel(
if uint64(cap(m.nulls)) < outputLen/64 {
// (batchSize-1)>>6+1 is the number of Int64s needed to encode the additional elements/nulls in the Vec.
// This is equivalent to ceil(batchSize/64).
m.nulls = append(m.nulls, make([]int64, (toAppend-1)>>6+1)...)
m.nulls = append(m.nulls, make([]uint64, (toAppend-1)>>6+1)...)
}
for i := uint16(0); i < toAppend; i++ {
// TODO(yuzefovich): this can be done more efficiently with a bitwise OR:
Expand Down
24 changes: 12 additions & 12 deletions pkg/sql/exec/mergejoiner_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -299,18 +299,19 @@ func _LEFT_SWITCH(isSel bool, hasNulls bool) { // */}}
}
// Loop over every row in the group.
for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ {
srcStartIdx := o.builderState.left.curSrcStartIdx
// Repeat each row numRepeats times.
srcStartIdx := o.builderState.left.curSrcStartIdx
// {{ if $.HasNulls }}
toAppend := leftGroup.numRepeats
if outStartIdx+toAppend > int(o.outputBatchSize) {
toAppend = int(o.outputBatchSize) - outStartIdx
}
if src.NullAt64(uint64(srcStartIdx)) {
out.SetNullRange(uint64(outStartIdx), uint64(outStartIdx+toAppend))
}
// {{ end }}
for ; o.builderState.left.numRepeatsIdx < leftGroup.numRepeats; o.builderState.left.numRepeatsIdx++ {
if outStartIdx < o.outputBatchSize {

// {{ if $.HasNulls }}
// TODO (georgeutsin): create a SetNullRange(start, end) function in coldata.Nulls,
// and place this outside the tight loop.
if src.NullAt64(uint64(srcStartIdx)) {
out.SetNull64(uint64(outStartIdx))
}
// {{ end }}
if outStartIdx < int(o.outputBatchSize) {

// {{ if $.IsSel }}
// TODO (georgeutsin): update template language to automatically generate template
Expand Down Expand Up @@ -381,13 +382,12 @@ func (o *mergeJoinOp) buildLeftGroups(
) {
o.builderState.left.finished = false
sel := bat.Selection()
outStartIdx := destStartIdx
initialBuilderState := o.builderState.left
// Loop over every column.
LeftColLoop:
for ; o.builderState.left.colIdx < len(input.outCols); o.builderState.left.colIdx++ {
colIdx := input.outCols[o.builderState.left.colIdx]
outStartIdx = destStartIdx
outStartIdx := int(destStartIdx)
out := o.output.ColVec(int(colIdx))
src := bat.ColVec(int(colIdx))
colType := input.sourceTypes[colIdx]
Expand Down
66 changes: 66 additions & 0 deletions pkg/testutils/pprof.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
// Copyright 2019 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.

package testutils

import (
"os"
"runtime"
"runtime/pprof"
"testing"
)

// WriteProfile serialized the pprof profile with the given name to a file at
// the given path.
func WriteProfile(t testing.TB, name string, path string) {
f, err := os.Create(path)
if err != nil {
t.Fatal(err)
}
defer f.Close()
if err := pprof.Lookup(name).WriteTo(f, 0); err != nil {
t.Fatal(err)
}
}

// AllocProfileDiff writes two alloc profiles, one before running closure and
// one after. This is similar in spirit to passing the -memprofile flag to a
// test or benchmark, but make it possible to subtract out setup code, which
// -memprofile does not.
//
// Example usage:
// setupCode()
// AllocProfileDiff(t, "mem.before", "mem.after", func() {
// interestingCode()
// })
//
// The resulting profiles are then diffed via:
// go tool pprof -base mem.before mem.after
func AllocProfileDiff(t testing.TB, beforePath, afterPath string, fn func()) {
// Use "allocs" instead of "heap" to match what -memprofile does. Also run
// runtime.GC immediately before grabbing the profile because the allocs
// profile is materialized on gc, so this makes sure we have the latest data.
//
// https://github.com/golang/go/blob/go1.12.4/src/testing/testing.go#L1264-L1269
runtime.GC()
WriteProfile(t, "allocs", beforePath)
fn()
runtime.GC()
WriteProfile(t, "allocs", afterPath)
t.Logf("to use your alloc profiles: go tool pprof -base %s %s", beforePath, afterPath)
}

// Make the unused linter happy.
var _ = WriteProfile
var _ = AllocProfileDiff

0 comments on commit 47168df

Please sign in to comment.