Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
53529: sql: fix limits on virtual index scans r=jordanlewis a=jordanlewis

Previously, virtual indexes mis-reported their key columns as containing
only their index key. Virtual indexes always need to contain the dummy
PK column since they're not guaranteed to be unique. This caused some
broken behavior like limits not working properly in some cases.

Fixes #53522 

Release note (bug fix): virtual index scans respect limits properly as
they did in 20.1 and prior.

Release justification: bug fix to new functionality

53572: roachtest: Bump up node/SSD size for clearrange roachtests r=itsbilal a=itsbilal

For the clearrange/* roachtests, the instances we spun up on aws
(c5d.xlarge) had 4x less local ssd storage space as the ones we spun
up on gce. This extra headroom made a pretty significant difference,
as the bigbank workload import would easily exceed the ~900gb usable capacity
on a 10x c5d.xlarge node cluster (it was observed to take around 1.9TB on
GCE).

This change bumps up the instance size indirectly, by requiring more CPUs
which is how the instance type is determined.

A related, though less pressing change, is to allow for increased concurrency
in AddSSTable requests to make the import go faster.

Release justification: Roachtest change, doesn't affect cockroach binary
Release note: None.

53585: coldata: optimize copying nulls in a common case r=yuzefovich a=yuzefovich

Whenever we're copying `coldata.Vec`s, we need to copy the nulls
vectors as well. Often, we would be copying the nulls vectors in such
a manner that the shifts within nulls bitmaps would be the same. Such
observation allows us to optimize the copying behavior (previously, we
would be checking and setting every null value one at a time), but now
we will do a more efficient copy for the largest central part of the
range to copy and handle the "head" and the "tail" of the range
separately. This gives us a nice speedup (e.g. on a quick projection
operator benchmark I see 2x improvement in the throughput).

Release justification: low risk, high benefit change.

Release note: None

53588: coldata,sql: remove some todos r=yuzefovich a=yuzefovich

This commit removes several TODOs that I have prototyped addressing and
decided to abandon the prototypes, namely:
- checking whether `coldata.BatchSize()` atomic has influence on
performance (the benchmarks and TPCH queries showed that the impact is
negligible if any)
- tuning default batch size (I did that a while ago, and the best
batch size according tpchvec/bench was 1280, barely better than current
1024 which is a lot nicer number)
- pooling allocations of `execFactory` objects (this showed some
improvement on one workload and a regression on another).

Release justification: non-production code changes.

Release note: None

53608: scripts: add --hide-crdb-folk option to release-notes r=jordanlewis a=jordanlewis

Add an option to the release notes script that generates all of the commits made by non-crdb people over the last time duration.

Release note: None
Release justification: non-code change

53645: builtins: implement ST_Collect aggregate r=otan a=erikgrinaker

Also contains `geomfn.Collect()` for the binary version of `ST_Collect` (#48902), but the builtin could not be implemented since it cannot have the same name as an aggregate function. The code is left behind for when this constraint is lifted, but I'm happy to remove it entirely if preferred.

The aggregation optimistically uses a multitype, but once it encounters an incompatible geometry it converts the existing multitype into a collection, using additional memory and CPU time - ideas for how to avoid this are welcome.

Test coverage is less than great, but I could not see any existing convention for comprehensive testing of aggregate functions.

Release justification: low risk, high benefit changes to existing
functionality

Release note (sql change): Implement the geometry aggregate builtin
ST_Collect.

Closes #48903.

53674: sql: de-experimentalize user defined schemas r=rohany a=rohany

Release justification: low-risk update to new functionality
Release note (sql change): De-experimentalize user defined schemas.

53681: stats: fix and unskip flaky test TestCreateStatsProgress r=rytaft a=rytaft

Release justification: non-production code changes

This commit fixes the flaky test `TestCreateStatsProgress` and unskips
it. `TestCreateStatsProgress` was flaky because of the recent changes to
the stats cache, which removed the guarantee that fresh stats would be
available in the cache immediately after stats creation. This commit
fixes the issue by explicitly invalidating the stats cache before the
part of `TestCreateStatsProgress` that expects certain stats to be present.

Fixes #52782

Release note: None

53682: sql: volatility for casts between tuples r=RaduBerinde a=RaduBerinde

#### sql: volatility for casts between tuples

We were missing code for determining volatility when casting between tuples.
This is because there is no way to express such a cast directly in SQL, but
there are cases where these casts appear implicitly (such as a conditional).

This change adds the missing logic and a corresponding test.

Fixes #53395.

Release justification: low-risk fix of regression.

Release note (bug fix): fixed an internal error related to casts between tuples.

Co-authored-by: Jordan Lewis <[email protected]>
Co-authored-by: Bilal Akhtar <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Rohan Yadav <[email protected]>
Co-authored-by: Rebecca Taft <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
8 people committed Aug 31, 2020
10 parents 410616f + 4aa891b + ac2ab9e + 1270ec1 + 34c79fc + 9fcca21 + f3bc7d4 + 3f99874 + 391b47d + e2285ee commit 42e73f7
Show file tree
Hide file tree
Showing 50 changed files with 984 additions and 566 deletions.
22 changes: 12 additions & 10 deletions AUTHORS
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
Aaron Blum <[email protected]>
Aaron Dunnington <[email protected]> aarondunnington <[email protected]>
Aayush Shah <[email protected]> <@cockroachlabs.com>
Abby Hersh <[email protected]>
Abhishek Madan <[email protected]> <[email protected]> <[email protected]> Abhemailk [email protected] <[email protected]>
Abhishek Soni <[email protected]>
Abhishek Saha <[email protected]> AbhishekSaha <[email protected]>
Expand All @@ -39,20 +40,23 @@ Amos Bird <[email protected]>
Amruta Ranade <[email protected]> Amruta <[email protected]> <[email protected]>
Anantha Krishnan <[email protected]> Ananthakrishnan <[email protected]>
Andrei Matei <[email protected]> <[email protected]>
Andrew B. Goode <[email protected]> nexdrew <[email protected]>
Andrew Bonventre <[email protected]> <[email protected]>
Andrew Couch <[email protected]> <[email protected]> <[email protected]>
Andy Kimball <[email protected]> <[email protected]> <[email protected]> Andrew Kimball <[email protected]>
Andrew Kryczka <[email protected]> Andrew Kryczka <[email protected]> <@cockroachlabs.com>
Andrew NS Yeow <[email protected]>
Andrew Werner <[email protected]>
Andy Woods <[email protected]> Andrew Woods <[email protected]>
Andrey Shinkevich <[email protected]>
Andrii Vorobiov <[email protected]> <@cockroachlabs.com>
Andy Kimball <[email protected]> <[email protected]> <[email protected]> Andrew Kimball <[email protected]>
Andy Woods <[email protected]> Andrew Woods <[email protected]>
Angela Chang <[email protected]> changangela <[email protected]> <[email protected]>
Antoine Grondin <[email protected]>
Anzo Teh <[email protected]> anzoteh96 <[email protected]> <[email protected]>
Archer Zhang <[email protected]> azhng <[email protected]>
Arjun Ravi Narayan <[email protected]> <[email protected]> Arjun Narayan <[email protected]> <[email protected]>
Art Nikpal <[email protected]>
Artem Ervits <[email protected]> <[email protected]>
Arul Ajmani <[email protected]> <[email protected]>
Asit Mahato <[email protected]>
bc <[email protected]>
Expand Down Expand Up @@ -116,7 +120,7 @@ Gustav Paul <[email protected]>
Haines Chan <[email protected]> hainesc <[email protected]>
Harshit Chopra <[email protected]>
Hayden A. James <[email protected]>
Helen He <[email protected]>
Helen He <[email protected]> <@cockroachlabs.com>
Ibrahim AshShohail <[email protected]>
Igor Kharin <[email protected]>
il9ue <[email protected]>
Expand All @@ -140,6 +144,7 @@ Jesse Seldess <[email protected]> <[email protected]>
Jessica Edwards <[email protected]> <[email protected]>
Jiajia Han <[email protected]>
Jiangming Yang <[email protected]> jiangmingyang <[email protected]>
Jim Hatcher <[email protected]>
Jimmy Larsson <[email protected]>
Jincheng Li <[email protected]>
Jingguo Yao <[email protected]>
Expand All @@ -165,7 +170,7 @@ Juan Leon <[email protected]> <[email protected]>
Justin Jaffray <[email protected]> <[email protected]>
Karan Vaidya <[email protected]>
Karl Southern <[email protected]>
Kate Doebler <[email protected]>
Kate Doebler <[email protected]> katedoebler <[email protected]>
Kathy Spradlin <[email protected]>
Kenji Kaneda <[email protected]> <[email protected]>
Kenjiro Nakayama <[email protected]>
Expand Down Expand Up @@ -202,14 +207,14 @@ Matthew O'Connor <[email protected]> <[email protected]>
Max Lang <[email protected]> <[email protected]>
Mayank Oli <[email protected]>
mbonaci <[email protected]>
mike czabator <[email protected]>
Mo Firouz <[email protected]>
Mohamed Elqdusy <[email protected]>
Nate Stewart <[email protected]> Nate <[email protected]> <[email protected]>
Nathan Johnson <[email protected]>
Nathan VanBenschoten <[email protected]> <@cockroachlabs.com>
Nathan Stilwell <[email protected]>
neeral <[email protected]>
Andrew B. Goode <[email protected]> nexdrew <[email protected]>
ngaut <[email protected]> liuqi <[email protected]> goroutine <[email protected]> <[email protected]> <[email protected]>
Nick <[email protected]>
Nick Gottlieb <[email protected]>
Expand Down Expand Up @@ -271,7 +276,7 @@ thundercw <[email protected]>
Tim O'Brien <[email protected]> tim-o <[email protected]> <@cockroachlabs.com>
Tommy Truongchau <[email protected]> <[email protected]>
Timothy Chen <[email protected]>
Tobias Schottdorf <[email protected]> <[email protected]> <[email protected]> <@cockroachlabs.com>
Tobias Grieger <[email protected]> <[email protected]> <[email protected]> <tbg@cockroachlabs.com>
Tristan Ohlson <[email protected]> <@cockroachlabs.com>
Tristan Rice <[email protected]> <[email protected]>
Txiaozhe <[email protected]>
Expand Down Expand Up @@ -303,11 +308,8 @@ yuhit <[email protected]>
Yulei Xiao <[email protected]>
YZ Chin <[email protected]>
Rafael Yim <[email protected]> yznming <[email protected]>
Ryan Kuo <[email protected]> taroface <[email protected]>
Ryan Kuo <[email protected]> taroface <[email protected]> <[email protected]>
Zach Brock <[email protected]> <[email protected]>
Zachary Smith <[email protected]> Zachary.smith <[email protected]>
何羿宏 <[email protected]>
智雅楠 <[email protected]>
Abby Hersh <[email protected]>
Jim Hatcher <[email protected]>
mike czabator <[email protected]>
2 changes: 2 additions & 0 deletions docs/generated/sql/aggregates.md
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,8 @@
</span></td></tr>
<tr><td><a name="sqrdiff"></a><code>sqrdiff(arg1: <a href="int.html">int</a>) &rarr; <a href="decimal.html">decimal</a></code></td><td><span class="funcdesc"><p>Calculates the sum of squared differences from the mean of the selected values.</p>
</span></td></tr>
<tr><td><a name="st_collect"></a><code>st_collect(arg1: geometry) &rarr; geometry</code></td><td><span class="funcdesc"><p>Collects geometries into a GeometryCollection or multi-type as appropriate.</p>
</span></td></tr>
<tr><td><a name="st_extent"></a><code>st_extent(arg1: geometry) &rarr; box2d</code></td><td><span class="funcdesc"><p>Forms a Box2D that encapsulates all provided geometries.</p>
</span></td></tr>
<tr><td><a name="st_makeline"></a><code>st_makeline(arg1: geometry) &rarr; geometry</code></td><td><span class="funcdesc"><p>Forms a LineString from Point, MultiPoint or LineStrings. Other shapes will be ignored.</p>
Expand Down
13 changes: 1 addition & 12 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1729,7 +1729,7 @@ func TestRestoreFailCleanup(t *testing.T) {
// failed restore.
sqlDB.Exec(t, `CREATE TYPE data.myenum AS ENUM ('hello')`)
// Do the same with a user defined schema.
sqlDB.Exec(t, `SET experimental_enable_user_defined_schemas = true; USE data; CREATE SCHEMA myschema`)
sqlDB.Exec(t, `USE data; CREATE SCHEMA myschema`)

sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, LocalFoo)
// Bugger the backup by removing the SST files.
Expand Down Expand Up @@ -1823,7 +1823,6 @@ func TestBackupRestoreUserDefinedSchemas(t *testing.T) {

var ts1, ts2, ts3, ts4, ts5, ts6 string
sqlDB.Exec(t, `
SET experimental_enable_user_defined_schemas = true;
CREATE DATABASE d;
USE d;
Expand Down Expand Up @@ -1911,8 +1910,6 @@ CREATE TABLE sc.t1 (a FLOAT);
_, _, sqlDB, dataDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitNone)
defer cleanupFn()
sqlDB.Exec(t, `
SET experimental_enable_user_defined_schemas = true;
CREATE DATABASE d;
USE d;
CREATE SCHEMA unused;
Expand Down Expand Up @@ -1948,8 +1945,6 @@ INSERT INTO sc.tb2 VALUES ('hello');
defer cleanupFn()

sqlDB.Exec(t, `
SET experimental_enable_user_defined_schemas = true;
CREATE DATABASE d;
USE d;
CREATE SCHEMA sc;
Expand Down Expand Up @@ -1984,8 +1979,6 @@ INSERT INTO sc.tb2 VALUES ('hello');
defer cleanupFn()

sqlDB.Exec(t, `
SET experimental_enable_user_defined_schemas = true;
CREATE DATABASE d;
USE d;
CREATE SCHEMA sc;
Expand Down Expand Up @@ -2044,8 +2037,6 @@ INSERT INTO sc.tb2 VALUES (1);
kvDB := tc.Server(0).DB()

sqlDB.Exec(t, `
SET experimental_enable_user_defined_schemas = true;
CREATE DATABASE d1;
USE d1;
CREATE SCHEMA sc1;
Expand Down Expand Up @@ -2104,8 +2095,6 @@ INSERT INTO sc4.tb VALUES (4);
defer cleanupFn()

sqlDB.Exec(t, `
SET experimental_enable_user_defined_schemas = true;
CREATE DATABASE d;
USE d;
CREATE SCHEMA sc;
Expand Down
2 changes: 0 additions & 2 deletions pkg/ccl/backupccl/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,8 +39,6 @@ func TestShowBackup(t *testing.T) {
defer cleanupFn()
defer cleanupEmptyCluster()
sqlDB.Exec(t, `
SET experimental_enable_user_defined_schemas = true;
CREATE TYPE data.welcome AS ENUM ('hello', 'hi');
USE data; CREATE SCHEMA sc;
CREATE TABLE data.sc.t1 (a INT);
Expand Down
2 changes: 0 additions & 2 deletions pkg/ccl/importccl/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1930,7 +1930,6 @@ b STRING) CSV DATA (%s)`, testFiles.files[0])); err != nil {

t.Run("user-defined-schemas", func(t *testing.T) {
sqlDB.Exec(t, `CREATE DATABASE uds`)
sqlDB.Exec(t, `SET experimental_enable_user_defined_schemas = true`)
sqlDB.Exec(t, `USE uds`)
sqlDB.Exec(t, `CREATE SCHEMA sc`)
// Now import into a table under sc.
Expand Down Expand Up @@ -5475,7 +5474,6 @@ func TestImportAvro(t *testing.T) {
}

t.Run("user-defined-schemas", func(t *testing.T) {
sqlDB.Exec(t, `SET experimental_enable_user_defined_schemas = true`)
sqlDB.Exec(t, `CREATE SCHEMA myschema`)
sqlDB.Exec(t, `CREATE TABLE myschema.simple (i INT8 PRIMARY KEY, s text, b bytea)`)
sqlDB.Exec(t, `IMPORT INTO myschema.simple (i, s, b) AVRO DATA ($1)`, simpleOcf)
Expand Down
9 changes: 2 additions & 7 deletions pkg/cli/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -201,15 +201,10 @@ func runDump(cmd *cobra.Command, args []string) error {
// Dump schema create statements, if any. If connecting to a cockroach version
// before 20.2 the list of schemas will be empty, so nothing will be emitted.
if shouldDumpSchemas && dumpCtx.dumpMode != dumpDataOnly {
if len(schemas) > 0 {
if _, err := fmt.Fprintf(w, "SET experimental_enable_user_defined_schemas = true;\n"); err != nil {
for _, schema := range schemas {
if _, err := fmt.Fprintf(w, "CREATE SCHEMA %s;\n\n", tree.Name(schema)); err != nil {
return err
}
for _, schema := range schemas {
if _, err := fmt.Fprintf(w, "CREATE SCHEMA %s;\n\n", tree.Name(schema)); err != nil {
return err
}
}
}
}

Expand Down
2 changes: 0 additions & 2 deletions pkg/cli/testdata/dump/schemas
Original file line number Diff line number Diff line change
@@ -1,5 +1,4 @@
sql
SET experimental_enable_user_defined_schemas = true;
CREATE DATABASE d;
USE d;

Expand All @@ -17,7 +16,6 @@ INSERT 1
dump d
----
----
SET experimental_enable_user_defined_schemas = true;
CREATE SCHEMA sc1;

CREATE SCHEMA sc2;
Expand Down
11 changes: 10 additions & 1 deletion pkg/cmd/roachtest/clearrange.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/version"
)

func registerClearRange(r *testRegistry) {
Expand All @@ -28,7 +29,7 @@ func registerClearRange(r *testRegistry) {
// to <3:30h but it varies.
Timeout: 5*time.Hour + 90*time.Minute,
MinVersion: "v19.1.0",
Cluster: makeClusterSpec(10),
Cluster: makeClusterSpec(10, cpu(16)),
Run: func(ctx context.Context, t *test, c *cluster) {
runClearRange(ctx, t, c, checks)
},
Expand Down Expand Up @@ -68,6 +69,14 @@ func runClearRange(ctx context.Context, t *test, c *cluster, aggressiveChecks bo
t.Status(`restoring tiny table`)
defer t.WorkerStatus()

if t.buildVersion.AtLeast(version.MustParse("v19.2.0")) {
conn := c.Conn(ctx, 1)
if _, err := conn.ExecContext(ctx, `SET CLUSTER SETTING kv.bulk_io_write.concurrent_addsstable_requests = $1`, c.spec.NodeCount); err != nil {
t.Fatal(err)
}
conn.Close()
}

// Use a 120s connect timeout to work around the fact that the server will
// declare itself ready before it's actually 100% ready. See:
// https://github.com/cockroachdb/cockroach/issues/34897#issuecomment-465089057
Expand Down
9 changes: 5 additions & 4 deletions pkg/col/coldata/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,15 +72,16 @@ type Batch interface {

var _ Batch = &MemBatch{}

// TODO(jordan): tune.
// defaultBatchSize is the size of batches that is used in the non-test setting.
// Initially, 1024 was picked based on MonetDB/X100 paper and was later
// confirmed to be very good using tpchvec/bench benchmark on TPC-H queries
// (the best number according to that benchmark was 1280, but it was negligibly
// better, so we decided to keep 1024 as it is a power of 2).
const defaultBatchSize = 1024

var batchSize int64 = defaultBatchSize

// BatchSize is the maximum number of tuples that fit in a column batch.
// TODO(yuzefovich): we are treating this method almost as if it were a
// constant while it performs an atomic operation. Think through whether it has
// a noticeable performance hit.
func BatchSize() int {
return int(atomic.LoadInt64(&batchSize))
}
Expand Down
60 changes: 51 additions & 9 deletions pkg/col/coldata/nulls.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,8 +98,8 @@ func (n *Nulls) SetNullRange(startIdx int, endIdx int) {
n.nulls[eIdx] &= mask
}

for i := sIdx + 1; i < eIdx; i++ {
n.nulls[i] = 0
for idx := int(sIdx + 1); idx < int(eIdx); {
idx += copy(n.nulls[idx:eIdx], zeroedNulls[:])
}
}

Expand Down Expand Up @@ -138,8 +138,8 @@ func (n *Nulls) UnsetNullRange(startIdx, endIdx int) {
n.nulls[eIdx] |= mask
}

for i := sIdx + 1; i < eIdx; i++ {
n.nulls[i] = onesMask
for idx := int(sIdx + 1); idx < int(eIdx); {
idx += copy(n.nulls[idx:eIdx], filledNulls[:])
}
}

Expand Down Expand Up @@ -213,6 +213,18 @@ func (n *Nulls) swap(iIdx, jIdx int) {
n.nulls[j/8] = (n.nulls[j/8] & ^jMask) | (ni << (j % 8))
}

// setSmallRange is a helper that copies over a slice [startIdx, startIdx+toSet)
// of src and puts it into this nulls starting at destIdx.
func (n *Nulls) setSmallRange(src *Nulls, destIdx, startIdx, toSet int) {
for i := 0; i < toSet; i++ {
if src.NullAt(startIdx + i) {
n.SetNull(destIdx + i)
} else {
n.UnsetNull(destIdx + i)
}
}
}

// set copies over a slice [args.SrcStartIdx: args.SrcEndIdx] of
// args.Src.Nulls() and puts it into this nulls starting at args.DestIdx. If
// the length of this nulls is smaller than args.DestIdx, then this nulls is
Expand All @@ -230,26 +242,56 @@ func (n *Nulls) set(args SliceArgs) {
if current < needed {
n.nulls = append(n.nulls, filledNulls[:needed-current]...)
}
// First, we unset the whole range that is overwritten. If there are any NULL
// values in the source, those will be copied over below, one at a time.
n.UnsetNullRange(args.DestIdx, args.DestIdx+toDuplicate)
if args.Src.MaybeHasNulls() {
n.maybeHasNulls = true
src := args.Src.Nulls()
if args.Sel != nil {
// With the selection vector present, we can't do any smarts, so we
// unset the whole range that is overwritten and then set new null
// values one at a time.
n.UnsetNullRange(args.DestIdx, args.DestIdx+toDuplicate)
for i := 0; i < toDuplicate; i++ {
if src.NullAt(args.Sel[args.SrcStartIdx+i]) {
n.SetNull(args.DestIdx + i)
}
}
} else {
if toDuplicate > 16 && args.DestIdx%8 == args.SrcStartIdx%8 {
// We have a special (but a very common) case when we're
// copying a lot of elements, and the shifts within the nulls
// vectors for the destination and the source ranges are the
// same, so we can optimize the performance here.
// The fact that shifts are the same allows us to copy all
// elements as is (except for the first and the last which are
// handled separately).
dstStart := args.DestIdx / 8
srcStart := args.SrcStartIdx / 8
srcEnd := (args.SrcEndIdx-1)/8 + 1
// Since the first and the last elements might not be fully
// included in the range to be set, we're not touching them.
copy(n.nulls[dstStart+1:], src.nulls[srcStart+1:srcEnd-1])
// Handle the first element.
n.setSmallRange(src, args.DestIdx, args.SrcStartIdx, 8-args.DestIdx%8)
// Handle the last element.
toSet := (args.DestIdx + toDuplicate) % 8
if toSet == 0 {
toSet = 8
}
offset := toDuplicate - toSet
n.setSmallRange(src, args.DestIdx+offset, args.SrcStartIdx+offset, toSet)
return
}
n.UnsetNullRange(args.DestIdx, args.DestIdx+toDuplicate)
for i := 0; i < toDuplicate; i++ {
// TODO(yuzefovich): this can be done more efficiently with a bitwise OR:
// like n.nulls[i] |= vec.nulls[i].
if src.NullAt(args.SrcStartIdx + i) {
n.SetNull(args.DestIdx + i)
}
}
}
} else {
// No nulls in the source, so we unset the whole range that is
// overwritten.
n.UnsetNullRange(args.DestIdx, args.DestIdx+toDuplicate)
}
}

Expand Down
Loading

0 comments on commit 42e73f7

Please sign in to comment.